You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by rv...@apache.org on 2014/10/26 02:21:44 UTC

[01/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Repository: giraph
Updated Branches:
  refs/heads/release-1.1 065d718d7 -> 4c139ee36 (forced update)


GIRAPH-842: option to dump histogram of memory usage when heap is low on memory (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/372c35eb
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/372c35eb
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/372c35eb

Branch: refs/heads/release-1.1
Commit: 372c35ebccedf64266a38a35e365d3810642fef2
Parents: 2e8c2c6
Author: Pavan Kumar <pa...@fb.com>
Authored: Tue Jun 10 12:12:43 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Tue Jun 10 12:13:30 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../apache/giraph/conf/GiraphConfiguration.java |   9 ++
 .../org/apache/giraph/conf/GiraphConstants.java |  16 +++
 .../apache/giraph/master/BspServiceMaster.java  |   4 +
 .../giraph/utils/ReactiveJMapHistoDumper.java   | 120 +++++++++++++++++++
 .../apache/giraph/worker/BspServiceWorker.java  |   4 +
 6 files changed, 155 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/372c35eb/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index bdc0c10..ec1f798 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-842: option to dump histogram of memory usage when heap is low on memory (pavanka)
+
   GIRAPH-904: Giraph can hang when hostnames include uppercase letters (netj via pavanka)
 
   GIRAPH-900: Remove timers for compute method (majakabiljo via pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/372c35eb/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 d764307..b6384dc 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
@@ -335,6 +335,15 @@ public class GiraphConfiguration extends Configuration
   }
 
   /**
+   * Check whether to enable heap memory supervisor thread
+   *
+   * @return true if jmap dumper is reactively enabled
+   */
+  public boolean isReactiveJmapHistogramDumpEnabled() {
+    return REACTIVE_JMAP_ENABLE.get(this);
+  }
+
+  /**
    * Set mapping from a key name to a list of classes.
    *
    * @param name String key name to use.

http://git-wip-us.apache.org/repos/asf/giraph/blob/372c35eb/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 dd0c9ae..7d7ceb2 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
@@ -443,6 +443,22 @@ public interface GiraphConstants {
           "Only print live objects in jmap?");
 
   /**
+   * Option used by ReactiveJMapHistoDumper to check for an imminent
+   * OOM in worker or master process
+   */
+  IntConfOption MIN_FREE_MBS_ON_HEAP =
+      new IntConfOption("giraph.heap.minFreeMb", 128, "Option used by " +
+          "worker and master observers to check for imminent OOM exception");
+  /**
+   * Option can be used to enable reactively dumping jmap histo when
+   * OOM is imminent
+   */
+  BooleanConfOption REACTIVE_JMAP_ENABLE =
+      new BooleanConfOption("giraph.heap.enableReactiveJmapDumping", false,
+          "Option to enable dumping jmap histogram reactively based on " +
+              "free memory on heap");
+
+  /**
    * Minimum percent of the maximum number of workers that have responded
    * in order to continue progressing. (float)
    */

http://git-wip-us.apache.org/repos/asf/giraph/blob/372c35eb/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index e367b94..ad7e045 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -54,6 +54,7 @@ import org.apache.giraph.metrics.ResetSuperstepMetricsObserver;
 import org.apache.giraph.metrics.SuperstepMetricsRegistry;
 import org.apache.giraph.metrics.WorkerSuperstepMetrics;
 import org.apache.giraph.utils.JMapHistoDumper;
+import org.apache.giraph.utils.ReactiveJMapHistoDumper;
 import org.apache.giraph.utils.ProgressableUtils;
 import org.apache.giraph.time.SystemTime;
 import org.apache.giraph.time.Time;
@@ -215,6 +216,9 @@ public class BspServiceMaster<I extends WritableComparable,
     if (conf.isJMapHistogramDumpEnabled()) {
       conf.addMasterObserverClass(JMapHistoDumper.class);
     }
+    if (conf.isReactiveJmapHistogramDumpEnabled()) {
+      conf.addMasterObserverClass(ReactiveJMapHistoDumper.class);
+    }
     observers = conf.createMasterObservers();
 
     GiraphMetrics.get().addSuperstepResetObserver(this);

http://git-wip-us.apache.org/repos/asf/giraph/blob/372c35eb/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java b/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
new file mode 100644
index 0000000..68369d9
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
@@ -0,0 +1,120 @@
+/*
+ * 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.utils;
+
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.master.MasterObserver;
+import org.apache.giraph.worker.WorkerObserver;
+import org.apache.log4j.Logger;
+
+/**
+ * An observer for both worker and master that periodically checks if available
+ * memory on heap is below certain threshold, and if found to be the case
+ * dumps jmap -histo for the process
+ */
+public class ReactiveJMapHistoDumper extends
+  DefaultImmutableClassesGiraphConfigurable implements
+  MasterObserver, WorkerObserver {
+  /** Logger */
+  private static final Logger LOG = Logger.getLogger(
+      ReactiveJMapHistoDumper.class);
+  /** Size of mb */
+  private static final int MB = 1024 * 1024;
+
+  /** How many msec to sleep between calls */
+  private int sleepMillis;
+  /** How many lines of output to print */
+  private int linesToPrint;
+  /** How much free memory is expected */
+  private int minFreeMemory;
+
+  /** The jmap printing thread */
+  private Thread thread;
+  /** Halt jmap thread */
+  private volatile boolean stop = false;
+
+  @Override
+  public void preApplication() {
+    // This is called by both WorkerObserver and MasterObserver
+    startSupervisorThread();
+  }
+
+  @Override
+  public void postApplication() {
+    // This is called by both WorkerObserver and MasterObserver
+    joinSupervisorThread();
+  }
+
+  /**
+   * Join the supervisor thread
+   */
+  private void joinSupervisorThread() {
+    stop = true;
+    try {
+      thread.join(sleepMillis + 5000);
+    } catch (InterruptedException e) {
+      LOG.error("Failed to join jmap thread");
+    }
+  }
+
+  /**
+   * Start the supervisor thread
+   */
+  public void startSupervisorThread() {
+    stop = false;
+    final Runtime runtime = Runtime.getRuntime();
+    thread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          while (!stop) {
+            long potentialMemory = (runtime.maxMemory() -
+                runtime.totalMemory()) + runtime.freeMemory();
+            if (potentialMemory / MB < minFreeMemory) {
+              JMap.heapHistogramDump(linesToPrint);
+            }
+            Thread.sleep(sleepMillis);
+          }
+        } catch (InterruptedException e) {
+          LOG.warn("JMap histogram sleep interrupted", e);
+        }
+      }
+    });
+    thread.setName("ReactiveJMapHistoDumperSupervisorThread");
+    thread.start();
+  }
+
+  @Override
+  public void preSuperstep(long superstep) { }
+
+  @Override
+  public void postSuperstep(long superstep) { }
+
+  @Override
+  public void applicationFailed(Exception e) { }
+
+  @Override
+  public void setConf(ImmutableClassesGiraphConfiguration configuration) {
+    sleepMillis = GiraphConstants.JMAP_SLEEP_MILLIS.get(configuration);
+    linesToPrint = GiraphConstants.JMAP_PRINT_LINES.get(configuration);
+    minFreeMemory = GiraphConstants.MIN_FREE_MBS_ON_HEAP.get(configuration);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/372c35eb/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index 104932c..a89da24 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -63,6 +63,7 @@ import org.apache.giraph.partition.PartitionStore;
 import org.apache.giraph.partition.WorkerGraphPartitioner;
 import org.apache.giraph.utils.CallableFactory;
 import org.apache.giraph.utils.JMapHistoDumper;
+import org.apache.giraph.utils.ReactiveJMapHistoDumper;
 import org.apache.giraph.utils.LoggerUtils;
 import org.apache.giraph.utils.MemoryUtils;
 import org.apache.giraph.utils.ProgressableUtils;
@@ -214,6 +215,9 @@ public class BspServiceWorker<I extends WritableComparable,
     if (conf.isJMapHistogramDumpEnabled()) {
       conf.addWorkerObserverClass(JMapHistoDumper.class);
     }
+    if (conf.isReactiveJmapHistogramDumpEnabled()) {
+      conf.addWorkerObserverClass(ReactiveJMapHistoDumper.class);
+    }
     observers = conf.createWorkerObservers();
 
     WorkerProgress.get().setTaskId(getTaskPartition());


[13/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-912: Support succinct representation of messages in messagestores (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/f31e9a32
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/f31e9a32
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/f31e9a32

Branch: refs/heads/release-1.1
Commit: f31e9a328d3b4f10906a10a8e69d2ae515e3aba0
Parents: 61cb37e
Author: Pavan Kumar <pa...@fb.com>
Authored: Wed Jul 9 17:08:48 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Wed Jul 9 17:08:48 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../giraph/comm/SendMessageToAllCache.java      | 308 -------------------
 .../giraph/comm/SendOneMessageToManyCache.java  | 306 ++++++++++++++++++
 .../java/org/apache/giraph/comm/ServerData.java |   2 +
 .../messages/AbstractListPerVertexStore.java    | 103 +++++++
 .../ByteArrayMessagesPerVertexStore.java        |  17 +-
 .../messages/InMemoryMessageStoreFactory.java   | 114 +++++--
 .../messages/MessageEncodeAndStoreType.java     |  59 ++++
 .../giraph/comm/messages/MessageStore.java      |  15 +
 .../comm/messages/OneMessagePerVertexStore.java |   5 +
 .../messages/PointerListMessagesIterable.java   | 105 +++++++
 .../messages/PointerListPerVertexStore.java     | 137 +++++++++
 .../comm/messages/SimpleMessageStore.java       |   4 +
 .../out_of_core/DiskBackedMessageStore.java     |   9 +
 .../primitives/IntByteArrayMessageStore.java    |   9 +
 .../primitives/IntFloatMessageStore.java        |   9 +
 .../primitives/LongByteArrayMessageStore.java   | 241 ---------------
 .../primitives/LongDoubleMessageStore.java      |   9 +
 .../long_id/LongAbstractListMessageStore.java   | 164 ++++++++++
 .../long_id/LongAbstractMessageStore.java       | 132 ++++++++
 .../long_id/LongByteArrayMessageStore.java      | 172 +++++++++++
 .../long_id/LongPointerListMessageStore.java    | 129 ++++++++
 .../primitives/long_id/package-info.java        |  22 ++
 .../NettyWorkerClientRequestProcessor.java      |   8 +-
 .../giraph/comm/netty/NettyWorkerServer.java    |   2 +-
 .../giraph/comm/requests/RequestType.java       |   6 +-
 .../SendWorkerOneMessageToManyRequest.java      | 156 ++++++++++
 .../SendWorkerOneToAllMessagesRequest.java      | 155 ----------
 .../apache/giraph/conf/GiraphConfiguration.java |  17 +-
 .../org/apache/giraph/conf/GiraphConstants.java |  20 +-
 .../utils/ByteArrayOneMessageToManyIds.java     | 105 +++++++
 .../giraph/utils/ByteArrayOneToAllMessages.java | 168 ----------
 .../utils/ByteStructVertexIdDataIterator.java   |   9 +
 .../ByteStructVertexIdMessageIterator.java      |  10 +
 .../utils/ExtendedByteArrayOutputBuffer.java    | 155 ++++++++++
 .../apache/giraph/utils/ExtendedDataOutput.java |   1 -
 .../utils/OneMessageToManyIdsIterator.java      | 143 +++++++++
 .../apache/giraph/utils/UnsafeArrayReads.java   |   2 +-
 .../org/apache/giraph/utils/UnsafeReads.java    |   2 +-
 .../utils/UnsafeReusableByteArrayInput.java     |  46 +++
 .../giraph/utils/VertexIdDataIterator.java      |   7 +
 .../giraph/utils/VertexIdMessageIterator.java   |  14 +
 .../org/apache/giraph/comm/RequestTest.java     |  14 +-
 .../TestLongDoublePrimitiveMessageStores.java   |   2 +-
 44 files changed, 2160 insertions(+), 955 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 13dfcd7..0263749 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-912: Support succinct representation of messages in messagestores (pavanka)
+
   GIRAPH-903: Detect crashes of Netty threads (edunov via pavanka)
 
   GIRAPH-925: Unit tests should pass even if zookeeper port not available (edunov via pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageToAllCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageToAllCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageToAllCache.java
deleted file mode 100644
index 60858ea..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageToAllCache.java
+++ /dev/null
@@ -1,308 +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.comm;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Iterator;
-
-import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.comm.netty.NettyWorkerClientRequestProcessor;
-import org.apache.giraph.comm.requests.SendWorkerMessagesRequest;
-import org.apache.giraph.comm.requests.SendWorkerOneToAllMessagesRequest;
-import org.apache.giraph.comm.requests.WritableRequest;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.partition.PartitionOwner;
-import org.apache.giraph.utils.ByteArrayOneToAllMessages;
-import org.apache.giraph.utils.VertexIdMessages;
-import org.apache.giraph.utils.ExtendedDataOutput;
-import org.apache.giraph.utils.PairList;
-import org.apache.giraph.worker.WorkerInfo;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.log4j.Logger;
-
-/**
- * Aggregates the messages to be sent to workers so they can be sent
- * in bulk.  Not thread-safe.
- *
- * @param <I> Vertex id
- * @param <M> Message data
- */
-public class SendMessageToAllCache<I extends WritableComparable,
-    M extends Writable> extends SendMessageCache<I, M> {
-  /** Class logger */
-  private static final Logger LOG =
-      Logger.getLogger(SendMessageToAllCache.class);
-  /** Cache serialized messages for each worker */
-  private final ByteArrayOneToAllMessages<I, M>[] oneToAllMsgCache;
-  /** Tracking one-to-all message sizes for each worker */
-  private final int[] oneToAllMsgSizes;
-  /** Reused byte array to serialize target ids on each worker */
-  private final ExtendedDataOutput[] idSerializer;
-  /** Reused int array to count target id distribution */
-  private final int[] idCounter;
-  /**
-   * Reused int array to record the partition id
-   * of the first target vertex id found on the worker.
-   */
-  private final int[] firstPartitionMap;
-  /** The WorkerInfo list */
-  private final WorkerInfo[] workerInfoList;
-
-  /**
-   * Constructor
-   *
-   * @param conf Giraph configuration
-   * @param serviceWorker Service worker
-   * @param processor NettyWorkerClientRequestProcessor
-   * @param maxMsgSize Max message size sent to a worker
-   */
-  public SendMessageToAllCache(ImmutableClassesGiraphConfiguration conf,
-      CentralizedServiceWorker<?, ?, ?> serviceWorker,
-      NettyWorkerClientRequestProcessor<I, ?, ?> processor,
-      int maxMsgSize) {
-    super(conf, serviceWorker, processor, maxMsgSize);
-    int numWorkers = getNumWorkers();
-    oneToAllMsgCache = new ByteArrayOneToAllMessages[numWorkers];
-    oneToAllMsgSizes = new int[numWorkers];
-    idSerializer = new ExtendedDataOutput[numWorkers];
-    // InitialBufferSizes is alo initialized based on the number of workers.
-    // As a result, initialBufferSizes is the same as idSerializer in length
-    int initialBufferSize = 0;
-    for (int i = 0; i < this.idSerializer.length; i++) {
-      initialBufferSize = getSendWorkerInitialBufferSize(i);
-      if (initialBufferSize > 0) {
-        // InitialBufferSizes is from super class.
-        // Each element is for one worker.
-        idSerializer[i] = conf.createExtendedDataOutput(initialBufferSize);
-      }
-    }
-    idCounter = new int[numWorkers];
-    firstPartitionMap = new int[numWorkers];
-    // Get worker info list.
-    workerInfoList = new WorkerInfo[numWorkers];
-    // Remember there could be null in the array.
-    for (WorkerInfo workerInfo : serviceWorker.getWorkerInfoList()) {
-      workerInfoList[workerInfo.getTaskId()] = workerInfo;
-    }
-  }
-
-  /**
-   * Reset ExtendedDataOutput array for id serialization
-   * in next "one-to-all" message sending.
-   */
-  private void resetIdSerializers() {
-    for (int i = 0; i < this.idSerializer.length; i++) {
-      if (idSerializer[i] != null) {
-        idSerializer[i].reset();
-      }
-    }
-  }
-
-  /**
-   * Reset id counter for next "one-to-all" message sending.
-   */
-  private void resetIdCounter() {
-    Arrays.fill(idCounter, 0);
-  }
-
-  /**
-   * Add message with multiple ids to
-   * one-to-all message cache.
-   *
-   * @param workerInfo The remote worker destination
-   * @param ids A byte array to hold serialized vertex ids
-   * @param idPos The end position of ids
-   *              information in the byte array above
-   * @param count The number of target ids
-   * @param message Message to send to remote worker
-   * @return The size of messages for the worker.
-   */
-  private int addOneToAllMessage(
-    WorkerInfo workerInfo, byte[] ids, int idPos, int count, M message) {
-    // Get the data collection
-    ByteArrayOneToAllMessages<I, M> workerData =
-      oneToAllMsgCache[workerInfo.getTaskId()];
-    if (workerData == null) {
-      workerData = new ByteArrayOneToAllMessages<I, M>(
-        getConf().getOutgoingMessageValueFactory());
-      workerData.setConf(getConf());
-      workerData.initialize(getSendWorkerInitialBufferSize(
-        workerInfo.getTaskId()));
-      oneToAllMsgCache[workerInfo.getTaskId()] = workerData;
-    }
-    workerData.add(ids, idPos, count, message);
-    // Update the size of cached, outgoing data per worker
-    oneToAllMsgSizes[workerInfo.getTaskId()] =
-      workerData.getSize();
-    return oneToAllMsgSizes[workerInfo.getTaskId()];
-  }
-
-  /**
-   * Gets the one-to-all
-   * messages for a worker and removes it from the cache.
-   * Here the ByteArrayOneToAllMessages returned could be null.
-   * But when invoking this method, we also check if the data size sent
-   * to this worker is above the threshold. Therefore, it doesn't matter
-   * if the result is null or not.
-   *
-   * @param workerInfo The target worker where one-to-all messages
-   *                   go to.
-   * @return ByteArrayOneToAllMessages that belong to the workerInfo
-   */
-  private ByteArrayOneToAllMessages<I, M>
-  removeWorkerOneToAllMessages(WorkerInfo workerInfo) {
-    ByteArrayOneToAllMessages<I, M> workerData =
-      oneToAllMsgCache[workerInfo.getTaskId()];
-    if (workerData != null) {
-      oneToAllMsgCache[workerInfo.getTaskId()] = null;
-      oneToAllMsgSizes[workerInfo.getTaskId()] = 0;
-    }
-    return workerData;
-  }
-
-  /**
-   * Gets all the one-to-all
-   * messages and removes them from the cache.
-   *
-   * @return All vertex messages for all workers
-   */
-  private PairList<WorkerInfo, ByteArrayOneToAllMessages<I, M>>
-  removeAllOneToAllMessages() {
-    PairList<WorkerInfo, ByteArrayOneToAllMessages<I, M>> allData =
-      new PairList<WorkerInfo, ByteArrayOneToAllMessages<I, M>>();
-    allData.initialize(oneToAllMsgCache.length);
-    for (WorkerInfo workerInfo : getWorkerPartitions().keySet()) {
-      ByteArrayOneToAllMessages<I, M> workerData =
-        removeWorkerOneToAllMessages(workerInfo);
-      if (workerData != null && !workerData.isEmpty()) {
-        allData.add(workerInfo, workerData);
-      }
-    }
-    return allData;
-  }
-
-  @Override
-  public void sendMessageToAllRequest(Iterator<I> vertexIdIterator, M message) {
-    // This is going to be reused through every message sending
-    resetIdSerializers();
-    resetIdCounter();
-    // Count messages
-    int currentMachineId = 0;
-    PartitionOwner owner = null;
-    WorkerInfo workerInfo = null;
-    I vertexId = null;
-    while (vertexIdIterator.hasNext()) {
-      vertexId = vertexIdIterator.next();
-      owner = getServiceWorker().getVertexPartitionOwner(vertexId);
-      workerInfo = owner.getWorkerInfo();
-      currentMachineId = workerInfo.getTaskId();
-      // Serialize this target vertex id
-      try {
-        vertexId.write(idSerializer[currentMachineId]);
-      } catch (IOException e) {
-        throw new IllegalStateException(
-          "Failed to serialize the target vertex id.");
-      }
-      idCounter[currentMachineId]++;
-      // Record the first partition id in the worker which message send to.
-      // If idCounter shows there is only one target on this worker
-      // then this is the partition number of the target vertex.
-      if (idCounter[currentMachineId] == 1) {
-        firstPartitionMap[currentMachineId] = owner.getPartitionId();
-      }
-    }
-    // Add the message to the cache
-    int idSerializerPos = 0;
-    int workerMessageSize = 0;
-    byte[] serializedId  = null;
-    WritableRequest writableRequest = null;
-    for (int i = 0; i < idCounter.length; i++) {
-      if (idCounter[i] == 1) {
-        serializedId = idSerializer[i].getByteArray();
-        idSerializerPos = idSerializer[i].getPos();
-        // Add the message to the cache
-        workerMessageSize = addMessage(workerInfoList[i],
-          firstPartitionMap[i], serializedId, idSerializerPos, message);
-
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("sendMessageToAllRequest: Send bytes (" +
-            message.toString() + ") to one target in  worker " +
-            workerInfoList[i]);
-        }
-        ++totalMsgsSentInSuperstep;
-        if (workerMessageSize >= maxMessagesSizePerWorker) {
-          PairList<Integer, VertexIdMessages<I, M>>
-            workerMessages = removeWorkerMessages(workerInfoList[i]);
-          writableRequest =
-            new SendWorkerMessagesRequest<I, M>(workerMessages);
-          totalMsgBytesSentInSuperstep += writableRequest.getSerializedSize();
-          clientProcessor.doRequest(workerInfoList[i], writableRequest);
-          // Notify sending
-          getServiceWorker().getGraphTaskManager().notifySentMessages();
-        }
-      } else if (idCounter[i] > 1) {
-        serializedId = idSerializer[i].getByteArray();
-        idSerializerPos = idSerializer[i].getPos();
-        workerMessageSize = addOneToAllMessage(
-          workerInfoList[i], serializedId, idSerializerPos, idCounter[i],
-          message);
-
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("sendMessageToAllRequest: Send bytes (" +
-            message.toString() + ") to all targets in worker" +
-            workerInfoList[i]);
-        }
-        totalMsgsSentInSuperstep += idCounter[i];
-        if (workerMessageSize >= maxMessagesSizePerWorker) {
-          ByteArrayOneToAllMessages<I, M> workerOneToAllMessages =
-            removeWorkerOneToAllMessages(workerInfoList[i]);
-          writableRequest =
-            new SendWorkerOneToAllMessagesRequest<I, M>(
-              workerOneToAllMessages, getConf());
-          totalMsgBytesSentInSuperstep += writableRequest.getSerializedSize();
-          clientProcessor.doRequest(workerInfoList[i], writableRequest);
-          // Notify sending
-          getServiceWorker().getGraphTaskManager().notifySentMessages();
-        }
-      }
-    }
-  }
-
-  @Override
-  public void flush() {
-    super.flush();
-    PairList<WorkerInfo, ByteArrayOneToAllMessages<I, M>>
-    remainingOneToAllMessageCache =
-      removeAllOneToAllMessages();
-    PairList<WorkerInfo,
-    ByteArrayOneToAllMessages<I, M>>.Iterator
-    oneToAllMsgIterator = remainingOneToAllMessageCache.getIterator();
-    while (oneToAllMsgIterator.hasNext()) {
-      oneToAllMsgIterator.next();
-      WritableRequest writableRequest =
-        new SendWorkerOneToAllMessagesRequest<I, M>(
-          oneToAllMsgIterator.getCurrentSecond(), getConf());
-      totalMsgBytesSentInSuperstep += writableRequest.getSerializedSize();
-      clientProcessor.doRequest(
-        oneToAllMsgIterator.getCurrentFirst(), writableRequest);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/SendOneMessageToManyCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendOneMessageToManyCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendOneMessageToManyCache.java
new file mode 100644
index 0000000..c67a20b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendOneMessageToManyCache.java
@@ -0,0 +1,306 @@
+/*
+ * 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.comm;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.netty.NettyWorkerClientRequestProcessor;
+import org.apache.giraph.comm.requests.SendWorkerOneMessageToManyRequest;
+import org.apache.giraph.comm.requests.SendWorkerMessagesRequest;
+import org.apache.giraph.comm.requests.WritableRequest;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.partition.PartitionOwner;
+import org.apache.giraph.utils.ByteArrayOneMessageToManyIds;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.giraph.utils.ExtendedDataOutput;
+import org.apache.giraph.utils.PairList;
+import org.apache.giraph.worker.WorkerInfo;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.log4j.Logger;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+/**
+ * Aggregates the messages to be sent to workers so they can be sent
+ * in bulk.
+ *
+ * @param <I> Vertex id
+ * @param <M> Message data
+ */
+@NotThreadSafe
+@SuppressWarnings("unchecked")
+public class SendOneMessageToManyCache<I extends WritableComparable,
+  M extends Writable> extends SendMessageCache<I, M> {
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(SendOneMessageToManyCache.class);
+  /** Cache serialized one to many messages for each worker */
+  private final ByteArrayOneMessageToManyIds<I, M>[] msgVidsCache;
+  /** Tracking message-vertexIds sizes for each worker */
+  private final int[] msgVidsSizes;
+  /** Reused byte array to serialize target ids on each worker */
+  private final ExtendedDataOutput[] idSerializer;
+  /** Reused int array to count target id distribution */
+  private final int[] idCounter;
+  /**
+   * Reused int array to record the partition id
+   * of the first target vertex id found on the worker.
+   */
+  private final int[] firstPartitionMap;
+  /** The WorkerInfo list */
+  private final WorkerInfo[] workerInfoList;
+
+  /**
+   * Constructor
+   *
+   * @param conf Giraph configuration
+   * @param serviceWorker Service worker
+   * @param processor NettyWorkerClientRequestProcessor
+   * @param maxMsgSize Max message size sent to a worker
+   */
+  public SendOneMessageToManyCache(ImmutableClassesGiraphConfiguration conf,
+    CentralizedServiceWorker<?, ?, ?> serviceWorker,
+    NettyWorkerClientRequestProcessor<I, ?, ?> processor,
+    int maxMsgSize) {
+    super(conf, serviceWorker, processor, maxMsgSize);
+    int numWorkers = getNumWorkers();
+    msgVidsCache = new ByteArrayOneMessageToManyIds[numWorkers];
+    msgVidsSizes = new int[numWorkers];
+    idSerializer = new ExtendedDataOutput[numWorkers];
+    // InitialBufferSizes is alo initialized based on the number of workers.
+    // As a result, initialBufferSizes is the same as idSerializer in length
+    int initialBufferSize = 0;
+    for (int i = 0; i < this.idSerializer.length; i++) {
+      initialBufferSize = getSendWorkerInitialBufferSize(i);
+      if (initialBufferSize > 0) {
+        // InitialBufferSizes is from super class.
+        // Each element is for one worker.
+        idSerializer[i] = conf.createExtendedDataOutput(initialBufferSize);
+      }
+    }
+    idCounter = new int[numWorkers];
+    firstPartitionMap = new int[numWorkers];
+    // Get worker info list.
+    workerInfoList = new WorkerInfo[numWorkers];
+    // Remember there could be null in the array.
+    for (WorkerInfo workerInfo : serviceWorker.getWorkerInfoList()) {
+      workerInfoList[workerInfo.getTaskId()] = workerInfo;
+    }
+  }
+
+  /**
+   * Reset ExtendedDataOutput array for id serialization
+   * in next message-Vids encoding
+   */
+  private void resetIdSerializers() {
+    for (int i = 0; i < this.idSerializer.length; i++) {
+      if (idSerializer[i] != null) {
+        idSerializer[i].reset();
+      }
+    }
+  }
+
+  /**
+   * Reset id counter for next message-vertexIds encoding
+   */
+  private void resetIdCounter() {
+    Arrays.fill(idCounter, 0);
+  }
+
+  /**
+   * Add message with multiple target ids to message cache.
+   *
+   * @param workerInfo The remote worker destination
+   * @param ids A byte array to hold serialized vertex ids
+   * @param idPos The end position of ids
+   *              information in the byte array above
+   * @param count The number of target ids
+   * @param message Message to send to remote worker
+   * @return The size of messages for the worker.
+   */
+  private int addOneToManyMessage(
+    WorkerInfo workerInfo, byte[] ids, int idPos, int count, M message) {
+    // Get the data collection
+    ByteArrayOneMessageToManyIds<I, M> workerData =
+      msgVidsCache[workerInfo.getTaskId()];
+    if (workerData == null) {
+      workerData = new ByteArrayOneMessageToManyIds<I, M>(
+        getConf().getOutgoingMessageValueFactory());
+      workerData.setConf(getConf());
+      workerData.initialize(getSendWorkerInitialBufferSize(
+        workerInfo.getTaskId()));
+      msgVidsCache[workerInfo.getTaskId()] = workerData;
+    }
+    workerData.add(ids, idPos, count, message);
+    // Update the size of cached, outgoing data per worker
+    msgVidsSizes[workerInfo.getTaskId()] =
+      workerData.getSize();
+    return msgVidsSizes[workerInfo.getTaskId()];
+  }
+
+  /**
+   * Gets the messages + vertexIds for a worker and removes it from the cache.
+   * Here the {@link org.apache.giraph.utils.ByteArrayOneMessageToManyIds}
+   * returned could be null.But when invoking this method, we also check if
+   * the data size sent to this worker is above the threshold.
+   * Therefore, it doesn't matter if the result is null or not.
+   *
+   * @param workerInfo Target worker to which one messages - many ids are sent
+   * @return {@link org.apache.giraph.utils.ByteArrayOneMessageToManyIds}
+   *         that belong to the workerInfo
+   */
+  private ByteArrayOneMessageToManyIds<I, M>
+  removeWorkerMsgVids(WorkerInfo workerInfo) {
+    ByteArrayOneMessageToManyIds<I, M> workerData =
+      msgVidsCache[workerInfo.getTaskId()];
+    if (workerData != null) {
+      msgVidsCache[workerInfo.getTaskId()] = null;
+      msgVidsSizes[workerInfo.getTaskId()] = 0;
+    }
+    return workerData;
+  }
+
+  /**
+   * Gets all messages - vertexIds and removes them from the cache.
+   *
+   * @return All vertex messages for all workers
+   */
+  private PairList<WorkerInfo, ByteArrayOneMessageToManyIds<I, M>>
+  removeAllMsgVids() {
+    PairList<WorkerInfo, ByteArrayOneMessageToManyIds<I, M>> allData =
+      new PairList<WorkerInfo, ByteArrayOneMessageToManyIds<I, M>>();
+    allData.initialize(msgVidsCache.length);
+    for (WorkerInfo workerInfo : getWorkerPartitions().keySet()) {
+      ByteArrayOneMessageToManyIds<I, M> workerData =
+        removeWorkerMsgVids(workerInfo);
+      if (workerData != null && !workerData.isEmpty()) {
+        allData.add(workerInfo, workerData);
+      }
+    }
+    return allData;
+  }
+
+  @Override
+  public void sendMessageToAllRequest(Iterator<I> vertexIdIterator, M message) {
+    // This is going to be reused through every message sending
+    resetIdSerializers();
+    resetIdCounter();
+    // Count messages
+    int currentMachineId = 0;
+    PartitionOwner owner = null;
+    WorkerInfo workerInfo = null;
+    I vertexId = null;
+    while (vertexIdIterator.hasNext()) {
+      vertexId = vertexIdIterator.next();
+      owner = getServiceWorker().getVertexPartitionOwner(vertexId);
+      workerInfo = owner.getWorkerInfo();
+      currentMachineId = workerInfo.getTaskId();
+      // Serialize this target vertex id
+      try {
+        vertexId.write(idSerializer[currentMachineId]);
+      } catch (IOException e) {
+        throw new IllegalStateException(
+          "Failed to serialize the target vertex id.");
+      }
+      idCounter[currentMachineId]++;
+      // Record the first partition id in the worker which message send to.
+      // If idCounter shows there is only one target on this worker
+      // then this is the partition number of the target vertex.
+      if (idCounter[currentMachineId] == 1) {
+        firstPartitionMap[currentMachineId] = owner.getPartitionId();
+      }
+    }
+    // Add the message to the cache
+    int idSerializerPos = 0;
+    int workerMessageSize = 0;
+    byte[] serializedId  = null;
+    WritableRequest writableRequest = null;
+    for (int i = 0; i < idCounter.length; i++) {
+      if (idCounter[i] == 1) {
+        serializedId = idSerializer[i].getByteArray();
+        idSerializerPos = idSerializer[i].getPos();
+        // Add the message to the cache
+        workerMessageSize = addMessage(workerInfoList[i],
+          firstPartitionMap[i], serializedId, idSerializerPos, message);
+
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("sendMessageToAllRequest: Send bytes (" +
+            message.toString() + ") to one target in  worker " +
+            workerInfoList[i]);
+        }
+        ++totalMsgsSentInSuperstep;
+        if (workerMessageSize >= maxMessagesSizePerWorker) {
+          PairList<Integer, VertexIdMessages<I, M>>
+            workerMessages = removeWorkerMessages(workerInfoList[i]);
+          writableRequest = new SendWorkerMessagesRequest<>(workerMessages);
+          totalMsgBytesSentInSuperstep += writableRequest.getSerializedSize();
+          clientProcessor.doRequest(workerInfoList[i], writableRequest);
+          // Notify sending
+          getServiceWorker().getGraphTaskManager().notifySentMessages();
+        }
+      } else if (idCounter[i] > 1) {
+        serializedId = idSerializer[i].getByteArray();
+        idSerializerPos = idSerializer[i].getPos();
+        workerMessageSize = addOneToManyMessage(
+            workerInfoList[i], serializedId, idSerializerPos, idCounter[i],
+            message);
+
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("sendMessageToAllRequest: Send bytes (" +
+            message.toString() + ") to all targets in worker" +
+            workerInfoList[i]);
+        }
+        totalMsgsSentInSuperstep += idCounter[i];
+        if (workerMessageSize >= maxMessagesSizePerWorker) {
+          ByteArrayOneMessageToManyIds<I, M> workerMsgVids =
+            removeWorkerMsgVids(workerInfoList[i]);
+          writableRequest =  new SendWorkerOneMessageToManyRequest<>(
+            workerMsgVids, getConf());
+          totalMsgBytesSentInSuperstep += writableRequest.getSerializedSize();
+          clientProcessor.doRequest(workerInfoList[i], writableRequest);
+          // Notify sending
+          getServiceWorker().getGraphTaskManager().notifySentMessages();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void flush() {
+    super.flush();
+    PairList<WorkerInfo, ByteArrayOneMessageToManyIds<I, M>>
+    remainingMsgVidsCache = removeAllMsgVids();
+    PairList<WorkerInfo,
+        ByteArrayOneMessageToManyIds<I, M>>.Iterator
+    msgIdsIterator = remainingMsgVidsCache.getIterator();
+    while (msgIdsIterator.hasNext()) {
+      msgIdsIterator.next();
+      WritableRequest writableRequest =
+        new SendWorkerOneMessageToManyRequest<>(
+            msgIdsIterator.getCurrentSecond(), getConf());
+      totalMsgBytesSentInSuperstep += writableRequest.getSerializedSize();
+      clientProcessor.doRequest(
+        msgIdsIterator.getCurrentFirst(), writableRequest);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
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 b3f8733..85bfe04 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
@@ -176,6 +176,8 @@ public class ServerData<I extends WritableComparable,
             messageStoreFactory.newStore(conf.getIncomingMessageValueFactory());
     incomingMessageStore =
         messageStoreFactory.newStore(conf.getOutgoingMessageValueFactory());
+    // finalize current message-store before resolving mutations
+    currentMessageStore.finalizeStore();
 
     currentWorkerToWorkerMessages = incomingWorkerToWorkerMessages;
     incomingWorkerToWorkerMessages =

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/AbstractListPerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/AbstractListPerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/AbstractListPerVertexStore.java
new file mode 100644
index 0000000..6840f86
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/AbstractListPerVertexStore.java
@@ -0,0 +1,103 @@
+/*
+ * 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.comm.messages;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.partition.PartitionOwner;
+import org.apache.giraph.utils.VertexIdIterator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Abstract Implementation of {@link SimpleMessageStore} where
+ * multiple messages are stored per vertex as a list
+ * Used when there is no combiner provided.
+ *
+ * @param <I> Vertex id
+ * @param <M> Message data
+ * @param <L> List type
+ */
+public abstract class AbstractListPerVertexStore<I extends WritableComparable,
+  M extends Writable, L extends List> extends SimpleMessageStore<I, M, L> {
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Message class held in the store
+   * @param service Service worker
+   * @param config Hadoop configuration
+   */
+  public AbstractListPerVertexStore(
+    MessageValueFactory<M> messageValueFactory,
+    CentralizedServiceWorker<I, ?, ?> service,
+    ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    super(messageValueFactory, service, config);
+  }
+
+  /**
+   * Create an instance of L
+   * @return instance of L
+   */
+  protected abstract L createList();
+
+  /**
+   * Get the list of pointers for a vertex
+   * Each pointer has information of how to access an encoded message
+   * for this vertex
+   *
+   * @param iterator vertex id iterator
+   * @return pointer list
+   */
+  protected L getOrCreateList(VertexIdIterator<I> iterator) {
+    PartitionOwner owner =
+        service.getVertexPartitionOwner(iterator.getCurrentVertexId());
+    int partitionId = owner.getPartitionId();
+    ConcurrentMap<I, L> partitionMap = getOrCreatePartitionMap(partitionId);
+    L list = partitionMap.get(iterator.getCurrentVertexId());
+    if (list == null) {
+      L newList = createList();
+      list = partitionMap.putIfAbsent(
+          iterator.releaseCurrentVertexId(), newList);
+      if (list == null) {
+        list = newList;
+      }
+    }
+    return list;
+  }
+
+  @Override
+  public Iterable<M> getVertexMessages(I vertexId) throws IOException {
+    ConcurrentMap<I, L> partitionMap =
+        map.get(getPartitionId(vertexId));
+    if (partitionMap == null) {
+      return Collections.<M>emptyList();
+    }
+    L list = partitionMap.get(vertexId);
+    return list == null ? Collections.<M>emptyList() :
+        getMessagesAsIterable(list);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
index 65939bb..57d255f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
@@ -62,6 +62,11 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
     super(messageValueFactory, service, config);
   }
 
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
   /**
    * Get the extended data output for a vertex id from the iterator, creating
    * if necessary.  This method will take ownership of the vertex id from the
@@ -89,12 +94,10 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
 
   @Override
   public void addPartitionMessages(
-      int partitionId,
-      VertexIdMessages<I, M> messages) throws IOException {
+    int partitionId, VertexIdMessages<I, M> messages) throws IOException {
     ConcurrentMap<I, DataInputOutput> partitionMap =
         getOrCreatePartitionMap(partitionId);
-    VertexIdMessageBytesIterator<I, M>
-        vertexIdMessageBytesIterator =
+    VertexIdMessageBytesIterator<I, M> vertexIdMessageBytesIterator =
         messages.getVertexIdMessageBytesIterator();
     // Try to copy the message buffer over rather than
     // doing a deserialization of a message just to know its size.  This
@@ -113,8 +116,8 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
         }
       }
     } else {
-      VertexIdMessageIterator<I, M>
-          vertexIdMessageIterator = messages.getVertexIdMessageIterator();
+      VertexIdMessageIterator<I, M> vertexIdMessageIterator =
+          messages.getVertexIdMessageIterator();
       while (vertexIdMessageIterator.hasNext()) {
         vertexIdMessageIterator.next();
         DataInputOutput dataInputOutput =
@@ -188,7 +191,7 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
    * @param <M> Message data
    */
   private static class Factory<I extends WritableComparable, M extends Writable>
-      implements MessageStoreFactory<I, M, MessageStore<I, M>> {
+    implements MessageStoreFactory<I, M, MessageStore<I, M>> {
     /** Service worker */
     private CentralizedServiceWorker<I, ?, ?> service;
     /** Hadoop configuration */

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
index f691d3e..db22503 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
@@ -22,8 +22,10 @@ import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.comm.messages.primitives.IntByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.IntFloatMessageStore;
-import org.apache.giraph.comm.messages.primitives.LongByteArrayMessageStore;
+import org.apache.giraph.comm.messages.primitives.long_id.LongByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.LongDoubleMessageStore;
+import org.apache.giraph.comm.messages.primitives.long_id.LongPointerListMessageStore;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.factories.MessageValueFactory;
 import org.apache.hadoop.io.DoubleWritable;
@@ -43,6 +45,7 @@ import org.apache.log4j.Logger;
  * @param <I> Vertex id
  * @param <M> Message data
  */
+@SuppressWarnings("unchecked")
 public class InMemoryMessageStoreFactory<I extends WritableComparable,
     M extends Writable>
     implements MessageStoreFactory<I, M, MessageStore<I, M>> {
@@ -51,9 +54,9 @@ public class InMemoryMessageStoreFactory<I extends WritableComparable,
       Logger.getLogger(InMemoryMessageStoreFactory.class);
 
   /** Service worker */
-  private CentralizedServiceWorker<I, ?, ?> service;
+  protected CentralizedServiceWorker<I, ?, ?> service;
   /** Hadoop configuration */
-  private ImmutableClassesGiraphConfiguration<I, ?, ?> conf;
+  protected ImmutableClassesGiraphConfiguration<I, ?, ?> conf;
 
   /**
    * Default constructor allowing class invocation via Reflection.
@@ -61,46 +64,89 @@ public class InMemoryMessageStoreFactory<I extends WritableComparable,
   public InMemoryMessageStoreFactory() {
   }
 
-  @Override
-  public MessageStore<I, M> newStore(
-      MessageValueFactory<M> messageValueFactory) {
+  /**
+   * MessageStore to be used when combiner is enabled
+   *
+   * @param messageValueFactory message value factory
+   * @return message store
+   */
+  protected MessageStore<I, M> newStoreWithCombiner(
+    MessageValueFactory<M> messageValueFactory) {
     Class<M> messageClass = messageValueFactory.getValueClass();
     MessageStore messageStore;
-    if (conf.useMessageCombiner()) {
-      Class<I> vertexIdClass = conf.getVertexIdClass();
-      if (vertexIdClass.equals(IntWritable.class) &&
-          messageClass.equals(FloatWritable.class)) {
-        messageStore = new IntFloatMessageStore(
-            (CentralizedServiceWorker<IntWritable, Writable, Writable>) service,
-            (MessageCombiner<IntWritable, FloatWritable>)
-                conf.<FloatWritable>createMessageCombiner());
-      } else if (vertexIdClass.equals(LongWritable.class) &&
-          messageClass.equals(DoubleWritable.class)) {
-        messageStore = new LongDoubleMessageStore(
+    Class<I> vertexIdClass = conf.getVertexIdClass();
+    if (vertexIdClass.equals(IntWritable.class) &&
+        messageClass.equals(FloatWritable.class)) {
+      messageStore = new IntFloatMessageStore(
+          (CentralizedServiceWorker<IntWritable, Writable, Writable>) service,
+          (MessageCombiner<IntWritable, FloatWritable>)
+              conf.<FloatWritable>createMessageCombiner());
+    } else if (vertexIdClass.equals(LongWritable.class) &&
+        messageClass.equals(DoubleWritable.class)) {
+      messageStore = new LongDoubleMessageStore(
           (CentralizedServiceWorker<LongWritable, Writable, Writable>) service,
           (MessageCombiner<LongWritable, DoubleWritable>)
               conf.<DoubleWritable>createMessageCombiner());
-      } else {
-        messageStore = new OneMessagePerVertexStore<I, M>(messageValueFactory,
+    } else {
+      messageStore = new OneMessagePerVertexStore(messageValueFactory,
           service, conf.<M>createMessageCombiner(), conf);
+    }
+    return messageStore;
+  }
+
+  /**
+   * MessageStore to be used when combiner is not enabled
+   *
+   * @param messageValueFactory message value factory
+   * @return message store
+   */
+  protected MessageStore<I, M> newStoreWithoutCombiner(
+    MessageValueFactory<M> messageValueFactory) {
+    MessageStore messageStore = null;
+    MessageEncodeAndStoreType encodeAndStore = GiraphConstants
+        .MESSAGE_ENCODE_AND_STORE_TYPE.get(conf);
+    Class<I> vertexIdClass = conf.getVertexIdClass();
+    if (vertexIdClass.equals(IntWritable.class)) { // INT
+      messageStore = new IntByteArrayMessageStore(messageValueFactory,
+          service, conf);
+    } else if (vertexIdClass.equals(LongWritable.class)) { // LONG
+      if (encodeAndStore.equals(
+          MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION) ||
+          encodeAndStore.equals(
+            MessageEncodeAndStoreType.EXTRACT_BYTEARRAY_PER_PARTITION)) {
+        messageStore = new LongByteArrayMessageStore(messageValueFactory,
+            service, conf);
+      } else if (encodeAndStore.equals(
+          MessageEncodeAndStoreType.POINTER_LIST_PER_VERTEX)) {
+        messageStore = new LongPointerListMessageStore(messageValueFactory,
+            service, conf);
       }
-    } else {
-      Class<I> vertexIdClass = conf.getVertexIdClass();
-      if (vertexIdClass.equals(IntWritable.class)) {
-        messageStore = new IntByteArrayMessageStore<M>(messageValueFactory,
-          (CentralizedServiceWorker<IntWritable, Writable, Writable>) service,
-          (ImmutableClassesGiraphConfiguration<IntWritable, Writable, Writable>)
-            conf);
-      } else if (vertexIdClass.equals(LongWritable.class)) {
-        messageStore = new LongByteArrayMessageStore<M>(messageValueFactory,
-          (CentralizedServiceWorker<LongWritable, Writable, Writable>) service,
-          (ImmutableClassesGiraphConfiguration<LongWritable, Writable,
-           Writable>) conf);
-      } else {
-        messageStore = new ByteArrayMessagesPerVertexStore<I, M>(
-          messageValueFactory, service, conf);
+    } else { // GENERAL
+      if (encodeAndStore.equals(
+          MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION) ||
+          encodeAndStore.equals(
+              MessageEncodeAndStoreType.EXTRACT_BYTEARRAY_PER_PARTITION)) {
+        messageStore = new ByteArrayMessagesPerVertexStore<>(
+            messageValueFactory, service, conf);
+      } else if (encodeAndStore.equals(
+          MessageEncodeAndStoreType.POINTER_LIST_PER_VERTEX)) {
+        messageStore = new PointerListPerVertexStore(messageValueFactory,
+            service, conf);
       }
     }
+    return messageStore;
+  }
+
+  @Override
+  public MessageStore<I, M> newStore(
+      MessageValueFactory<M> messageValueFactory) {
+    Class<M> messageClass = messageValueFactory.getValueClass();
+    MessageStore messageStore;
+    if (conf.useMessageCombiner()) {
+      messageStore = newStoreWithCombiner(messageValueFactory);
+    } else {
+      messageStore = newStoreWithoutCombiner(messageValueFactory);
+    }
 
     if (LOG.isInfoEnabled()) {
       LOG.info("newStore: Created " + messageStore.getClass() +

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageEncodeAndStoreType.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageEncodeAndStoreType.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageEncodeAndStoreType.java
new file mode 100644
index 0000000..7a5b702
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageEncodeAndStoreType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.comm.messages;
+
+/**
+ * There are two types of message-stores currently
+ * pointer based, and default byte-array based
+ */
+public enum MessageEncodeAndStoreType {
+  /**
+   * Use message-store which is based on list of pointers to encoded messages
+   */
+  POINTER_LIST_PER_VERTEX(true),
+  /**
+   * Extract a byte array per partition from one message to many ids encoding
+   * and then store
+   */
+  EXTRACT_BYTEARRAY_PER_PARTITION(true),
+  /**
+   * Use a byte-array to store messages for each partition
+   */
+  BYTEARRAY_PER_PARTITION(false);
+
+  /** Can use one message to many ids encoding? */
+  private final boolean oneMessageToManyIdsEncoding;
+
+  /**
+   * Constructor
+   *
+   * @param oneMessageToManyIdsEncoding use one message to many ids encoding
+   */
+  MessageEncodeAndStoreType(boolean oneMessageToManyIdsEncoding) {
+    this.oneMessageToManyIdsEncoding = oneMessageToManyIdsEncoding;
+  }
+
+  /**
+   * True if one message to many ids encoding is set
+   * @return return oneMessageToManyIdsEncoding
+   */
+  public boolean useOneMessageToManyIdsEncoding() {
+    return oneMessageToManyIdsEncoding;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStore.java
index 7d0bbc6..6f1179a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStore.java
@@ -34,6 +34,14 @@ import org.apache.hadoop.io.WritableComparable;
 public interface MessageStore<I extends WritableComparable,
     M extends Writable> {
   /**
+   * True if this message-store encodes messages as a list of long pointers
+   * to compact serialized messages
+   *
+   * @return true if we encode messages as a list of pointers
+   */
+  boolean isPointerListEncoding();
+
+  /**
    * Gets messages for a vertex.  The lifetime of every message is only
    * guaranteed until the iterator's next() method is called. Do not hold
    * references to objects returned by this iterator.
@@ -79,6 +87,13 @@ public interface MessageStore<I extends WritableComparable,
     throws IOException;
 
   /**
+   * Called before start of computation in bspworker
+   * Since it is run from a single thread while the store is not being
+   * accessed by any other thread - this is ensured to be thread-safe
+   */
+  void finalizeStore();
+
+  /**
    * Gets vertex ids from selected partition which we have messages for
    *
    * @param partitionId Id of partition

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
index 9bede06..d3942d4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
@@ -63,6 +63,11 @@ public class OneMessagePerVertexStore<I extends WritableComparable,
   }
 
   @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
+  @Override
   public void addPartitionMessages(
       int partitionId,
       VertexIdMessages<I, M> messages) throws IOException {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListMessagesIterable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListMessagesIterable.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListMessagesIterable.java
new file mode 100644
index 0000000..e5a1691
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListMessagesIterable.java
@@ -0,0 +1,105 @@
+/*
+ * 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.comm.messages;
+
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongListIterator;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.utils.ExtendedByteArrayOutputBuffer;
+import org.apache.giraph.utils.ExtendedDataOutput;
+import org.apache.giraph.utils.UnsafeReusableByteArrayInput;
+import org.apache.hadoop.io.Writable;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ * Create an iterable for messages based on a pointer list
+ *
+ * @param <M> messageType
+ */
+public class PointerListMessagesIterable<M extends Writable>
+  implements Iterable<M> {
+  /** Message class */
+  private final MessageValueFactory<M> messageValueFactory;
+  /** List of pointers to messages in byte array */
+  private final LongArrayList pointers;
+  /** Holds the byte arrays of serialized messages */
+  private final ExtendedByteArrayOutputBuffer msgBuffer;
+  /** Reader to read data from byte buffer */
+  private final UnsafeReusableByteArrayInput messageReader;
+
+  /**
+   *
+   * @param messageValueFactory message value factory
+   * @param pointers pointers to messages in buffer
+   * @param msgBuffer holds the byte arrays of serialized messages
+   */
+  public PointerListMessagesIterable(MessageValueFactory<M> messageValueFactory,
+    LongArrayList pointers, ExtendedByteArrayOutputBuffer msgBuffer) {
+    this.messageValueFactory = messageValueFactory;
+    this.pointers = pointers;
+    this.msgBuffer = msgBuffer;
+    // TODO - if needed implement same for Safe as well
+    messageReader = new UnsafeReusableByteArrayInput();
+  }
+
+  /**
+   * Create message from factory
+   *
+   * @return message instance
+   */
+  protected M createMessage() {
+    return messageValueFactory.newInstance();
+  }
+
+  @Override
+  public Iterator<M> iterator() {
+    return new Iterator<M>() {
+      private final LongListIterator iterator = pointers.iterator();
+      private final M reusableMsg =
+        PointerListMessagesIterable.this.createMessage();
+      @Override
+      public boolean hasNext() {
+        return iterator.hasNext();
+      }
+
+      @Override
+      public M next() {
+        long pointer = iterator.next();
+        try {
+          int index = (int) (pointer >>> 32);
+          int offset = (int) pointer;
+          ExtendedDataOutput buffer = msgBuffer.getDataOutput(index);
+          messageReader.initialize(buffer.getByteArray(), offset,
+            buffer.getPos());
+          reusableMsg.readFields(messageReader);
+        } catch (IOException e) {
+          throw new IllegalStateException("Got exception : " + e);
+        }
+        return reusableMsg;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListPerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListPerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListPerVertexStore.java
new file mode 100644
index 0000000..cce0439
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/PointerListPerVertexStore.java
@@ -0,0 +1,137 @@
+/*
+ * 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.comm.messages;
+
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.utils.ExtendedByteArrayOutputBuffer;
+import org.apache.giraph.utils.ExtendedDataOutput;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.giraph.utils.ExtendedByteArrayOutputBuffer.IndexAndDataOut;
+
+/**
+ * Implementation of {@link SimpleMessageStore} where multiple messages are
+ * stored as a list of long pointers to extended data output objects
+ * Used when there is no combiner provided.
+ *
+ * @param <I> vertexId type
+ * @param <M> message type
+ */
+public class PointerListPerVertexStore<I extends WritableComparable,
+  M extends Writable> extends AbstractListPerVertexStore<I, M, LongArrayList> {
+
+  /** Buffers of byte array outputs used to store messages - thread safe */
+  private final ExtendedByteArrayOutputBuffer bytesBuffer;
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Message class held in the store
+   * @param service Service worker
+   * @param config Hadoop configuration
+   */
+  public PointerListPerVertexStore(
+      MessageValueFactory<M> messageValueFactory,
+      CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    super(messageValueFactory, service, config);
+    bytesBuffer = new ExtendedByteArrayOutputBuffer(config);
+  }
+
+  @Override
+  public boolean isPointerListEncoding() {
+    return true;
+  }
+
+  @Override
+  protected LongArrayList createList() {
+    return new LongArrayList();
+  }
+
+  @Override
+  public void addPartitionMessages(
+    int partitionId, VertexIdMessages<I, M> messages) throws IOException {
+    VertexIdMessageIterator<I, M> vertexIdMessageIterator =
+        messages.getVertexIdMessageIterator();
+    long pointer = 0;
+    LongArrayList list;
+    while (vertexIdMessageIterator.hasNext()) {
+      vertexIdMessageIterator.next();
+      M msg = vertexIdMessageIterator.getCurrentMessage();
+      list = getOrCreateList(vertexIdMessageIterator);
+      if (vertexIdMessageIterator.isNewMessage()) {
+        IndexAndDataOut indexAndDataOut = bytesBuffer.getIndexAndDataOut();
+        pointer = indexAndDataOut.getIndex();
+        pointer <<= 32;
+        ExtendedDataOutput dataOutput = indexAndDataOut.getDataOutput();
+        pointer += dataOutput.getPos();
+        msg.write(dataOutput);
+      }
+      synchronized (list) {
+        list.add(pointer);
+      }
+    }
+  }
+
+  /**
+   * Get messages as an iterable from message storage
+   *
+   * @param pointers list of pointers to messages
+   * @return Messages as an iterable
+   */
+  @Override
+  public Iterable<M> getMessagesAsIterable(LongArrayList pointers) {
+    return new PointerListMessagesIterable<>(messageValueFactory, pointers,
+      bytesBuffer);
+  }
+
+  @Override
+  protected int getNumberOfMessagesIn(ConcurrentMap<I,
+    LongArrayList> partitionMap) {
+    int numberOfMessages = 0;
+    for (LongArrayList list : partitionMap.values()) {
+      numberOfMessages += list.size();
+    }
+    return numberOfMessages;
+  }
+
+  // FIXME -- complete these for check-pointing
+  @Override
+  protected void writeMessages(LongArrayList messages, DataOutput out)
+    throws IOException {
+
+  }
+
+  @Override
+  protected LongArrayList readFieldsForMessages(DataInput in)
+    throws IOException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
index 13292a2..37b8c05 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
@@ -139,6 +139,10 @@ public abstract class SimpleMessageStore<I extends WritableComparable,
   }
 
   @Override
+  public void finalizeStore() {
+  }
+
+  @Override
   public Iterable<I> getPartitionDestinationVertices(int partitionId) {
     ConcurrentMap<I, ?> partitionMap = map.get(partitionId);
     return (partitionMap == null) ? Collections.<I>emptyList() :

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/out_of_core/DiskBackedMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/out_of_core/DiskBackedMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/out_of_core/DiskBackedMessageStore.java
index 18b7798..3000cd4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/out_of_core/DiskBackedMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/out_of_core/DiskBackedMessageStore.java
@@ -83,6 +83,11 @@ public class DiskBackedMessageStore<I extends WritableComparable,
   }
 
   @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
+  @Override
   public void addPartitionMessages(
       int partitionId,
       VertexIdMessages<I, M> messages) throws IOException {
@@ -106,6 +111,10 @@ public class DiskBackedMessageStore<I extends WritableComparable,
   }
 
   @Override
+  public void finalizeStore() {
+  }
+
+  @Override
   public Iterable<M> getVertexMessages(I vertexId) throws IOException {
     if (hasMessagesForVertex(vertexId)) {
       return getMessageStore(vertexId).getVertexMessages(vertexId);

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntByteArrayMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntByteArrayMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntByteArrayMessageStore.java
index dbc1ce8..0012bf0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntByteArrayMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntByteArrayMessageStore.java
@@ -94,6 +94,11 @@ public class IntByteArrayMessageStore<M extends Writable>
     }
   }
 
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
   /**
    * Get map which holds messages for partition which vertex belongs to.
    *
@@ -161,6 +166,10 @@ public class IntByteArrayMessageStore<M extends Writable>
   }
 
   @Override
+  public void finalizeStore() {
+  }
+
+  @Override
   public void clearPartition(int partitionId) throws IOException {
     map.get(partitionId).clear();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
index be75ee8..97086e1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
@@ -86,6 +86,11 @@ public class IntFloatMessageStore
     }
   }
 
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
   /**
    * Get map which holds messages for partition which vertex belongs to.
    *
@@ -126,6 +131,10 @@ public class IntFloatMessageStore
   }
 
   @Override
+  public void finalizeStore() {
+  }
+
+  @Override
   public void clearPartition(int partitionId) throws IOException {
     map.get(partitionId).clear();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongByteArrayMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongByteArrayMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongByteArrayMessageStore.java
deleted file mode 100644
index 3110864..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongByteArrayMessageStore.java
+++ /dev/null
@@ -1,241 +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.comm.messages.primitives;
-
-import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.comm.messages.MessageStore;
-import org.apache.giraph.comm.messages.MessagesIterable;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.factories.MessageValueFactory;
-import org.apache.giraph.partition.Partition;
-import org.apache.giraph.utils.VertexIdMessageBytesIterator;
-import org.apache.giraph.utils.VertexIdMessageIterator;
-import org.apache.giraph.utils.VertexIdMessages;
-import org.apache.giraph.utils.VerboseByteStructMessageWrite;
-import org.apache.giraph.utils.EmptyIterable;
-import org.apache.giraph.utils.io.DataInputOutput;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-
-import com.google.common.collect.Lists;
-
-import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
-import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
-import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
-import it.unimi.dsi.fastutil.longs.LongIterator;
-import it.unimi.dsi.fastutil.objects.ObjectIterator;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * Special message store to be used when ids are LongWritable and no combiner
- * is used.
- * Uses fastutil primitive maps in order to decrease number of objects and
- * get better performance.
- *
- * @param <M> Message type
- */
-public class LongByteArrayMessageStore<M extends Writable>
-    implements MessageStore<LongWritable, M> {
-  /** Message value factory */
-  protected final MessageValueFactory<M> messageValueFactory;
-  /** Map from partition id to map from vertex id to message */
-  private final
-  Int2ObjectOpenHashMap<Long2ObjectOpenHashMap<DataInputOutput>> map;
-  /** Service worker */
-  private final CentralizedServiceWorker<LongWritable, ?, ?> service;
-  /** Giraph configuration */
-  private final ImmutableClassesGiraphConfiguration<LongWritable, ?, ?> config;
-
-  /**
-   * Constructor
-   *
-   * @param messageValueFactory Factory for creating message values
-   * @param service      Service worker
-   * @param config       Hadoop configuration
-   */
-  public LongByteArrayMessageStore(
-      MessageValueFactory<M> messageValueFactory,
-      CentralizedServiceWorker<LongWritable, Writable, Writable> service,
-      ImmutableClassesGiraphConfiguration<LongWritable, Writable, Writable>
-        config) {
-    this.messageValueFactory = messageValueFactory;
-    this.service = service;
-    this.config = config;
-
-    map =
-        new Int2ObjectOpenHashMap<Long2ObjectOpenHashMap<DataInputOutput>>();
-    for (int partitionId : service.getPartitionStore().getPartitionIds()) {
-      Partition<LongWritable, Writable, Writable> partition =
-          service.getPartitionStore().getOrCreatePartition(partitionId);
-      Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
-          new Long2ObjectOpenHashMap<DataInputOutput>(
-              (int) partition.getVertexCount());
-      map.put(partitionId, partitionMap);
-      service.getPartitionStore().putPartition(partition);
-    }
-  }
-
-  /**
-   * Get map which holds messages for partition which vertex belongs to.
-   *
-   * @param vertexId Id of the vertex
-   * @return Map which holds messages for partition which vertex belongs to.
-   */
-  private Long2ObjectOpenHashMap<DataInputOutput> getPartitionMap(
-      LongWritable vertexId) {
-    return map.get(service.getPartitionId(vertexId));
-  }
-
-  /**
-   * Get the DataInputOutput for a vertex id, creating if necessary.
-   *
-   * @param partitionMap Partition map to look in
-   * @param vertexId Id of the vertex
-   * @return DataInputOutput for this vertex id (created if necessary)
-   */
-  private DataInputOutput getDataInputOutput(
-      Long2ObjectOpenHashMap<DataInputOutput> partitionMap,
-      long vertexId) {
-    DataInputOutput dataInputOutput = partitionMap.get(vertexId);
-    if (dataInputOutput == null) {
-      dataInputOutput = config.createMessagesInputOutput();
-      partitionMap.put(vertexId, dataInputOutput);
-    }
-    return dataInputOutput;
-  }
-
-  @Override
-  public void addPartitionMessages(int partitionId,
-      VertexIdMessages<LongWritable, M> messages) throws
-      IOException {
-    Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
-        map.get(partitionId);
-    synchronized (partitionMap) {
-      VertexIdMessageBytesIterator<LongWritable, M>
-          vertexIdMessageBytesIterator =
-          messages.getVertexIdMessageBytesIterator();
-      // Try to copy the message buffer over rather than
-      // doing a deserialization of a message just to know its size.  This
-      // should be more efficient for complex objects where serialization is
-      // expensive.  If this type of iterator is not available, fall back to
-      // deserializing/serializing the messages
-      if (vertexIdMessageBytesIterator != null) {
-        while (vertexIdMessageBytesIterator.hasNext()) {
-          vertexIdMessageBytesIterator.next();
-          DataInputOutput dataInputOutput = getDataInputOutput(partitionMap,
-              vertexIdMessageBytesIterator.getCurrentVertexId().get());
-          vertexIdMessageBytesIterator.writeCurrentMessageBytes(
-              dataInputOutput.getDataOutput());
-        }
-      } else {
-        VertexIdMessageIterator<LongWritable, M>
-            iterator = messages.getVertexIdMessageIterator();
-        while (iterator.hasNext()) {
-          iterator.next();
-          DataInputOutput dataInputOutput =  getDataInputOutput(partitionMap,
-              iterator.getCurrentVertexId().get());
-          VerboseByteStructMessageWrite.verboseWriteCurrentMessage(iterator,
-              dataInputOutput.getDataOutput());
-        }
-      }
-    }
-  }
-
-  @Override
-  public void clearPartition(int partitionId) throws IOException {
-    map.get(partitionId).clear();
-  }
-
-  @Override
-  public boolean hasMessagesForVertex(LongWritable vertexId) {
-    return getPartitionMap(vertexId).containsKey(vertexId.get());
-  }
-
-  @Override
-  public Iterable<M> getVertexMessages(
-      LongWritable vertexId) throws IOException {
-    DataInputOutput dataInputOutput =
-        getPartitionMap(vertexId).get(vertexId.get());
-    if (dataInputOutput == null) {
-      return EmptyIterable.get();
-    } else {
-      return new MessagesIterable<M>(dataInputOutput, messageValueFactory);
-    }
-  }
-
-  @Override
-  public void clearVertexMessages(LongWritable vertexId) throws IOException {
-    getPartitionMap(vertexId).remove(vertexId.get());
-  }
-
-  @Override
-  public void clearAll() throws IOException {
-    map.clear();
-  }
-
-  @Override
-  public Iterable<LongWritable> getPartitionDestinationVertices(
-      int partitionId) {
-    Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
-        map.get(partitionId);
-    List<LongWritable> vertices =
-        Lists.newArrayListWithCapacity(partitionMap.size());
-    LongIterator iterator = partitionMap.keySet().iterator();
-    while (iterator.hasNext()) {
-      vertices.add(new LongWritable(iterator.nextLong()));
-    }
-    return vertices;
-  }
-
-  @Override
-  public void writePartition(DataOutput out,
-      int partitionId) throws IOException {
-    Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
-        map.get(partitionId);
-    out.writeInt(partitionMap.size());
-    ObjectIterator<Long2ObjectMap.Entry<DataInputOutput>> iterator =
-        partitionMap.long2ObjectEntrySet().fastIterator();
-    while (iterator.hasNext()) {
-      Long2ObjectMap.Entry<DataInputOutput> entry = iterator.next();
-      out.writeLong(entry.getLongKey());
-      entry.getValue().write(out);
-    }
-  }
-
-  @Override
-  public void readFieldsForPartition(DataInput in,
-      int partitionId) throws IOException {
-    int size = in.readInt();
-    Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
-        new Long2ObjectOpenHashMap<DataInputOutput>(size);
-    while (size-- > 0) {
-      long vertexId = in.readLong();
-      DataInputOutput dataInputOutput = config.createMessagesInputOutput();
-      dataInputOutput.readFields(in);
-      partitionMap.put(vertexId, dataInputOutput);
-    }
-    synchronized (map) {
-      map.put(partitionId, partitionMap);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
index 264e65a..b0452c1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
@@ -83,6 +83,11 @@ public class LongDoubleMessageStore
     }
   }
 
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
   /**
    * Get map which holds messages for partition which vertex belongs to.
    *
@@ -123,6 +128,10 @@ public class LongDoubleMessageStore
   }
 
   @Override
+  public void finalizeStore() {
+  }
+
+  @Override
   public void clearPartition(int partitionId) throws IOException {
     map.get(partitionId).clear();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractListMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractListMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractListMessageStore.java
new file mode 100644
index 0000000..ae61de4
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractListMessageStore.java
@@ -0,0 +1,164 @@
+/*
+ * 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.comm.messages.primitives.long_id;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.partition.PartitionOwner;
+import org.apache.giraph.utils.VertexIdIterator;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import java.util.List;
+
+/**
+ * Special message store to be used when ids are LongWritable and no combiner
+ * is used.
+ * Uses fastutil primitive maps in order to decrease number of objects and
+ * get better performance.
+ *
+ * @param <M> message type
+ * @param <L> list type
+ */
+public abstract class LongAbstractListMessageStore<M extends Writable,
+  L extends List> extends LongAbstractMessageStore<M, L> {
+  /**
+   * Map used to store messages for nascent vertices i.e., ones
+   * that did not exist at the start of current superstep but will get
+   * created because of sending message to a non-existent vertex id
+   */
+  private final
+  Int2ObjectOpenHashMap<Long2ObjectOpenHashMap<L>> nascentMap;
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Factory for creating message values
+   * @param service             Service worker
+   * @param config              Hadoop configuration
+   */
+  public LongAbstractListMessageStore(
+      MessageValueFactory<M> messageValueFactory,
+      CentralizedServiceWorker<LongWritable, Writable, Writable> service,
+      ImmutableClassesGiraphConfiguration<LongWritable,
+          Writable, Writable> config) {
+    super(messageValueFactory, service, config);
+    populateMap();
+
+    // create map for vertex ids (i.e., nascent vertices) not known yet
+    nascentMap = new Int2ObjectOpenHashMap<>();
+    for (int partitionId : service.getPartitionStore().getPartitionIds()) {
+      nascentMap.put(partitionId, new Long2ObjectOpenHashMap<L>());
+    }
+  }
+
+  /**
+   * Populate the map with all vertexIds for each partition
+   */
+  private void populateMap() { // TODO - can parallelize?
+    // populate with vertex ids already known
+    for (int partitionId : service.getPartitionStore().getPartitionIds()) {
+      Partition<LongWritable, ?, ?> partition = service.getPartitionStore()
+          .getOrCreatePartition(partitionId);
+      Long2ObjectOpenHashMap<L> partitionMap = map.get(partitionId);
+      for (Vertex<LongWritable, ?, ?> vertex : partition) {
+        partitionMap.put(vertex.getId().get(), createList());
+      }
+    }
+  }
+
+  /**
+   * Create an instance of L
+   * @return instance of L
+   */
+  protected abstract L createList();
+
+  /**
+   * Get list for the current vertexId
+   *
+   * @param iterator vertexId iterator
+   * @return list for current vertexId
+   */
+  protected L getList(
+    VertexIdIterator<LongWritable> iterator) {
+    PartitionOwner owner =
+        service.getVertexPartitionOwner(iterator.getCurrentVertexId());
+    long vertexId = iterator.getCurrentVertexId().get();
+    int partitionId = owner.getPartitionId();
+    Long2ObjectOpenHashMap<L> partitionMap = map.get(partitionId);
+    if (!partitionMap.containsKey(vertexId)) {
+      synchronized (nascentMap) {
+        // assumption: not many nascent vertices are created
+        // so overall synchronization is negligible
+        Long2ObjectOpenHashMap<L> nascentPartitionMap =
+          nascentMap.get(partitionId);
+        if (nascentPartitionMap.get(vertexId) == null) {
+          nascentPartitionMap.put(vertexId, createList());
+        }
+        return nascentPartitionMap.get(vertexId);
+      }
+    }
+    return partitionMap.get(vertexId);
+  }
+
+  @Override
+  public void finalizeStore() {
+    for (int partitionId : nascentMap.keySet()) {
+      // nascent vertices are present only in nascent map
+      map.get(partitionId).putAll(nascentMap.get(partitionId));
+    }
+    nascentMap.clear();
+  }
+
+  // TODO - discussion
+  /*
+  some approaches for ensuring correctness with parallel inserts
+  - current approach: uses a small extra bit of memory by pre-populating
+  map & pushes everything map cannot handle to nascentMap
+  at the beginning of next superstep compute a single threaded finalizeStore is
+  called (so little extra memory + 1 sequential finish ops)
+  - used striped parallel fast utils instead (unsure of perf)
+  - use concurrent map (every get gets far slower)
+  - use reader writer locks (unsure of perf)
+  (code looks something like underneath)
+
+      private final ReadWriteLock rwl = new ReentrantReadWriteLock();
+      rwl.readLock().lock();
+      L list = partitionMap.get(vertexId);
+      if (list == null) {
+        rwl.readLock().unlock();
+        rwl.writeLock().lock();
+        if (partitionMap.get(vertexId) == null) {
+          list = createList();
+          partitionMap.put(vertexId, list);
+        }
+        rwl.readLock().lock();
+        rwl.writeLock().unlock();
+      }
+      rwl.readLock().unlock();
+  - adopted from the article
+    http://docs.oracle.com/javase/1.5.0/docs/api/java/util/concurrent/locks/\
+    ReentrantReadWriteLock.html
+   */
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractMessageStore.java
new file mode 100644
index 0000000..9ee090e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongAbstractMessageStore.java
@@ -0,0 +1,132 @@
+/*
+ * 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.comm.messages.primitives.long_id;
+
+import com.google.common.collect.Lists;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.longs.LongIterator;
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.partition.Partition;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Special message store to be used when ids are LongWritable and no combiner
+ * is used.
+ * Uses fastutil primitive maps in order to decrease number of objects and
+ * get better performance.
+ *
+ * @param <M> message type
+ * @param <T> datastructure used to hold messages
+ */
+public abstract class LongAbstractMessageStore<M extends Writable, T>
+  implements MessageStore<LongWritable, M> {
+  /** Message value factory */
+  protected final MessageValueFactory<M> messageValueFactory;
+  /** Map from partition id to map from vertex id to message */
+  protected final
+  Int2ObjectOpenHashMap<Long2ObjectOpenHashMap<T>> map;
+  /** Service worker */
+  protected final CentralizedServiceWorker<LongWritable, ?, ?> service;
+  /** Giraph configuration */
+  protected final ImmutableClassesGiraphConfiguration<LongWritable, ?, ?>
+  config;
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Factory for creating message values
+   * @param service      Service worker
+   * @param config       Hadoop configuration
+   */
+  public LongAbstractMessageStore(
+      MessageValueFactory<M> messageValueFactory,
+      CentralizedServiceWorker<LongWritable, Writable, Writable> service,
+      ImmutableClassesGiraphConfiguration<LongWritable, Writable, Writable>
+          config) {
+    this.messageValueFactory = messageValueFactory;
+    this.service = service;
+    this.config = config;
+
+    map = new Int2ObjectOpenHashMap<>();
+    for (int partitionId : service.getPartitionStore().getPartitionIds()) {
+      Partition<LongWritable, Writable, Writable> partition =
+          service.getPartitionStore().getOrCreatePartition(partitionId);
+      Long2ObjectOpenHashMap<T> partitionMap =
+          new Long2ObjectOpenHashMap<T>(
+              (int) partition.getVertexCount());
+      map.put(partitionId, partitionMap);
+      service.getPartitionStore().putPartition(partition);
+    }
+  }
+
+  /**
+   * Get map which holds messages for partition which vertex belongs to.
+   *
+   * @param vertexId Id of the vertex
+   * @return Map which holds messages for partition which vertex belongs to.
+   */
+  protected Long2ObjectOpenHashMap<T> getPartitionMap(
+      LongWritable vertexId) {
+    return map.get(service.getPartitionId(vertexId));
+  }
+
+  @Override
+  public void clearPartition(int partitionId) throws IOException {
+    map.get(partitionId).clear();
+  }
+
+  @Override
+  public boolean hasMessagesForVertex(LongWritable vertexId) {
+    return getPartitionMap(vertexId).containsKey(vertexId.get());
+  }
+
+  @Override
+  public void clearVertexMessages(LongWritable vertexId) throws IOException {
+    getPartitionMap(vertexId).remove(vertexId.get());
+  }
+
+
+  @Override
+  public void clearAll() throws IOException {
+    map.clear();
+  }
+
+  @Override
+  public Iterable<LongWritable> getPartitionDestinationVertices(
+      int partitionId) {
+    Long2ObjectOpenHashMap<T> partitionMap =
+        map.get(partitionId);
+    List<LongWritable> vertices =
+        Lists.newArrayListWithCapacity(partitionMap.size());
+    LongIterator iterator = partitionMap.keySet().iterator();
+    while (iterator.hasNext()) {
+      vertices.add(new LongWritable(iterator.nextLong()));
+    }
+    return vertices;
+  }
+
+}


[12/47] GIRAPH-912: Support succinct representation of messages in messagestores (pavanka)

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongByteArrayMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongByteArrayMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongByteArrayMessageStore.java
new file mode 100644
index 0000000..092d963
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongByteArrayMessageStore.java
@@ -0,0 +1,172 @@
+/*
+ * 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.comm.messages.primitives.long_id;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.messages.MessagesIterable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.utils.VertexIdMessageBytesIterator;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.giraph.utils.VerboseByteStructMessageWrite;
+import org.apache.giraph.utils.EmptyIterable;
+import org.apache.giraph.utils.io.DataInputOutput;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
+import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Special message store to be used when ids are LongWritable and no combiner
+ * is used.
+ * Uses fastutil primitive maps in order to decrease number of objects and
+ * get better performance.
+ *
+ * @param <M> Message type
+ */
+public class LongByteArrayMessageStore<M extends Writable>
+  extends LongAbstractMessageStore<M, DataInputOutput> {
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Factory for creating message values
+   * @param service             Service worker
+   * @param config              Hadoop configuration
+   */
+  public LongByteArrayMessageStore(
+      MessageValueFactory<M> messageValueFactory,
+      CentralizedServiceWorker<LongWritable, Writable, Writable> service,
+      ImmutableClassesGiraphConfiguration<LongWritable,
+          Writable, Writable> config) {
+    super(messageValueFactory, service, config);
+  }
+
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+
+  /**
+   * Get the DataInputOutput for a vertex id, creating if necessary.
+   *
+   * @param partitionMap Partition map to look in
+   * @param vertexId Id of the vertex
+   * @return DataInputOutput for this vertex id (created if necessary)
+   */
+  private DataInputOutput getDataInputOutput(
+    Long2ObjectOpenHashMap<DataInputOutput> partitionMap, long vertexId) {
+    DataInputOutput dataInputOutput = partitionMap.get(vertexId);
+    if (dataInputOutput == null) {
+      dataInputOutput = config.createMessagesInputOutput();
+      partitionMap.put(vertexId, dataInputOutput);
+    }
+    return dataInputOutput;
+  }
+
+  @Override
+  public void addPartitionMessages(int partitionId,
+    VertexIdMessages<LongWritable, M> messages) throws IOException {
+    Long2ObjectOpenHashMap<DataInputOutput> partitionMap = map.get(partitionId);
+    synchronized (partitionMap) {
+      VertexIdMessageBytesIterator<LongWritable, M>
+          vertexIdMessageBytesIterator =
+          messages.getVertexIdMessageBytesIterator();
+      // Try to copy the message buffer over rather than
+      // doing a deserialization of a message just to know its size.  This
+      // should be more efficient for complex objects where serialization is
+      // expensive.  If this type of iterator is not available, fall back to
+      // deserializing/serializing the messages
+      if (vertexIdMessageBytesIterator != null) {
+        while (vertexIdMessageBytesIterator.hasNext()) {
+          vertexIdMessageBytesIterator.next();
+          DataInputOutput dataInputOutput = getDataInputOutput(partitionMap,
+              vertexIdMessageBytesIterator.getCurrentVertexId().get());
+          vertexIdMessageBytesIterator.writeCurrentMessageBytes(
+              dataInputOutput.getDataOutput());
+        }
+      } else {
+        VertexIdMessageIterator<LongWritable, M>
+            iterator = messages.getVertexIdMessageIterator();
+        while (iterator.hasNext()) {
+          iterator.next();
+          DataInputOutput dataInputOutput =  getDataInputOutput(partitionMap,
+              iterator.getCurrentVertexId().get());
+          VerboseByteStructMessageWrite.verboseWriteCurrentMessage(iterator,
+              dataInputOutput.getDataOutput());
+        }
+      }
+    }
+  }
+
+  @Override
+  public void finalizeStore() {
+  }
+
+  @Override
+  public Iterable<M> getVertexMessages(
+    LongWritable vertexId) throws IOException {
+    DataInputOutput dataInputOutput =
+        getPartitionMap(vertexId).get(vertexId.get());
+    if (dataInputOutput == null) {
+      return EmptyIterable.get();
+    } else {
+      return new MessagesIterable<M>(dataInputOutput, messageValueFactory);
+    }
+  }
+
+  @Override
+  public void writePartition(DataOutput out, int partitionId)
+    throws IOException {
+    Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
+        map.get(partitionId);
+    out.writeInt(partitionMap.size());
+    ObjectIterator<Long2ObjectMap.Entry<DataInputOutput>> iterator =
+        partitionMap.long2ObjectEntrySet().fastIterator();
+    while (iterator.hasNext()) {
+      Long2ObjectMap.Entry<DataInputOutput> entry = iterator.next();
+      out.writeLong(entry.getLongKey());
+      entry.getValue().write(out);
+    }
+  }
+
+  @Override
+  public void readFieldsForPartition(DataInput in,
+    int partitionId) throws IOException {
+    int size = in.readInt();
+    Long2ObjectOpenHashMap<DataInputOutput> partitionMap =
+        new Long2ObjectOpenHashMap<DataInputOutput>(size);
+    while (size-- > 0) {
+      long vertexId = in.readLong();
+      DataInputOutput dataInputOutput = config.createMessagesInputOutput();
+      dataInputOutput.readFields(in);
+      partitionMap.put(vertexId, dataInputOutput);
+    }
+    synchronized (map) {
+      map.put(partitionId, partitionMap);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongPointerListMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongPointerListMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongPointerListMessageStore.java
new file mode 100644
index 0000000..32296ad
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/LongPointerListMessageStore.java
@@ -0,0 +1,129 @@
+/*
+ * 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.comm.messages.primitives.long_id;
+
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.comm.messages.PointerListMessagesIterable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.utils.EmptyIterable;
+import org.apache.giraph.utils.ExtendedByteArrayOutputBuffer;
+import org.apache.giraph.utils.ExtendedDataOutput;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import static org.apache.giraph.utils.ExtendedByteArrayOutputBuffer.IndexAndDataOut;
+
+/**
+ * This stores messages in
+ * {@link org.apache.giraph.utils.ExtendedByteArrayOutputBuffer}
+ * and stores long pointers that point to serialized messages
+ *
+ * @param <M> message type
+ */
+public class LongPointerListMessageStore<M extends Writable>
+  extends LongAbstractListMessageStore<M, LongArrayList>
+  implements MessageStore<LongWritable, M> {
+
+  /** Buffers of byte array outputs used to store messages - thread safe */
+  private final ExtendedByteArrayOutputBuffer bytesBuffer;
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Factory for creating message values
+   * @param service             Service worker
+   * @param config              Hadoop configuration
+   */
+  public LongPointerListMessageStore(
+    MessageValueFactory<M> messageValueFactory,
+    CentralizedServiceWorker<LongWritable, Writable, Writable> service,
+    ImmutableClassesGiraphConfiguration<LongWritable,
+    Writable, Writable> config) {
+    super(messageValueFactory, service, config);
+    bytesBuffer = new ExtendedByteArrayOutputBuffer(config);
+  }
+
+  @Override
+  public boolean isPointerListEncoding() {
+    return true;
+  }
+
+  @Override
+  protected LongArrayList createList() {
+    return new LongArrayList();
+  }
+
+  @Override
+  public void addPartitionMessages(int partitionId,
+    VertexIdMessages<LongWritable, M> messages) throws IOException {
+    VertexIdMessageIterator<LongWritable, M> iterator =
+        messages.getVertexIdMessageIterator();
+    long pointer = 0;
+    LongArrayList list;
+    while (iterator.hasNext()) {
+      iterator.next();
+      M msg = iterator.getCurrentMessage();
+      list = getList(iterator);
+      if (iterator.isNewMessage()) {
+        IndexAndDataOut indexAndDataOut = bytesBuffer.getIndexAndDataOut();
+        pointer = indexAndDataOut.getIndex();
+        pointer <<= 32;
+        ExtendedDataOutput dataOutput = indexAndDataOut.getDataOutput();
+        pointer += dataOutput.getPos();
+        msg.write(dataOutput);
+      }
+      synchronized (list) { // TODO - any better way?
+        list.add(pointer);
+      }
+    }
+  }
+
+  @Override
+  public Iterable<M> getVertexMessages(
+    LongWritable vertexId) throws IOException {
+    LongArrayList list = getPartitionMap(vertexId).get(
+        vertexId.get());
+    if (list == null) {
+      return EmptyIterable.get();
+    } else {
+      return new PointerListMessagesIterable<>(messageValueFactory,
+        list, bytesBuffer);
+    }
+  }
+
+  // FIXME -- complete these for check-pointing
+  @Override
+  public void writePartition(DataOutput out, int partitionId)
+    throws IOException {
+  }
+
+  @Override
+  public void readFieldsForPartition(DataInput in, int partitionId)
+    throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/package-info.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/package-info.java
new file mode 100644
index 0000000..121d1db
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/long_id/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Message store based off of primitives when I = LongWritable
+ */
+package org.apache.giraph.comm.messages.primitives.long_id;

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/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 ef3f824..f762f46 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
@@ -22,9 +22,9 @@ import com.yammer.metrics.core.Gauge;
 import com.yammer.metrics.util.PercentGauge;
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.SendOneMessageToManyCache;
 import org.apache.giraph.comm.SendEdgeCache;
 import org.apache.giraph.comm.SendMessageCache;
-import org.apache.giraph.comm.SendMessageToAllCache;
 import org.apache.giraph.comm.SendMutationsCache;
 import org.apache.giraph.comm.SendPartitionCache;
 import org.apache.giraph.comm.ServerData;
@@ -134,9 +134,9 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
         GiraphConfiguration.MAX_MSG_REQUEST_SIZE.get(conf);
     maxVerticesSizePerWorker =
         GiraphConfiguration.MAX_VERTEX_REQUEST_SIZE.get(conf);
-    if (this.configuration.isOneToAllMsgSendingEnabled()) {
+    if (this.configuration.useOneMessageToManyIdsEncoding()) {
       sendMessageCache =
-        new SendMessageToAllCache<I, Writable>(conf, serviceWorker,
+        new SendOneMessageToManyCache<I, Writable>(conf, serviceWorker,
           this, maxMessagesSizePerWorker);
     } else {
       sendMessageCache =
@@ -395,7 +395,7 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
   @Override
   public void flush() throws IOException {
     // Execute the remaining sends messages (if any)
-    // including one-to-one and one-to-all messages.
+    // including individual and compact messages.
     sendMessageCache.flush();
 
     // Execute the remaining sends vertices (if any)

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/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 22ecc0e..81c892d 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
@@ -122,7 +122,7 @@ public class NettyWorkerServer<I extends WritableComparable,
 
   @Override
   public void prepareSuperstep() {
-    serverData.prepareSuperstep();
+    serverData.prepareSuperstep(); // updates the current message-store
     resolveMutations();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
index 408295c..c7561ee 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
@@ -38,9 +38,9 @@ public enum RequestType {
   SEND_WORKER_VERTICES_REQUEST(SendWorkerVerticesRequest.class),
   /** Sending a partition of messages for next superstep */
   SEND_WORKER_MESSAGES_REQUEST(SendWorkerMessagesRequest.class),
-  /** Sending one-to-all messages to a worker for next superstep */
-  SEND_WORKER_ONETOALL_MESSAGES_REQUEST(
-    SendWorkerOneToAllMessagesRequest.class),
+  /** Sending one message to many ids in a single request */
+  SEND_WORKER_ONE_MESSAGE_TO_MANY_REQUEST(
+      SendWorkerOneMessageToManyRequest.class),
   /**
    * Sending a partition of messages for current superstep
    * (used during partition exchange)

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneMessageToManyRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneMessageToManyRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneMessageToManyRequest.java
new file mode 100644
index 0000000..798ddfa
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneMessageToManyRequest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.comm.requests;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.ServerData;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.partition.PartitionOwner;
+import org.apache.giraph.utils.ByteArrayOneMessageToManyIds;
+import org.apache.giraph.utils.ByteArrayVertexIdMessages;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Send a collection of ByteArrayOneMessageToManyIds messages to a worker.
+ *
+ * @param <I> Vertex id
+ * @param <M> Message data
+ */
+@SuppressWarnings("unchecked")
+public class SendWorkerOneMessageToManyRequest<I extends WritableComparable,
+    M extends Writable> extends WritableRequest<I, Writable, Writable>
+    implements WorkerRequest<I, Writable, Writable> {
+  /** ByteArrayOneMessageToManyIds encoding of vertexId & messages */
+  protected ByteArrayOneMessageToManyIds<I, M> oneMessageToManyIds;
+
+  /**
+   * Constructor used for reflection only.
+   */
+  public SendWorkerOneMessageToManyRequest() { }
+
+  /**
+   * Constructor used to send request.
+   *
+   * @param oneMessageToManyIds ByteArrayOneMessageToManyIds
+   * @param conf ImmutableClassesGiraphConfiguration
+   */
+  public SendWorkerOneMessageToManyRequest(
+      ByteArrayOneMessageToManyIds<I, M> oneMessageToManyIds,
+      ImmutableClassesGiraphConfiguration conf) {
+    this.oneMessageToManyIds = oneMessageToManyIds;
+    setConf(conf);
+  }
+
+  @Override
+  public RequestType getType() {
+    return RequestType.SEND_WORKER_ONE_MESSAGE_TO_MANY_REQUEST;
+  }
+
+  @Override
+  public void readFieldsRequest(DataInput input) throws IOException {
+    oneMessageToManyIds = new ByteArrayOneMessageToManyIds<I, M>(
+      getConf().<M>getOutgoingMessageValueFactory());
+    oneMessageToManyIds.setConf(getConf());
+    oneMessageToManyIds.readFields(input);
+  }
+
+  @Override
+  public void writeRequest(DataOutput output) throws IOException {
+    this.oneMessageToManyIds.write(output);
+  }
+
+  @Override
+  public int getSerializedSize() {
+    return super.getSerializedSize() +
+        this.oneMessageToManyIds.getSerializedSize();
+  }
+
+  @Override
+  public void doRequest(ServerData serverData) {
+    try {
+      MessageStore<I, M> messageStore = serverData.getIncomingMessageStore();
+      if (messageStore.isPointerListEncoding()) {
+        // if message store is pointer list based then send data as is
+        messageStore.addPartitionMessages(-1, oneMessageToManyIds);
+      } else { // else split the data per partition and send individually
+        CentralizedServiceWorker<I, ?, ?> serviceWorker =
+            serverData.getServiceWorker();
+        // Get the initial size of ByteArrayVertexIdMessages per partition
+        // on this worker. To make sure every ByteArrayVertexIdMessages to have
+        // enough space to store the messages, we divide the original
+        // ByteArrayOneMessageToManyIds message size by the number of partitions
+        // and double the size
+        // (Assume the major component in ByteArrayOneMessageToManyIds message
+        // is a id list. Now each target id has a copy of message,
+        // therefore we double the buffer size)
+        // to get the initial size of ByteArrayVertexIdMessages.
+        int initialSize = oneMessageToManyIds.getSize() /
+            serverData.getPartitionStore().getNumPartitions() * 2;
+        // Create ByteArrayVertexIdMessages for
+        // message reformatting.
+        Int2ObjectOpenHashMap<ByteArrayVertexIdMessages> partitionIdMsgs =
+            new Int2ObjectOpenHashMap<>();
+
+        // Put data from ByteArrayOneMessageToManyIds
+        // to ByteArrayVertexIdMessages
+        VertexIdMessageIterator<I, M> vertexIdMessageIterator =
+          oneMessageToManyIds.getVertexIdMessageIterator();
+        while (vertexIdMessageIterator.hasNext()) {
+          vertexIdMessageIterator.next();
+          M msg = vertexIdMessageIterator.getCurrentMessage();
+          I vertexId = vertexIdMessageIterator.getCurrentVertexId();
+          PartitionOwner owner =
+              serviceWorker.getVertexPartitionOwner(vertexId);
+          int partitionId = owner.getPartitionId();
+          ByteArrayVertexIdMessages<I, M> idMsgs = partitionIdMsgs
+              .get(partitionId);
+          if (idMsgs == null) {
+            idMsgs = new ByteArrayVertexIdMessages<>(
+                getConf().<M>getOutgoingMessageValueFactory());
+            idMsgs.setConf(getConf());
+            idMsgs.initialize(initialSize);
+            partitionIdMsgs.put(partitionId, idMsgs);
+          }
+          idMsgs.add(vertexId, msg);
+        }
+
+        // Read ByteArrayVertexIdMessages and write to message store
+        for (Entry<Integer, ByteArrayVertexIdMessages> idMsgs :
+            partitionIdMsgs.entrySet()) {
+          if (!idMsgs.getValue().isEmpty()) {
+            serverData.getIncomingMessageStore().addPartitionMessages(
+                idMsgs.getKey(), idMsgs.getValue());
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new IllegalStateException("doRequest: Got IOException ", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
deleted file mode 100644
index 5f1ed53..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
+++ /dev/null
@@ -1,155 +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.comm.requests;
-
-import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.comm.ServerData;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.partition.PartitionOwner;
-import org.apache.giraph.utils.ByteArrayOneToAllMessages;
-import org.apache.giraph.utils.ByteArrayVertexIdMessages;
-import org.apache.giraph.utils.ExtendedDataInput;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Send a collection of one-to-all messages to a worker.
- *
- * @param <I> Vertex id
- * @param <M> Message data
- */
-@SuppressWarnings("unchecked")
-public class SendWorkerOneToAllMessagesRequest<I extends WritableComparable,
-    M extends Writable> extends WritableRequest<I, Writable, Writable>
-    implements WorkerRequest<I, Writable, Writable> {
-  /** The byte array of one-to-all messages */
-  private ByteArrayOneToAllMessages<I, M> oneToAllMsgs;
-
-  /**
-   * Constructor used for reflection only.
-   */
-  public SendWorkerOneToAllMessagesRequest() { }
-
-  /**
-   * Constructor used to send request.
-   *
-   * @param oneToAllMsgs A byte array of all one-to-all messages
-   * @param conf ImmutableClassesGiraphConfiguration
-   */
-  public SendWorkerOneToAllMessagesRequest(
-      ByteArrayOneToAllMessages<I, M> oneToAllMsgs,
-      ImmutableClassesGiraphConfiguration conf) {
-    this.oneToAllMsgs = oneToAllMsgs;
-    setConf(conf);
-  }
-
-  @Override
-  public RequestType getType() {
-    return RequestType.SEND_WORKER_ONETOALL_MESSAGES_REQUEST;
-  }
-
-  @Override
-  public void readFieldsRequest(DataInput input) throws IOException {
-    oneToAllMsgs = new ByteArrayOneToAllMessages<I, M>(
-      getConf().<M>getOutgoingMessageValueFactory());
-    oneToAllMsgs.setConf(getConf());
-    oneToAllMsgs.readFields(input);
-  }
-
-  @Override
-  public void writeRequest(DataOutput output) throws IOException {
-    this.oneToAllMsgs.write(output);
-  }
-
-  @Override
-  public int getSerializedSize() {
-    return super.getSerializedSize() + this.oneToAllMsgs.getSerializedSize();
-  }
-
-  @Override
-  public void doRequest(ServerData serverData) {
-    CentralizedServiceWorker<I, ?, ?> serviceWorker =
-      serverData.getServiceWorker();
-    // Get the initial size of ByteArrayVertexIdMessages per partition
-    // on this worker. To make sure every ByteArrayVertexIdMessages to have
-    // enough space to store the messages, we divide the original one-to-all
-    // message size by the number of partitions and double the size
-    // (Assume the major component in one-to-all message is a id list.
-    // Now each target id has a copy of message,
-    // therefore we double the buffer size)
-    // to get the initial size of ByteArrayVertexIdMessages.
-    int initialSize = oneToAllMsgs.getSize() /
-      serverData.getPartitionStore().getNumPartitions() * 2;
-    // Create ByteArrayVertexIdMessages for
-    // message reformatting.
-    Int2ObjectOpenHashMap<ByteArrayVertexIdMessages>
-      partitionIdMsgs =
-        new Int2ObjectOpenHashMap<ByteArrayVertexIdMessages>();
-
-    // Put data from ByteArrayOneToAllMessages to ByteArrayVertexIdMessages
-    ExtendedDataInput reader = oneToAllMsgs.getOneToAllMessagesReader();
-    I vertexId = getConf().createVertexId();
-    M msg = oneToAllMsgs.createMessage();
-    int idCount = 0;
-    int partitionId = 0;
-    try {
-      while (!reader.endOfInput()) {
-        msg.readFields(reader);
-        idCount = reader.readInt();
-        for (int i = 0; i < idCount; i++) {
-          vertexId.readFields(reader);
-          PartitionOwner owner =
-            serviceWorker.getVertexPartitionOwner(vertexId);
-          partitionId = owner.getPartitionId();
-          ByteArrayVertexIdMessages<I, M> idMsgs =
-            partitionIdMsgs.get(partitionId);
-          if (idMsgs == null) {
-            idMsgs = new ByteArrayVertexIdMessages<I, M>(
-              getConf().<M>getOutgoingMessageValueFactory());
-            idMsgs.setConf(getConf());
-            idMsgs.initialize(initialSize);
-            partitionIdMsgs.put(partitionId, idMsgs);
-          }
-          idMsgs.add(vertexId, msg);
-        }
-      }
-    } catch (IOException e) {
-      throw new RuntimeException("doRequest: Got IOException ", e);
-    }
-    // Read ByteArrayVertexIdMessages and write to message store
-    try {
-      for (Entry<Integer, ByteArrayVertexIdMessages> idMsgs :
-          partitionIdMsgs.entrySet()) {
-        if (!idMsgs.getValue().isEmpty()) {
-          serverData.getIncomingMessageStore().addPartitionMessages(
-            idMsgs.getKey(), idMsgs.getValue());
-        }
-      }
-    } catch (IOException e) {
-      throw new RuntimeException("doRequest: Got IOException.", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/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 ee88b04..953f49f 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
@@ -1176,23 +1176,18 @@ public class GiraphConfiguration extends Configuration
   }
 
   /**
-   * Enable communication optimization for one-to-all messages.
-   */
-  public void enableOneToAllMsgSending() {
-    ONE_TO_ALL_MSG_SENDING.set(this, true);
-  }
-
-  /**
-   * Return if one-to-all messsage sending is enabled.
+   * Return if oneMessageToManyIds encoding can be enabled
    *
-   * @return True if this option is enabled.
+   * @return True if this option is true.
    */
-  public boolean isOneToAllMsgSendingEnabled() {
-    return ONE_TO_ALL_MSG_SENDING.isTrue(this);
+  public boolean useOneMessageToManyIdsEncoding() {
+    return MESSAGE_ENCODE_AND_STORE_TYPE.get(this)
+      .useOneMessageToManyIdsEncoding();
   }
 
   /**
    * Get option whether to create a source vertex present only in edge input
+   *
    * @return CREATE_EDGE_SOURCE_VERTICES option
    */
   public boolean getCreateSourceVertex() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 1879a25..ab0570f 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
@@ -21,6 +21,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.aggregators.TextAggregatorWriter;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.comm.messages.InMemoryMessageStoreFactory;
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
 import org.apache.giraph.comm.messages.MessageStoreFactory;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.EdgeStoreFactory;
@@ -556,12 +557,6 @@ public interface GiraphConstants {
       new IntConfOption("giraph.nettyRequestEncoderBufferSize", 32 * ONE_KB,
           "How big to make the encoder buffer?");
 
-  /** Whether or not netty request encoder should use direct byte buffers */
-  BooleanConfOption NETTY_REQUEST_ENCODER_USE_DIRECT_BUFFERS =
-      new BooleanConfOption("giraph.nettyRequestEncoderUseDirectBuffers",
-                            false, "Whether or not netty request encoder " +
-                                   "should use direct byte buffers");
-
   /** Netty client threads */
   IntConfOption NETTY_CLIENT_THREADS =
       new IntConfOption("giraph.nettyClientThreads", 4, "Netty client threads");
@@ -1054,13 +1049,14 @@ public interface GiraphConstants {
           "edges every time.");
 
   /**
-   * This option will enable communication optimization for one-to-all
-   * message sending. For multiple target ids on the same machine,
-   * we only send one message to all the targets.
+   * This option will tell which message encode & store enum to use when
+   * combining is not enabled
    */
-  BooleanConfOption ONE_TO_ALL_MSG_SENDING =
-    new BooleanConfOption("giraph.oneToAllMsgSending", false, "Enable " +
-        "one-to-all message sending strategy");
+  EnumConfOption<MessageEncodeAndStoreType> MESSAGE_ENCODE_AND_STORE_TYPE =
+      EnumConfOption.create("giraph.messageEncodeAndStoreType",
+          MessageEncodeAndStoreType.class,
+          MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION,
+          "Select the message_encode_and_store_type to use");
 
   /**
    * This option can be used to specify if a source vertex present in edge

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneMessageToManyIds.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneMessageToManyIds.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneMessageToManyIds.java
new file mode 100644
index 0000000..674b0b0
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneMessageToManyIds.java
@@ -0,0 +1,105 @@
+/*
+ * 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.utils;
+
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.IOException;
+
+/**
+ * Stores a message and a list of target vertex ids.
+ *
+ * @param <I> Vertex id
+ * @param <M> Message data
+ */
+@SuppressWarnings("unchecked")
+public class ByteArrayOneMessageToManyIds<I extends WritableComparable,
+  M extends Writable> extends ByteArrayVertexIdData<I, M>
+  implements VertexIdMessages<I, M> {
+  /** Message value class */
+  private MessageValueFactory<M> messageValueFactory;
+
+  /**
+   * Constructor.
+   *
+   * @param messageValueFactory Class for messages
+   */
+  public ByteArrayOneMessageToManyIds(
+      MessageValueFactory<M> messageValueFactory) {
+    this.messageValueFactory = messageValueFactory;
+  }
+
+  @Override
+  public M createData() {
+    return messageValueFactory.newInstance();
+  }
+
+  @Override
+  public void writeData(ExtendedDataOutput out, M message) throws IOException {
+    message.write(out);
+  }
+
+  @Override
+  public void readData(ExtendedDataInput in, M message) throws IOException {
+    message.readFields(in);
+  }
+
+  /**
+   * Add a message.
+   * The order is: the message>id count>ids .
+   *
+   * @param ids   The byte array which holds target ids
+   *              of this message on the worker
+   * @param idPos The end position of the ids
+   *              information in the byte array above.
+   * @param count The number of ids
+   * @param msg   The message sent
+   */
+  public void add(byte[] ids, int idPos, int count, M msg) {
+    try {
+      msg.write(extendedDataOutput);
+      extendedDataOutput.writeInt(count);
+      extendedDataOutput.write(ids, 0, idPos);
+    } catch (IOException e) {
+      throw new IllegalStateException("add: IOException", e);
+    }
+  }
+
+  @Override
+  public void add(I vertexId, M data) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void add(byte[] serializedId, int idPos, M data) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public VertexIdMessageBytesIterator<I, M> getVertexIdMessageBytesIterator() {
+    return null;
+  }
+
+  @Override
+  public VertexIdMessageIterator<I, M> getVertexIdMessageIterator() {
+    return new OneMessageToManyIdsIterator<>(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneToAllMessages.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneToAllMessages.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneToAllMessages.java
deleted file mode 100644
index f190c17..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayOneToAllMessages.java
+++ /dev/null
@@ -1,168 +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.utils;
-
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.factories.MessageValueFactory;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-/**
- * Stores a message and a list of target vertex ids.
- *
- * @param <I> Vertex id
- * @param <M> Message data
- */
-@SuppressWarnings("unchecked")
-public class ByteArrayOneToAllMessages<
-    I extends WritableComparable, M extends Writable>
-    implements Writable, ImmutableClassesGiraphConfigurable {
-  /** Extended data output */
-  private ExtendedDataOutput extendedDataOutput;
-  /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, ?, ?> configuration;
-  /** Message value class */
-  private MessageValueFactory<M> messageValueFactory;
-
-  /**
-   * Constructor.
-   *
-   * @param messageValueFactory Class for messages
-   */
-  public ByteArrayOneToAllMessages(
-      MessageValueFactory<M> messageValueFactory) {
-    this.messageValueFactory = messageValueFactory;
-  }
-
-  /**
-   * Initialize the inner state. Must be called before {@code add()} is called.
-   */
-  public void initialize() {
-    extendedDataOutput = configuration.createExtendedDataOutput();
-  }
-
-  /**
-   * Initialize the inner state, with a known size. Must be called before
-   * {@code add()} is called.
-   *
-   * @param expectedSize Number of bytes to be expected
-   */
-  public void initialize(int expectedSize) {
-    extendedDataOutput = configuration.createExtendedDataOutput(expectedSize);
-  }
-
-  @Override
-  public void setConf(ImmutableClassesGiraphConfiguration configuration) {
-    this.configuration = configuration;
-  }
-
-  @Override
-  public ImmutableClassesGiraphConfiguration getConf() {
-    return this.configuration;
-  }
-
-  /**
-   * Add a message.
-   * The order is: the message>id count>ids .
-   *
-   * @param ids The byte array which holds target ids
-   *                     of this message on the worker
-   * @param idPos The end position of the ids
-   *                     information in the byte array above.
-   * @param count The number of ids
-   * @param msg The message sent
-   */
-  public void add(byte[] ids, int idPos, int count, M msg) {
-    try {
-      msg.write(extendedDataOutput);
-      extendedDataOutput.writeInt(count);
-      extendedDataOutput.write(ids, 0, idPos);
-    } catch (IOException e) {
-      throw new IllegalStateException("add: IOException", e);
-    }
-  }
-
-  /**
-   * Create a message.
-   *
-   * @return A created message object.
-   */
-  public M createMessage() {
-    return messageValueFactory.newInstance();
-  }
-
-  /**
-   * Get the number of bytes used.
-   *
-   * @return Bytes used
-   */
-  public int getSize() {
-    return extendedDataOutput.getPos();
-  }
-
-  /**
-   * Get the size of ByteArrayOneToAllMessages after serialization.
-   * Here 4 is the size of an integer which represents the size of whole
-   * byte array.
-   *
-   * @return The size (in bytes) of the serialized object
-   */
-  public int getSerializedSize() {
-    return  4 + getSize();
-  }
-
-  @Override
-  public void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeInt(extendedDataOutput.getPos());
-    dataOutput.write(extendedDataOutput.getByteArray(), 0,
-      extendedDataOutput.getPos());
-  }
-
-  @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    int size = dataInput.readInt();
-    byte[] buf = new byte[size];
-    dataInput.readFully(buf);
-    extendedDataOutput = configuration.createExtendedDataOutput(buf, size);
-  }
-
-  /**
-   * Check if the byte array is empty.
-   *
-   * @return True if the position of the byte array is 0.
-   */
-  public boolean isEmpty() {
-    return extendedDataOutput.getPos() == 0;
-  }
-
-  /**
-   * Get the reader of this OneToAllMessages
-   *
-   * @return ExtendedDataInput
-   */
-  public ExtendedDataInput getOneToAllMessagesReader() {
-    return configuration.createExtendedDataInput(
-      extendedDataOutput.getByteArray(), 0, extendedDataOutput.getPos());
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdDataIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdDataIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdDataIterator.java
index cefec0e..962bc75 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdDataIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdDataIterator.java
@@ -39,6 +39,8 @@ public class ByteStructVertexIdDataIterator<I extends WritableComparable, T>
   extends ByteStructVertexIdIterator<I> implements VertexIdDataIterator<I, T> {
   /** VertexIdData to iterate over */
   protected AbstractVertexIdData<I, T> vertexIdData;
+  /** Serialized size of the message object in bytestore */
+  protected int dataSize;
   /** Current data. */
   private T data;
 
@@ -63,13 +65,20 @@ public class ByteStructVertexIdDataIterator<I extends WritableComparable, T>
     }
     try {
       vertexId.readFields(extendedDataInput);
+      int initial = extendedDataInput.getPos();
       vertexIdData.readData(extendedDataInput, data);
+      dataSize = extendedDataInput.getPos() - initial;
     } catch (IOException e) {
       throw new IllegalStateException("next: IOException", e);
     }
   }
 
   @Override
+  public int getCurrentDataSize() {
+    return dataSize;
+  }
+
+  @Override
   public T getCurrentData() {
     return data;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdMessageIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdMessageIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdMessageIterator.java
index b686211..dd91ea2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdMessageIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdMessageIterator.java
@@ -46,4 +46,14 @@ public class ByteStructVertexIdMessageIterator<I extends WritableComparable,
   public M getCurrentMessage() {
     return getCurrentData();
   }
+
+  @Override
+  public int getCurrentMessageSize() {
+    return getCurrentDataSize();
+  }
+
+  @Override
+  public boolean isNewMessage() {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayOutputBuffer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayOutputBuffer.java b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayOutputBuffer.java
new file mode 100644
index 0000000..80c3aee
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayOutputBuffer.java
@@ -0,0 +1,155 @@
+/*
+ * 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.utils;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.giraph.conf.FloatConfOption;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.IntConfOption;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Wraps a list of byte array outputs and provides convenient
+ * utilities on top of it
+ */
+public class ExtendedByteArrayOutputBuffer {
+  /**
+   * This option sets the capacity of an
+   * {@link org.apache.giraph.utils.ExtendedDataOutput} instance created in
+   * {@link org.apache.giraph.utils.ExtendedByteArrayOutputBuffer}
+   */
+  public static final IntConfOption CAPACITY_OF_DATAOUT_IN_BUFFER =
+      new IntConfOption("giraph.capacityOfDataOutInBuffer",
+          1024 * GiraphConstants.ONE_KB,
+          "Set the capacity of dataoutputs in dataout buffer");
+
+  /**
+   * This option sets the maximum fraction of a
+   * {@link org.apache.giraph.utils.ExtendedDataOutput} instance (stored in
+   * {@link org.apache.giraph.utils.ExtendedByteArrayOutputBuffer})
+   * that can be filled
+   */
+  public static final FloatConfOption FILLING_THRESHOLD_OF_DATAOUT_IN_BUFFER =
+      new FloatConfOption("giraph.fillingThresholdOfDataoutInBuffer", 0.98f,
+          "Set the maximum fraction of dataoutput capacity allowed to fill");
+
+  /** Maximum size allowed for one byte array output */
+  private final int maxBufSize;
+  /** Stop writing to buffer after threshold has been reached */
+  private final int threshold;
+  /** Giraph configuration */
+  private final ImmutableClassesGiraphConfiguration<?, ? , ?> config;
+
+  /** Map of index => byte array outputs */
+  private final Int2ObjectOpenHashMap<ExtendedDataOutput>
+  bytearrayOutputs = new Int2ObjectOpenHashMap<>();
+  /** Size of byte array outputs map */
+  private final AtomicInteger mapSize = new AtomicInteger(0);
+  /** Thread local variable to get hold of a byte array output stream */
+  private final ThreadLocal<IndexAndDataOut> threadLocal =
+      new ThreadLocal<IndexAndDataOut>() {
+        @Override
+        protected IndexAndDataOut initialValue() {
+          return newIndexAndDataOutput();
+        }
+      };
+
+  /**
+   * Constructor
+   *
+   * @param config configuration
+   */
+  public ExtendedByteArrayOutputBuffer(
+    ImmutableClassesGiraphConfiguration<?, ?, ?> config) {
+    this.config = config;
+
+    maxBufSize = CAPACITY_OF_DATAOUT_IN_BUFFER.get(config);
+    threshold = (int) (FILLING_THRESHOLD_OF_DATAOUT_IN_BUFFER.get(config) *
+      maxBufSize);
+  }
+
+  /**
+   * Return threadLocal indexAndDataOutput instance
+   *
+   * @return threadLocal indexAndDataOutput instance
+   */
+  public IndexAndDataOut getIndexAndDataOut() {
+    IndexAndDataOut indexAndDataOut = threadLocal.get();
+    if (indexAndDataOut.dataOutput.getPos() >= threshold) {
+      indexAndDataOut = newIndexAndDataOutput();
+      threadLocal.set(indexAndDataOut);
+    }
+    return indexAndDataOut;
+  }
+
+  /**
+   * Get dataoutput from bytearrayOutputs
+   *
+   * @param index index in bytearrayOutputs
+   * @return extendeddataoutput at given index
+   */
+  public ExtendedDataOutput getDataOutput(int index) {
+    return bytearrayOutputs.get(index);
+  }
+
+  /**
+   * Holder for index & DataOutput objects
+   */
+  public static class IndexAndDataOut {
+    /** Index */
+    private final int index;
+    /** Dataouput instance */
+    private final ExtendedDataOutput dataOutput;
+
+    /**
+     * Constructor
+     *
+     * @param index index in bytearrayOutputs
+     * @param dataOutput dataoutput
+     */
+    public IndexAndDataOut(int index, ExtendedDataOutput dataOutput) {
+      this.index = index;
+      this.dataOutput = dataOutput;
+    }
+
+    public int getIndex() {
+      return index;
+    }
+
+    public ExtendedDataOutput getDataOutput() {
+      return dataOutput;
+    }
+  }
+
+  /**
+   * Create a new IndexAndDataOutput instance
+   * @return new IndexAndDataOutput instance
+   */
+  private IndexAndDataOut newIndexAndDataOutput() {
+    int index = mapSize.getAndIncrement();
+    ExtendedDataOutput output = config.createExtendedDataOutput(
+        maxBufSize);
+    synchronized (bytearrayOutputs) {
+      bytearrayOutputs.put(index, output);
+    }
+    return new IndexAndDataOut(index, output);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataOutput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataOutput.java b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataOutput.java
index bc979af..0da9681 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataOutput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataOutput.java
@@ -82,4 +82,3 @@ public interface ExtendedDataOutput extends DataOutput {
    */
   void reset();
 }
-

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/OneMessageToManyIdsIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/OneMessageToManyIdsIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/OneMessageToManyIdsIterator.java
new file mode 100644
index 0000000..f353b2d
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/OneMessageToManyIdsIterator.java
@@ -0,0 +1,143 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.IOException;
+
+/**
+ * VertexIdData iterator for
+ * {@link ByteArrayOneMessageToManyIds}
+ *
+ * @param <I> vertexId type
+ * @param <M> message type
+ */
+public class OneMessageToManyIdsIterator<I extends WritableComparable,
+    M extends Writable> implements VertexIdMessageIterator<I, M> {
+  /** VertexIdMessages object to iterate over */
+  private final ByteArrayOneMessageToManyIds<I, M> vertexIdMessages;
+  /** Reader of the serialized edges */
+  private final ExtendedDataInput extendedDataInput;
+
+  /** Current vertex Id*/
+  private I vertexId;
+  /** Current message */
+  private M msg;
+  /** Counts of ids left to read before next message */
+  private int idsToRead = 0;
+  /** Size of message read */
+  private int msgSize = 0;
+  /** Is current message newly read */
+  private boolean newMessage;
+
+  /**
+   * Constructor
+   *
+   * @param vertexIdMessages vertexId messages object to iterate over
+   */
+  public OneMessageToManyIdsIterator(
+      final ByteArrayOneMessageToManyIds<I, M> vertexIdMessages) {
+    this.vertexIdMessages = vertexIdMessages;
+    this.extendedDataInput = vertexIdMessages.getConf()
+        .createExtendedDataInput(vertexIdMessages.extendedDataOutput);
+  }
+
+  @Override
+  public I getCurrentVertexId() {
+    return vertexId;
+  }
+
+  @Override
+  public M getCurrentMessage() {
+    return getCurrentData();
+  }
+
+  @Override
+  public M getCurrentData() {
+    return msg;
+  }
+
+  @Override
+  public M releaseCurrentData() {
+    M releasedData = msg;
+    msg = null;
+    return releasedData;
+  }
+
+  @Override
+  public I releaseCurrentVertexId() {
+    I releasedVertexId = vertexId;
+    vertexId = null;
+    return releasedVertexId;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return extendedDataInput.available() > 0;
+  }
+
+  /**
+   * Properly initialize vertexId & msg object before calling next()
+   */
+  private void initialize() {
+    if (vertexId == null) {
+      vertexId = vertexIdMessages.getConf().createVertexId();
+    }
+    if (msg == null) {
+      msg = vertexIdMessages.createData();
+    }
+  }
+
+  @Override
+  public void next() {
+    initialize();
+    try {
+      if (idsToRead == 0) {
+        newMessage = true; // a new message is read
+        int initial = extendedDataInput.getPos();
+        msg.readFields(extendedDataInput);
+        msgSize = extendedDataInput.getPos() - initial;
+        idsToRead = extendedDataInput.readInt();
+      } else {
+        newMessage = false; // same as previous message
+      }
+      vertexId.readFields(extendedDataInput);
+      idsToRead -= 1;
+    } catch (IOException e) {
+      throw new IllegalStateException("next: IOException", e);
+    }
+  }
+
+  @Override
+  public int getCurrentMessageSize() {
+    return getCurrentDataSize();
+  }
+
+  @Override
+  public int getCurrentDataSize() {
+    return msgSize;
+  }
+
+  @Override
+  public boolean isNewMessage() {
+    return newMessage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
index 1ab8de6..c5587e1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
@@ -55,7 +55,7 @@ public class UnsafeArrayReads extends UnsafeReads {
       UNSAFE.arrayBaseOffset(byte[].class);
 
   /** Byte buffer */
-  private final byte[] buf;
+  protected byte[] buf;
 
   /**
    * Constructor

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReads.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReads.java b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReads.java
index 5f99846..39ab352 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReads.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReads.java
@@ -28,7 +28,7 @@ import java.io.UTFDataFormatException;
 public abstract class UnsafeReads implements ExtendedDataInput {
 
   /** Buffer length */
-  protected final int bufLength;
+  protected int bufLength;
   /** Position in the buffer */
   protected long pos = 0;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReusableByteArrayInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReusableByteArrayInput.java b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReusableByteArrayInput.java
new file mode 100644
index 0000000..a75815a
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeReusableByteArrayInput.java
@@ -0,0 +1,46 @@
+/*
+ * 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.utils;
+
+/**
+ * UnsafeReusableByteArrayInput is a data structure to read from a
+ * byte buffer with a read pointer that can be moved to desired location
+ */
+public class UnsafeReusableByteArrayInput extends UnsafeArrayReads {
+
+  /**
+   * Default Constructor
+   */
+  public UnsafeReusableByteArrayInput() {
+    super(null, 0, 0);
+  }
+
+  /**
+   * Initialize the object with all required parameters
+   *
+   * @param buf byte buffer
+   * @param offset offset in the buffer
+   * @param length length of the valid data
+   */
+  public void initialize(byte[] buf, int offset, int length) {
+    this.buf = buf;
+    this.pos = offset;
+    this.bufLength = length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdDataIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdDataIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdDataIterator.java
index 6aea8ea..80792a5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdDataIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdDataIterator.java
@@ -41,6 +41,13 @@ public interface VertexIdDataIterator<I extends WritableComparable, T>
   T getCurrentData();
 
   /**
+   * Get serialized size of current data
+   *
+   * @return serialized size of data
+   */
+  int getCurrentDataSize();
+
+  /**
    * Release the current data object.
    *
    * @return Released data object

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdMessageIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdMessageIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdMessageIterator.java
index c241cea..288f7ce 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdMessageIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdMessageIterator.java
@@ -36,4 +36,18 @@ public interface VertexIdMessageIterator<I extends WritableComparable,
    * @return Current message
    */
   M getCurrentMessage();
+
+  /**
+   * Get the serialized size of current message
+   *
+   * @return serialized size of current message
+   */
+  int getCurrentMessageSize();
+
+  /**
+   * Return true of current message is new
+   *
+   * @return true if current message is new
+   */
+  boolean isNewMessage();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
index 8037db9..b56bab3 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
@@ -24,7 +24,7 @@ import org.apache.giraph.comm.netty.handler.WorkerRequestServerHandler;
 import org.apache.giraph.comm.requests.SendPartitionMutationsRequest;
 import org.apache.giraph.comm.requests.SendVertexRequest;
 import org.apache.giraph.comm.requests.SendWorkerMessagesRequest;
-import org.apache.giraph.comm.requests.SendWorkerOneToAllMessagesRequest;
+import org.apache.giraph.comm.requests.SendWorkerOneMessageToManyRequest;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
@@ -36,8 +36,8 @@ import org.apache.giraph.graph.VertexMutations;
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionStore;
+import org.apache.giraph.utils.ByteArrayOneMessageToManyIds;
 import org.apache.giraph.utils.VertexIdMessages;
-import org.apache.giraph.utils.ByteArrayOneToAllMessages;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
 import org.apache.giraph.utils.ExtendedDataOutput;
 import org.apache.giraph.utils.IntNoOpComputation;
@@ -196,10 +196,10 @@ public class RequestTest {
   }
 
   @Test
-  public void sendWorkerOneToAllMessagesRequest() throws IOException {
+  public void sendWorkerIndividualMessagesRequest() throws IOException {
     // Data to send
-    ByteArrayOneToAllMessages<IntWritable, IntWritable>
-        dataToSend = new ByteArrayOneToAllMessages<>(new
+    ByteArrayOneMessageToManyIds<IntWritable, IntWritable>
+        dataToSend = new ByteArrayOneMessageToManyIds<>(new
         TestMessageValueFactory<>(IntWritable.class));
     dataToSend.setConf(conf);
     dataToSend.initialize();
@@ -211,8 +211,8 @@ public class RequestTest {
     dataToSend.add(output.getByteArray(), output.getPos(), 7, new IntWritable(1));
 
     // Send the request
-    SendWorkerOneToAllMessagesRequest<IntWritable, IntWritable> request =
-      new SendWorkerOneToAllMessagesRequest<>(dataToSend, conf);
+    SendWorkerOneMessageToManyRequest<IntWritable, IntWritable> request =
+      new SendWorkerOneMessageToManyRequest<>(dataToSend, conf);
     client.sendWritableRequest(workerInfo.getTaskId(), request);
     client.waitAllRequests();
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f31e9a32/giraph-core/src/test/java/org/apache/giraph/comm/messages/TestLongDoublePrimitiveMessageStores.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/messages/TestLongDoublePrimitiveMessageStores.java b/giraph-core/src/test/java/org/apache/giraph/comm/messages/TestLongDoublePrimitiveMessageStores.java
index d3c392e..5903eb8 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/messages/TestLongDoublePrimitiveMessageStores.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/messages/TestLongDoublePrimitiveMessageStores.java
@@ -25,7 +25,7 @@ import junit.framework.Assert;
 
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.combiner.DoubleSumMessageCombiner;
-import org.apache.giraph.comm.messages.primitives.LongByteArrayMessageStore;
+import org.apache.giraph.comm.messages.primitives.long_id.LongByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.LongDoubleMessageStore;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;


[29/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-931: Provide a Strongly Connected Components algorithm (gianluca via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/de0efb07
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/de0efb07
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/de0efb07

Branch: refs/heads/release-1.1
Commit: de0efb07518082439b9a5cccd503270b09f40e84
Parents: 5adca63
Author: Maja Kabiljo <ma...@fb.com>
Authored: Tue Aug 26 12:09:54 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Tue Aug 26 12:11:00 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 giraph-examples/pom.xml                         |   4 +
 .../giraph/examples/scc/SccComputation.java     | 213 +++++++++++++++++++
 .../scc/SccLongLongNullTextInputFormat.java     |  90 ++++++++
 .../examples/scc/SccPhaseMasterCompute.java     | 136 ++++++++++++
 .../giraph/examples/scc/SccVertexValue.java     | 157 ++++++++++++++
 .../giraph/examples/scc/package-info.java       |  21 ++
 .../scc/SccComputationTestInMemory.java         | 128 +++++++++++
 8 files changed, 751 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index b64ce2c..d5b284e 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-931: Provide a Strongly Connected Components algorithm (gianluca via majakabiljo)
+
   GIRAPH-933: Checkpointing improvements (edunov via majakabiljo)
 
   GIRAPH-943: Perf regression due to netty 4.0.21 (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-examples/pom.xml b/giraph-examples/pom.xml
index 90f6889..f8304a1 100644
--- a/giraph-examples/pom.xml
+++ b/giraph-examples/pom.xml
@@ -415,6 +415,10 @@ under the License.
       <artifactId>commons-collections</artifactId>
     </dependency>
     <dependency>
+    	<groupId>it.unimi.dsi</groupId>
+    	<artifactId>fastutil</artifactId>
+    </dependency>
+    <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccComputation.java
new file mode 100644
index 0000000..ca194f6
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccComputation.java
@@ -0,0 +1,213 @@
+/*
+ * 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.scc;
+
+import static org.apache.giraph.examples.scc.SccPhaseMasterCompute.PHASE;
+import static org.apache.giraph.examples.scc.SccPhaseMasterCompute.NEW_MAXIMUM;
+import static org.apache.giraph.examples.scc.SccPhaseMasterCompute.CONVERGED;
+
+import java.io.IOException;
+
+import org.apache.giraph.Algorithm;
+import org.apache.giraph.examples.scc.SccPhaseMasterCompute.Phases;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * Finds strongly connected components of the graph.
+ */
+@Algorithm(name = "Strongly Connected Components",
+           description = "Finds strongly connected components of the graph")
+public class SccComputation extends
+    BasicComputation<LongWritable, SccVertexValue, NullWritable, LongWritable> {
+
+  /**
+   * Current phase of the computation as defined in SccPhaseMasterCompute
+   */
+  private Phases currPhase;
+
+  /**
+   * Reusable object to encapsulate message value, in order to avoid
+   * creating a new instance every time a message is sent.
+   */
+  private LongWritable messageValue = new LongWritable();
+
+  /**
+   * Reusable object to encapsulate a parent vertex id.
+   */
+  private LongWritable parentId = new LongWritable();
+
+  @Override
+  public void preSuperstep() {
+    IntWritable phaseInt = getAggregatedValue(PHASE);
+    currPhase = SccPhaseMasterCompute.getPhase(phaseInt);
+  }
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, SccVertexValue, NullWritable> vertex,
+      Iterable<LongWritable> messages) throws IOException {
+
+    SccVertexValue vertexValue = vertex.getValue();
+
+    if (!vertexValue.isActive()) {
+      vertex.voteToHalt();
+      return;
+    }
+
+    switch (currPhase) {
+    case TRANSPOSE :
+      vertexValue.clearParents();
+      sendMessageToAllEdges(vertex, vertex.getId());
+      break;
+    case TRIMMING :
+      trim(vertex, messages);
+      break;
+    case FORWARD_TRAVERSAL :
+      forwardTraversal(vertex, messages);
+      break;
+    case BACKWARD_TRAVERSAL_START :
+      backwardTraversalStart(vertex);
+      break;
+    case BACKWARD_TRAVERSAL_REST :
+      backwardTraversalRest(vertex, messages);
+      break;
+    default :
+      break;
+    }
+
+  }
+
+  /**
+   * Creates list of parents based on the received ids and halts the vertices
+   * that don't have any parent or outgoing edge, hence, they can't be
+   * part of an SCC.
+   * @param vertex Current vertex.
+   * @param messages Received ids from the Transpose phase.
+   */
+  private void trim(Vertex<LongWritable, SccVertexValue, NullWritable> vertex,
+                    Iterable<LongWritable> messages) {
+    SccVertexValue vertexValue = vertex.getValue();
+    // Keep the ids of the parent nodes to allow for backwards traversal
+    for (LongWritable parent : messages) {
+      vertexValue.addParent(parent.get());
+    }
+    // If this node doesn't have any parents or outgoing edges,
+    // it can't be part of an SCC
+    vertexValue.set(vertex.getId().get());
+    if (vertex.getNumEdges() == 0 || vertexValue.getParents() == null) {
+      vertexValue.deactivate();
+    } else {
+      messageValue.set(vertexValue.get());
+      sendMessageToAllEdges(vertex, messageValue);
+    }
+  }
+
+  /**
+   * Traverse the graph through outgoing edges and keep the maximum vertex
+   * value.
+   * If a new maximum value is found, propagate it until convergence.
+   * @param vertex Current vertex.
+   * @param messages Received values from neighbor vertices.
+   */
+  private void forwardTraversal(
+      Vertex<LongWritable, SccVertexValue, NullWritable> vertex,
+      Iterable<LongWritable> messages) {
+    SccVertexValue vertexValue = vertex.getValue();
+    boolean changed = setMaxValue(vertexValue, messages);
+    if (changed) {
+      messageValue.set(vertexValue.get());
+      sendMessageToAllEdges(vertex, messageValue);
+      aggregate(NEW_MAXIMUM, new BooleanWritable(true));
+    }
+  }
+
+  /**
+   * Traverse the transposed graph and keep the maximum vertex value.
+   * @param vertex Current vertex.
+   */
+  private void backwardTraversalStart(
+      Vertex<LongWritable, SccVertexValue, NullWritable> vertex) {
+    SccVertexValue vertexValue = vertex.getValue();
+    if (vertexValue.get() == vertex.getId().get()) {
+      messageValue.set(vertexValue.get());
+      sendMessageToAllParents(vertex, messageValue);
+    }
+  }
+
+  /**
+   * Traverse the transposed graph and keep the maximum vertex value.
+   * @param vertex Current vertex.
+   * @param messages Received values from children vertices.
+   */
+  private void backwardTraversalRest(
+      Vertex<LongWritable, SccVertexValue, NullWritable> vertex,
+      Iterable<LongWritable> messages) {
+    SccVertexValue vertexValue = vertex.getValue();
+    for (LongWritable m : messages) {
+      if (vertexValue.get() == m.get()) {
+        sendMessageToAllParents(vertex, m);
+        aggregate(CONVERGED, new BooleanWritable(true));
+        vertexValue.deactivate();
+        vertex.voteToHalt();
+        break;
+      }
+    }
+  }
+
+  /**
+   * Compares the messages values with the current vertex value and finds
+   * the maximum.
+   * If the maximum value is different from the vertex value, makes it the
+   * new vertex value and returns true, otherwise, returns false.
+   * @param vertexValue Current vertex value.
+   * @param messages Messages containing neighbors' vertex values.
+   * @return True if a new maximum was found, otherwise, returns false.
+   */
+  private boolean setMaxValue(SccVertexValue vertexValue,
+                              Iterable<LongWritable> messages) {
+    boolean changed = false;
+    for (LongWritable m : messages) {
+      if (vertexValue.get() < m.get()) {
+        vertexValue.set(m.get());
+        changed = true;
+      }
+    }
+    return changed;
+  }
+
+
+  /**
+   * Send message to all parents.
+   * @param vertex Current vertex.
+   * @param message Message to be sent.
+   */
+  private void sendMessageToAllParents(
+      Vertex<LongWritable, SccVertexValue, NullWritable> vertex,
+      LongWritable message) {
+    for (Long id : vertex.getValue().getParents()) {
+      parentId.set(id);
+      sendMessage(parentId, message);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccLongLongNullTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccLongLongNullTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccLongLongNullTextInputFormat.java
new file mode 100644
index 0000000..e5a4c86
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccLongLongNullTextInputFormat.java
@@ -0,0 +1,90 @@
+/*
+ * 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.scc;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
+ * unweighted graphs with long ids.
+ *
+ * Each line consists of: vertex neighbor1 neighbor2 ...
+ */
+public class SccLongLongNullTextInputFormat extends
+    TextVertexInputFormat<LongWritable, SccVertexValue, NullWritable> {
+  /** Separator of the vertex and neighbors */
+  private static final Pattern SEPARATOR = Pattern.compile("[\t ]");
+
+  @Override
+  public TextVertexReader createVertexReader(InputSplit split,
+                                             TaskAttemptContext context)
+    throws IOException {
+    return new LongLongNullVertexReader();
+  }
+
+  /**
+   * Vertex reader associated with {@link LongLongNullLongTextInputFormat}.
+   */
+  public class LongLongNullVertexReader extends
+      TextVertexReaderFromEachLineProcessed<String[]> {
+    /** Cached vertex id for the current line */
+    private LongWritable id;
+
+    @Override
+    protected String[] preprocessLine(Text line) throws IOException {
+      String[] tokens = SEPARATOR.split(line.toString());
+      id = new LongWritable(Long.parseLong(tokens[0]));
+      return tokens;
+    }
+
+    @Override
+    protected LongWritable getId(String[] tokens) throws IOException {
+      return id;
+    }
+
+    @Override
+    protected SccVertexValue getValue(String[] tokens) throws IOException {
+      return new SccVertexValue(Long.parseLong(tokens[0]));
+    }
+
+    @Override
+    protected Iterable<Edge<LongWritable, NullWritable>> getEdges(
+        String[] tokens) throws IOException {
+      List<Edge<LongWritable, NullWritable>> edges =
+          Lists.newArrayListWithCapacity(tokens.length - 1);
+      for (int n = 1; n < tokens.length; n++) {
+        edges.add(EdgeFactory.create(
+            new LongWritable(Long.parseLong(tokens[n]))));
+      }
+      return edges;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccPhaseMasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccPhaseMasterCompute.java b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccPhaseMasterCompute.java
new file mode 100644
index 0000000..f5fd82b
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccPhaseMasterCompute.java
@@ -0,0 +1,136 @@
+/*
+ * 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.scc;
+
+import org.apache.giraph.aggregators.BooleanOverwriteAggregator;
+import org.apache.giraph.aggregators.IntOverwriteAggregator;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.IntWritable;
+
+/**
+ * This master compute keeps track of what phase is being currently executed by
+ * the Strongly Connected Components computation. The phases comprehend the
+ * following: 1 - Transpose (comprehends 2 supersteps, one to propagate parent
+ * vertices ids and another one to store them by their respective children) 2 -
+ * Trimming (this phase can happen multiple times) 3 - Forward Traversal 4 -
+ * Backward Traversal
+ */
+public class SccPhaseMasterCompute extends DefaultMasterCompute {
+
+  /**
+   * Aggregator that stores the current phase
+   */
+  public static final String PHASE = "scccompute.phase";
+
+  /**
+   * Flags whether a new maximum was found in the Forward Traversal phase
+   */
+  public static final String NEW_MAXIMUM = "scccompute.max";
+
+  /**
+   * Flags whether a vertex converged in the Backward Traversal phase
+   */
+  public static final String CONVERGED = "scccompute.converged";
+
+  /**
+   * Enumerates the possible phases of the algorithm.
+   */
+  public enum Phases {
+    /** Tranpose and Trimming phases **/
+    TRANSPOSE, TRIMMING,
+    /** Maximum id propagation **/
+    FORWARD_TRAVERSAL,
+    /** Vertex convergence in SCC **/
+    BACKWARD_TRAVERSAL_START, BACKWARD_TRAVERSAL_REST
+  };
+
+  @Override
+  public void initialize() throws InstantiationException,
+      IllegalAccessException {
+    registerPersistentAggregator(PHASE, IntOverwriteAggregator.class);
+    registerAggregator(NEW_MAXIMUM, BooleanOverwriteAggregator.class);
+    registerAggregator(CONVERGED, BooleanOverwriteAggregator.class);
+  }
+
+  @Override
+  public void compute() {
+    if (getSuperstep() == 0) {
+      setPhase(Phases.TRANSPOSE);
+    } else {
+      Phases currPhase = getPhase();
+      switch (currPhase) {
+      case TRANSPOSE:
+        setPhase(Phases.TRIMMING);
+        break;
+      case TRIMMING :
+        setPhase(Phases.FORWARD_TRAVERSAL);
+        break;
+      case FORWARD_TRAVERSAL :
+        BooleanWritable newMaxFound = getAggregatedValue(NEW_MAXIMUM);
+        // If no new maximum value was found it means the propagation
+        // converged, so we can move to the next phase
+        if (!newMaxFound.get()) {
+          setPhase(Phases.BACKWARD_TRAVERSAL_START);
+        }
+        break;
+      case BACKWARD_TRAVERSAL_START :
+        setPhase(Phases.BACKWARD_TRAVERSAL_REST);
+        break;
+      case BACKWARD_TRAVERSAL_REST :
+        BooleanWritable converged = getAggregatedValue(CONVERGED);
+        if (!converged.get()) {
+          setPhase(Phases.TRANSPOSE);
+        }
+        break;
+      default :
+        break;
+      }
+    }
+  }
+
+  /**
+   * Sets the next phase of the algorithm.
+   * @param phase
+   *          Next phase.
+   */
+  private void setPhase(Phases phase) {
+    setAggregatedValue(PHASE, new IntWritable(phase.ordinal()));
+  }
+
+  /**
+   * Get current phase.
+   * @return Current phase as enumerator.
+   */
+  private Phases getPhase() {
+    IntWritable phaseInt = getAggregatedValue(PHASE);
+    return getPhase(phaseInt);
+  }
+
+  /**
+   * Helper function to convert from internal aggregated value to a Phases
+   * enumerator.
+   * @param phaseInt
+   *          An integer that matches a position in the Phases enumerator.
+   * @return A Phases' item for the given position.
+   */
+  public static Phases getPhase(IntWritable phaseInt) {
+    return Phases.values()[phaseInt.get()];
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccVertexValue.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccVertexValue.java b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccVertexValue.java
new file mode 100644
index 0000000..63c23c5
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/SccVertexValue.java
@@ -0,0 +1,157 @@
+/*
+ * 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.scc;
+
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Vertex value for the Strongly Connected Components algorithm. It keeps track
+ * of the parents of the vertex in order to traverse the graph backwards.
+ */
+public class SccVertexValue implements Writable {
+
+  /** Vertex's parents **/
+  private LongArrayList parents;
+
+  /** Current vertex value **/
+  private long value = Long.MIN_VALUE;
+
+  /** Indicates whether the vertex was trimmed, hence,
+   * it can't be part of the computation anymore.
+   */
+  private boolean active = true;
+
+  /**
+   * Public constructor required for serialization.
+   */
+  public SccVertexValue() {
+  }
+
+  /**
+   * Constructor
+   * @param value Initial value for this vertex.
+   */
+  public SccVertexValue(long value) {
+    this.value = value;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    value = in.readLong();
+
+    int size = in.readInt();
+    if (size != 0) {
+      for (int i = 0; i < size; i++) {
+        addParent(in.readLong());
+      }
+    }
+
+    active = in.readBoolean();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(value);
+
+    int size = parents == null ? 0 : parents.size();
+    out.writeInt(size);
+    if (size != 0) {
+      for (long incomingId : parents) {
+        out.writeLong(incomingId);
+      }
+    }
+
+    out.writeBoolean(active);
+  }
+
+  /**
+   * Returns the list of parent vertices, i.e., vertices that are at the other
+   * end of incoming edges. If the vertex doesn't have any incoming edge, it
+   * returns null.
+   * @return List of the vertex's parents.
+   */
+  public LongArrayList getParents() {
+    return parents;
+  }
+
+  /**
+   * Adds a vertex id to the list of parent vertices.
+   * @param vertexId It of the parent vertex.
+   */
+  public void addParent(long vertexId) {
+    // Initialize the list of parent vertices only when one attempts to add
+    // the first item, so we save some memory on vertices that have no incoming
+    // edges
+    if (parents == null) {
+      parents = new LongArrayList();
+    }
+    parents.add(vertexId);
+  }
+
+  /**
+   * Clear parents list.
+   */
+  public void clearParents() {
+    parents = null;
+  }
+
+  /**
+   * Sets the vertex value. At the end of the SCC computation, vertices with the
+   * same vertex value are part of the same component.
+   * @param value Vertex value.
+   */
+  public void set(long value) {
+    this.value = value;
+  }
+
+  /**
+   * Returns the vertex value. At the end of the SCC computation, vertices with
+   * the same vertex value are part of the same component.
+   * @return Current vertex value.
+   */
+  public long get() {
+    return value;
+  }
+
+  /**
+   * Remove this vertex from the computation.
+   */
+  public void deactivate() {
+    this.active = false;
+  }
+
+  /**
+   * Indicates whether the vertex was removed in a Trimming phase.
+   * @return True if the vertex was trimmed, otherwise, return false.
+   */
+  public boolean isActive() {
+    return active;
+  }
+
+  @Override
+  public String toString() {
+    return String.valueOf(value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/src/main/java/org/apache/giraph/examples/scc/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/scc/package-info.java b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/package-info.java
new file mode 100644
index 0000000..70e345a
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/scc/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Classes for Strongly Connected Components computation.
+ */
+package org.apache.giraph.examples.scc;

http://git-wip-us.apache.org/repos/asf/giraph/blob/de0efb07/giraph-examples/src/test/java/org/apache/giraph/examples/scc/SccComputationTestInMemory.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/scc/SccComputationTestInMemory.java b/giraph-examples/src/test/java/org/apache/giraph/examples/scc/SccComputationTestInMemory.java
new file mode 100644
index 0000000..833c43e
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/scc/SccComputationTestInMemory.java
@@ -0,0 +1,128 @@
+/*
+ * 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.scc;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.apache.giraph.utils.TestGraph;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for {@link SccComputation}
+ */
+public class SccComputationTestInMemory {
+  @SuppressWarnings("unchecked")
+  public static Entry<LongWritable, NullWritable>[] makeEdges(long... args) {
+    Entry<LongWritable, NullWritable> result[] = new Entry[args.length];
+    for (int i = 0; i < args.length; i++) {
+      result[i] = new SimpleEntry<LongWritable, NullWritable>(new LongWritable(
+          args[i]), NullWritable.get());
+    }
+    return result;
+  }
+
+  /**
+   * Connects the {@outgoingVertices} to the given vertex id
+   * with null-valued edges.
+   * 
+   * @param graph
+   * @param id
+   * @param outgoingVertices
+   */
+  public static void addVertex(
+      TestGraph<LongWritable, SccVertexValue, NullWritable> graph, long id,
+      long... outgoingVertices) {
+    graph.addVertex(new LongWritable(id), new SccVertexValue(id),
+        makeEdges(outgoingVertices));
+  }
+
+  @Test
+  public void testToyData() throws Exception {
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setComputationClass(SccComputation.class);
+    conf.setMasterComputeClass(SccPhaseMasterCompute.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+
+
+    TestGraph<LongWritable, SccVertexValue, NullWritable> graph = new TestGraph<LongWritable, SccVertexValue, NullWritable>(
+        conf);
+
+    addVertex(graph, 0, 1, 2, 4);
+    addVertex(graph, 1, 3, 20);
+    addVertex(graph, 2, 3);
+    addVertex(graph, 3, 0);
+    addVertex(graph, 20, 21);
+    addVertex(graph, 21, 22);
+    addVertex(graph, 22, 23);
+    addVertex(graph, 23, 24);
+    addVertex(graph, 24, 25);
+    addVertex(graph, 25, 20);
+    addVertex(graph, 4, 5);
+    addVertex(graph, 5, 6);
+
+    TestGraph<LongWritable, SccVertexValue, NullWritable> results = InternalVertexRunner.runWithInMemoryOutput(conf, graph);
+
+    Map<Long, List<Long>> scc = parse(results);
+
+    List<Long> components = scc.get(3l);
+    Collections.sort(components);
+    Assert.assertEquals(Arrays.asList(0l, 1l, 2l, 3l), components);
+
+    Assert.assertEquals(Arrays.asList(4l), scc.get(4l));
+    Assert.assertEquals(Arrays.asList(5l), scc.get(5l));
+    Assert.assertEquals(Arrays.asList(6l), scc.get(6l));
+
+    components = scc.get(25l);
+    Collections.sort(components);
+    Assert.assertEquals(Arrays.asList(20l, 21l, 22l, 23l, 24l, 25l), components);
+  }
+
+  private Map<Long, List<Long>> parse(
+      TestGraph<LongWritable, SccVertexValue, NullWritable> g) {
+    Map<Long, List<Long>> scc = new HashMap<Long, List<Long>>();
+    for (LongWritable v : g.getVertices().keySet()) {
+      Vertex<LongWritable, SccVertexValue, NullWritable> vertex = g
+          .getVertex(v);
+      long sccId = vertex.getValue().get();
+      List<Long> verticesIds = scc.get(sccId);
+      if (verticesIds == null) {// New SCC
+        List<Long> newScc = new ArrayList<Long>();
+        newScc.add(vertex.getId().get());
+        scc.put(sccId, newScc);
+      } else {
+        verticesIds.add(vertex.getId().get());
+      }
+    }
+    return scc;
+  }
+}


[34/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
removing unnecessary specialization in MessageCombiners (ikabiljo via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/414ef89e
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/414ef89e
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/414ef89e

Branch: refs/heads/release-1.1
Commit: 414ef89ebd00594ab93e410d09ce007eaad3b202
Parents: 1852057
Author: Maja Kabiljo <ma...@fb.com>
Authored: Wed Oct 1 10:21:00 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Wed Oct 1 10:21:00 2014 -0700

----------------------------------------------------------------------
 .../org/apache/giraph/combiner/DoubleSumMessageCombiner.java   | 6 +++---
 .../org/apache/giraph/combiner/FloatSumMessageCombiner.java    | 6 +++---
 .../apache/giraph/combiner/MinimumDoubleMessageCombiner.java   | 6 +++---
 .../org/apache/giraph/combiner/MinimumIntMessageCombiner.java  | 5 +++--
 .../org/apache/giraph/combiner/SimpleSumMessageCombiner.java   | 6 +++---
 5 files changed, 15 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/414ef89e/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
index 163e0d8..c7c97a4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
@@ -19,16 +19,16 @@
 package org.apache.giraph.combiner;
 
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
 
 /**
  * A combiner that sums double-valued messages
  */
 public class DoubleSumMessageCombiner
     extends
-    MessageCombiner<LongWritable, DoubleWritable> {
+    MessageCombiner<WritableComparable, DoubleWritable> {
   @Override
-  public void combine(LongWritable vertexIndex, DoubleWritable originalMessage,
+  public void combine(WritableComparable vertexIndex, DoubleWritable originalMessage,
       DoubleWritable messageToCombine) {
     originalMessage.set(originalMessage.get() + messageToCombine.get());
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/414ef89e/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
index b13a7f7..0bbea4e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
@@ -19,16 +19,16 @@
 package org.apache.giraph.combiner;
 
 import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.WritableComparable;
 
 /**
  * A combiner that sums float-valued messages
  */
 public class FloatSumMessageCombiner
     extends
-    MessageCombiner<IntWritable, FloatWritable> {
+    MessageCombiner<WritableComparable, FloatWritable> {
   @Override
-  public void combine(IntWritable vertexIndex, FloatWritable originalMessage,
+  public void combine(WritableComparable vertexIndex, FloatWritable originalMessage,
       FloatWritable messageToCombine) {
     originalMessage.set(originalMessage.get() + messageToCombine.get());
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/414ef89e/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
index a1f4bd7..ebd9f0a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
@@ -19,16 +19,16 @@
 package org.apache.giraph.combiner;
 
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
 
 /**
  * MessageCombiner which finds the minimum of {@link DoubleWritable}.
  */
 public class MinimumDoubleMessageCombiner
     extends
-    MessageCombiner<LongWritable, DoubleWritable> {
+    MessageCombiner<WritableComparable, DoubleWritable> {
   @Override
-  public void combine(LongWritable vertexIndex, DoubleWritable originalMessage,
+  public void combine(WritableComparable vertexIndex, DoubleWritable originalMessage,
       DoubleWritable messageToCombine) {
     if (originalMessage.get() > messageToCombine.get()) {
       originalMessage.set(messageToCombine.get());

http://git-wip-us.apache.org/repos/asf/giraph/blob/414ef89e/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
index 227c6e6..0aec90a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
@@ -19,14 +19,15 @@
 package org.apache.giraph.combiner;
 
 import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.WritableComparable;
 
 /**
  * {@link MessageCombiner} that finds the minimum {@link IntWritable}
  */
 public class MinimumIntMessageCombiner
-    extends MessageCombiner<IntWritable, IntWritable> {
+    extends MessageCombiner<WritableComparable, IntWritable> {
   @Override
-  public void combine(IntWritable vertexIndex, IntWritable originalMessage,
+  public void combine(WritableComparable vertexIndex, IntWritable originalMessage,
       IntWritable messageToCombine) {
     if (originalMessage.get() > messageToCombine.get()) {
       originalMessage.set(messageToCombine.get());

http://git-wip-us.apache.org/repos/asf/giraph/blob/414ef89e/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
index 1b4f5ef..cd00fbb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
@@ -19,16 +19,16 @@
 package org.apache.giraph.combiner;
 
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
 
 /**
  * MessageCombiner which sums up {@link IntWritable} message values.
  */
 public class SimpleSumMessageCombiner
-    extends MessageCombiner<LongWritable, IntWritable> {
+    extends MessageCombiner<WritableComparable, IntWritable> {
 
   @Override
-  public void combine(LongWritable vertexIndex, IntWritable originalMessage,
+  public void combine(WritableComparable vertexIndex, IntWritable originalMessage,
       IntWritable messageToCombine) {
     originalMessage.set(originalMessage.get() + messageToCombine.get());
   }


[11/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-903: Detect crashes of Netty threads (edunov via pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/61cb37ec
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/61cb37ec
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/61cb37ec

Branch: refs/heads/release-1.1
Commit: 61cb37ecd50b0d9400873624e46692c3282e4cfc
Parents: 7f9218a
Author: Pavan Kumar <pa...@fb.com>
Authored: Tue Jul 8 12:08:53 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Tue Jul 8 12:11:12 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 +
 findbugs-exclude.xml                            |  6 +--
 .../apache/giraph/comm/netty/NettyClient.java   | 38 +++++++++++---
 .../giraph/comm/netty/NettyMasterClient.java    |  8 ++-
 .../giraph/comm/netty/NettyMasterServer.java    |  6 ++-
 .../apache/giraph/comm/netty/NettyServer.java   | 36 ++++++++-----
 .../giraph/comm/netty/NettyWorkerClient.java    |  8 ++-
 .../giraph/comm/netty/NettyWorkerServer.java    |  6 ++-
 .../handler/MasterRequestServerHandler.java     | 11 ++--
 .../netty/handler/RequestServerHandler.java     | 18 ++++---
 .../handler/WorkerRequestServerHandler.java     | 11 ++--
 .../org/apache/giraph/graph/GraphMapper.java    | 23 +++------
 .../apache/giraph/graph/GraphTaskManager.java   | 30 +++++++++++
 .../apache/giraph/master/BspServiceMaster.java  |  9 ++--
 .../org/apache/giraph/utils/ThreadUtils.java    | 54 ++++++++++++++++++++
 .../apache/giraph/worker/BspServiceWorker.java  |  6 ++-
 .../org/apache/giraph/yarn/GiraphYarnTask.java  | 13 -----
 .../org/apache/giraph/comm/ConnectionTest.java  | 26 ++++++----
 .../giraph/comm/MockExceptionHandler.java       | 26 ++++++++++
 .../apache/giraph/comm/RequestFailureTest.java  |  5 +-
 .../org/apache/giraph/comm/RequestTest.java     |  5 +-
 .../apache/giraph/comm/SaslConnectionTest.java  |  6 ++-
 22 files changed, 255 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 834b45f..13dfcd7 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-903: Detect crashes of Netty threads (edunov via pavanka)
+
   GIRAPH-925: Unit tests should pass even if zookeeper port not available (edunov via pavanka)
 
   GIRAPH-713: Provide an option to do request compression (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/findbugs-exclude.xml b/findbugs-exclude.xml
index e0466f7..9ac4412 100644
--- a/findbugs-exclude.xml
+++ b/findbugs-exclude.xml
@@ -39,11 +39,7 @@
     <Bug pattern="DM_EXIT"/>
   </Match>
   <Match>
-    <Class name="org.apache.giraph.graph.GraphMapper$OverrideExceptionHandler"/>
-    <Bug pattern="DM_EXIT"/>
-  </Match>
-  <Match>
-    <Class name="org.apache.giraph.yarn.GiraphYarnTask$OverrideExceptionHandler"/>
+    <Class name="org.apache.giraph.graph.GraphTaskManager$OverrideExceptionHandler"/>
     <Bug pattern="DM_EXIT"/>
   </Match>
   <Match>

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
index 5bb5545..97394bf 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
@@ -35,6 +35,7 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
 import org.apache.giraph.utils.PipelineUtils;
 import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.giraph.utils.ThreadUtils;
 import org.apache.giraph.utils.TimedLogger;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
@@ -42,7 +43,6 @@ import org.apache.log4j.Logger;
 import com.google.common.collect.Lists;
 import com.google.common.collect.MapMaker;
 import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -175,6 +175,12 @@ public class NettyClient {
   /** When was the last time we checked if we should resend some requests */
   private final AtomicLong lastTimeCheckedRequestsForProblems =
       new AtomicLong(0);
+  /**
+   * Logger used to dump stack traces for every exception that happens
+   * in netty client threads.
+   */
+  private final LogOnErrorChannelFutureListener logErrorListener =
+      new LogOnErrorChannelFutureListener();
 
   /**
    * Only constructor
@@ -182,10 +188,13 @@ public class NettyClient {
    * @param context Context for progress
    * @param conf Configuration
    * @param myTaskInfo Current task info
+   * @param exceptionHandler handler for uncaught exception. Will
+   *                         terminate job.
    */
   public NettyClient(Mapper<?, ?, ?, ?>.Context context,
                      final ImmutableClassesGiraphConfiguration conf,
-                     TaskInfo myTaskInfo) {
+                     TaskInfo myTaskInfo,
+                     final Thread.UncaughtExceptionHandler exceptionHandler) {
     this.context = context;
     this.myTaskInfo = myTaskInfo;
     this.channelsPerServer = GiraphConstants.CHANNELS_PER_SERVER.get(conf);
@@ -226,8 +235,8 @@ public class NettyClient {
     if (useExecutionGroup) {
       int executionThreads = NETTY_CLIENT_EXECUTION_THREADS.get(conf);
       executionGroup = new DefaultEventExecutorGroup(executionThreads,
-          new ThreadFactoryBuilder().setNameFormat("netty-client-exec-%d")
-              .build());
+          ThreadUtils.createThreadFactory(
+              "netty-client-exec-%d", exceptionHandler));
       if (LOG.isInfoEnabled()) {
         LOG.info("NettyClient: Using execution handler with " +
             executionThreads + " threads after " +
@@ -238,8 +247,8 @@ public class NettyClient {
     }
 
     workerGroup = new NioEventLoopGroup(maxPoolSize,
-        new ThreadFactoryBuilder().setNameFormat(
-            "netty-client-worker-%d").build());
+        ThreadUtils.createThreadFactory(
+            "netty-client-worker-%d", exceptionHandler));
 
     bootstrap = new Bootstrap();
     bootstrap.group(workerGroup)
@@ -696,6 +705,7 @@ public class NettyClient {
     }
     ChannelFuture writeFuture = channel.write(request);
     newRequestInfo.setWriteFuture(writeFuture);
+    writeFuture.addListener(logErrorListener);
 
     if (limitNumberOfOpenRequests &&
         clientRequestIdRequestInfoMap.size() > maxNumberOfOpenRequests) {
@@ -868,6 +878,7 @@ public class NettyClient {
       }
       ChannelFuture writeFuture = channel.write(requestInfo.getRequest());
       requestInfo.setWriteFuture(writeFuture);
+      writeFuture.addListener(logErrorListener);
     }
     addedRequestIds.clear();
     addedRequestInfos.clear();
@@ -906,4 +917,19 @@ public class NettyClient {
     }
     return address;
   }
+
+  /**
+   * This listener class just dumps exception stack traces if
+   * something happens.
+   */
+  private static class LogOnErrorChannelFutureListener
+      implements ChannelFutureListener {
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isDone() && !future.isSuccess()) {
+        LOG.error("Request failed", future.cause());
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/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 c982209..1218d29 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
@@ -54,12 +54,16 @@ public class NettyMasterClient implements MasterClient {
    * @param context Context from mapper
    * @param configuration Configuration
    * @param service Centralized service
+   * @param exceptionHandler handler for uncaught exception. Will
+   *                         terminate job.
    */
   public NettyMasterClient(Mapper<?, ?, ?, ?>.Context context,
                            ImmutableClassesGiraphConfiguration configuration,
-                           CentralizedServiceMaster<?, ?, ?> service) {
+                           CentralizedServiceMaster<?, ?, ?> service,
+                           Thread.UncaughtExceptionHandler exceptionHandler) {
     this.nettyClient =
-        new NettyClient(context, configuration, service.getMasterInfo());
+        new NettyClient(context, configuration, service.getMasterInfo(),
+            exceptionHandler);
     this.service = service;
     this.progressable = context;
     maxBytesPerAggregatorRequest = configuration.getInt(

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
index cb36c3e..1c05910 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
@@ -39,13 +39,15 @@ public class NettyMasterServer implements MasterServer {
    * @param conf Hadoop configuration
    * @param service Centralized service
    * @param progressable Progressable for reporting progress
+   * @param exceptionHandler to handle uncaught exceptions
    */
   public NettyMasterServer(ImmutableClassesGiraphConfiguration conf,
       CentralizedServiceMaster<?, ?, ?> service,
-      Progressable progressable) {
+      Progressable progressable,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
     nettyServer = new NettyServer(conf,
         new MasterRequestServerHandler.Factory(service.getAggregatorHandler()),
-        service.getMasterInfo(), progressable);
+        service.getMasterInfo(), progressable, exceptionHandler);
     nettyServer.start();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
index 8162857..454232a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
@@ -33,6 +33,7 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
 import org.apache.giraph.utils.PipelineUtils;
 import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.giraph.utils.ThreadUtils;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
 import io.netty.bootstrap.ServerBootstrap;
@@ -54,8 +55,6 @@ import io.netty.util.concurrent.EventExecutorGroup;
 import io.netty.util.concurrent.ImmediateEventExecutor;
 import io.netty.channel.AdaptiveRecvByteBufAllocator;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 
@@ -122,6 +121,8 @@ public class NettyServer {
   private final EventExecutorGroup executionGroup;
   /** Name of the handler before the execution handler (if used) */
   private final String handlerToUseExecutionGroup;
+  /** Handles all uncaught exceptions in netty threads */
+  private final Thread.UncaughtExceptionHandler exceptionHandler;
 
   /**
    * Constructor for creating the server
@@ -130,10 +131,12 @@ public class NettyServer {
    * @param requestServerHandlerFactory Factory for request handlers
    * @param myTaskInfo Current task info
    * @param progressable Progressable for reporting progress
+   * @param exceptionHandler handle uncaught exceptions
    */
   public NettyServer(ImmutableClassesGiraphConfiguration conf,
       RequestServerHandler.Factory requestServerHandlerFactory,
-      TaskInfo myTaskInfo, Progressable progressable) {
+      TaskInfo myTaskInfo, Progressable progressable,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
     this.conf = conf;
     this.progressable = progressable;
     this.requestServerHandlerFactory = requestServerHandlerFactory;
@@ -141,6 +144,7 @@ public class NettyServer {
     this.saslServerHandlerFactory = new SaslServerHandler.Factory();
     /*end[HADOOP_NON_SECURE]*/
     this.myTaskInfo = myTaskInfo;
+    this.exceptionHandler = exceptionHandler;
     sendBufferSize = GiraphConstants.SERVER_SEND_BUFFER_SIZE.get(conf);
     receiveBufferSize = GiraphConstants.SERVER_RECEIVE_BUFFER_SIZE.get(conf);
 
@@ -149,12 +153,12 @@ public class NettyServer {
     maxPoolSize = GiraphConstants.NETTY_SERVER_THREADS.get(conf);
 
     bossGroup = new NioEventLoopGroup(4,
-        new ThreadFactoryBuilder().setNameFormat(
-          "netty-server-boss-%d").build());
+        ThreadUtils.createThreadFactory(
+            "netty-server-boss-%d", exceptionHandler));
 
     workerGroup = new NioEventLoopGroup(maxPoolSize,
-        new ThreadFactoryBuilder().setNameFormat(
-          "netty-server-worker-%d").build());
+        ThreadUtils.createThreadFactory(
+            "netty-server-worker-%d", exceptionHandler));
 
     try {
       this.localHostname = conf.getLocalHostname();
@@ -173,8 +177,8 @@ public class NettyServer {
     if (useExecutionGroup) {
       int executionThreads = conf.getNettyServerExecutionThreads();
       executionGroup = new DefaultEventExecutorGroup(executionThreads,
-          new ThreadFactoryBuilder().setNameFormat("netty-server-exec-%d").
-              build());
+          ThreadUtils.createThreadFactory(
+              "netty-server-exec-%d", exceptionHandler));
       if (LOG.isInfoEnabled()) {
         LOG.info("NettyServer: Using execution group with " +
             executionThreads + " threads for " +
@@ -194,13 +198,16 @@ public class NettyServer {
    * @param myTaskInfo Current task info
    * @param progressable Progressable for reporting progress
    * @param saslServerHandlerFactory  Factory for SASL handlers
+   * @param exceptionHandler handle uncaught exceptions
    */
   public NettyServer(ImmutableClassesGiraphConfiguration conf,
                      RequestServerHandler.Factory requestServerHandlerFactory,
                      TaskInfo myTaskInfo,
                      Progressable progressable,
-                     SaslServerHandler.Factory saslServerHandlerFactory) {
-    this(conf, requestServerHandlerFactory, myTaskInfo, progressable);
+                     SaslServerHandler.Factory saslServerHandlerFactory,
+                     Thread.UncaughtExceptionHandler exceptionHandler) {
+    this(conf, requestServerHandlerFactory, myTaskInfo,
+        progressable, exceptionHandler);
     this.saslServerHandlerFactory = saslServerHandlerFactory;
   }
 /*end[HADOOP_NON_SECURE]*/
@@ -267,8 +274,8 @@ public class NettyServer {
               executionGroup, ch);
           PipelineUtils.addLastWithExecutorCheck("requestServerHandler",
               requestServerHandlerFactory.newHandler(workerRequestReservedMap,
-                  conf, myTaskInfo), handlerToUseExecutionGroup,
-              executionGroup, ch);
+                  conf, myTaskInfo, exceptionHandler),
+              handlerToUseExecutionGroup, executionGroup, ch);
           // Removed after authentication completes:
           PipelineUtils.addLastWithExecutorCheck("responseEncoder",
               new ResponseEncoder(), handlerToUseExecutionGroup,
@@ -310,7 +317,7 @@ public class NettyServer {
               handlerToUseExecutionGroup, executionGroup, ch);
           PipelineUtils.addLastWithExecutorCheck("requestServerHandler",
               requestServerHandlerFactory.newHandler(
-                  workerRequestReservedMap, conf, myTaskInfo),
+                  workerRequestReservedMap, conf, myTaskInfo, exceptionHandler),
               handlerToUseExecutionGroup, executionGroup, ch);
 /*if_not[HADOOP_NON_SECURE]*/
         }
@@ -404,5 +411,6 @@ public class NettyServer {
   public InetSocketAddress getMyAddress() {
     return myAddress;
   }
+
 }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/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 7541418..c893a24 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
@@ -74,13 +74,17 @@ public class NettyWorkerClient<I extends WritableComparable,
    * @param context Context from mapper
    * @param configuration Configuration
    * @param service Used to get partition mapping
+   * @param exceptionHandler handler for uncaught exception. Will
+   *                         terminate job.
    */
   public NettyWorkerClient(
       Mapper<?, ?, ?, ?>.Context context,
       ImmutableClassesGiraphConfiguration<I, V, E> configuration,
-      CentralizedServiceWorker<I, V, E> service) {
+      CentralizedServiceWorker<I, V, E> service,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
     this.nettyClient =
-        new NettyClient(context, configuration, service.getWorkerInfo());
+        new NettyClient(context, configuration, service.getWorkerInfo(),
+            exceptionHandler);
     this.conf = configuration;
     this.service = service;
     this.superstepRequestCounters = Maps.newHashMap();

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/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 adb96cb..22ecc0e 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
@@ -77,10 +77,12 @@ public class NettyWorkerServer<I extends WritableComparable,
    * @param conf Configuration
    * @param service Service to get partition mappings
    * @param context Mapper context
+   * @param exceptionHandler handle uncaught exceptions
    */
   public NettyWorkerServer(ImmutableClassesGiraphConfiguration<I, V, E> conf,
       CentralizedServiceWorker<I, V, E> service,
-      Mapper<?, ?, ?, ?>.Context context) {
+      Mapper<?, ?, ?, ?>.Context context,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
     this.conf = conf;
     this.service = service;
     this.context = context;
@@ -91,7 +93,7 @@ public class NettyWorkerServer<I extends WritableComparable,
 
     nettyServer = new NettyServer(conf,
         new WorkerRequestServerHandler.Factory<I, V, E>(serverData),
-        service.getWorkerInfo(), context);
+        service.getWorkerInfo(), context, exceptionHandler);
     nettyServer.start();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/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 3e06026..e043314 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
@@ -36,13 +36,15 @@ public class MasterRequestServerHandler extends
    * @param conf                     Configuration
    * @param myTaskInfo               Current task info
    * @param aggregatorHandler        Master aggregator handler
+   * @param exceptionHandler         Handles uncaught exceptions
    */
   public MasterRequestServerHandler(
       WorkerRequestReservedMap workerRequestReservedMap,
       ImmutableClassesGiraphConfiguration conf,
       TaskInfo myTaskInfo,
-      MasterAggregatorHandler aggregatorHandler) {
-    super(workerRequestReservedMap, conf, myTaskInfo);
+      MasterAggregatorHandler aggregatorHandler,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
+    super(workerRequestReservedMap, conf, myTaskInfo, exceptionHandler);
     this.aggregatorHandler = aggregatorHandler;
   }
 
@@ -71,9 +73,10 @@ public class MasterRequestServerHandler extends
     public RequestServerHandler newHandler(
         WorkerRequestReservedMap workerRequestReservedMap,
         ImmutableClassesGiraphConfiguration conf,
-        TaskInfo myTaskInfo) {
+        TaskInfo myTaskInfo,
+        Thread.UncaughtExceptionHandler exceptionHandler) {
       return new MasterRequestServerHandler(workerRequestReservedMap, conf,
-          myTaskInfo, aggregatorHandler);
+          myTaskInfo, aggregatorHandler, exceptionHandler);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/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 b6d0533..d75870a 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
@@ -56,6 +56,8 @@ public abstract class RequestServerHandler<R> extends
   private final TaskInfo myTaskInfo;
   /** Start nanoseconds for the processing time */
   private long startProcessingNanoseconds = -1;
+  /** Handler for uncaught exceptions */
+  private final Thread.UncaughtExceptionHandler exceptionHandler;
 
   /**
    * Constructor
@@ -63,14 +65,17 @@ public abstract class RequestServerHandler<R> extends
    * @param workerRequestReservedMap Worker request reservation map
    * @param conf Configuration
    * @param myTaskInfo Current task info
+   * @param exceptionHandler Handles uncaught exceptions
    */
   public RequestServerHandler(
       WorkerRequestReservedMap workerRequestReservedMap,
       ImmutableClassesGiraphConfiguration conf,
-      TaskInfo myTaskInfo) {
+      TaskInfo myTaskInfo,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
     this.workerRequestReservedMap = workerRequestReservedMap;
     closeFirstRequest = NETTY_SIMULATE_FIRST_REQUEST_CLOSED.get(conf);
     this.myTaskInfo = myTaskInfo;
+    this.exceptionHandler = exceptionHandler;
   }
 
   @Override
@@ -159,10 +164,9 @@ public abstract class RequestServerHandler<R> extends
   }
 
   @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
-    throws Exception {
-    LOG.warn("exceptionCaught: Channel failed with " +
-        "remote address " + ctx.channel().remoteAddress(), cause);
+  public void exceptionCaught(
+      ChannelHandlerContext ctx, Throwable cause) throws Exception {
+    exceptionHandler.uncaughtException(Thread.currentThread(), cause);
   }
 
   /**
@@ -175,11 +179,13 @@ public abstract class RequestServerHandler<R> extends
      * @param workerRequestReservedMap Worker request reservation map
      * @param conf Configuration to use
      * @param myTaskInfo Current task info
+     * @param exceptionHandler Handles uncaught exceptions
      * @return New {@link RequestServerHandler}
      */
     RequestServerHandler newHandler(
         WorkerRequestReservedMap workerRequestReservedMap,
         ImmutableClassesGiraphConfiguration conf,
-        TaskInfo myTaskInfo);
+        TaskInfo myTaskInfo,
+        Thread.UncaughtExceptionHandler exceptionHandler);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
index f64c373..574e413 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
@@ -46,12 +46,14 @@ public class WorkerRequestServerHandler<I extends WritableComparable,
    * @param workerRequestReservedMap Worker request reservation map
    * @param conf                     Configuration
    * @param myTaskInfo               Current task info
+   * @param exceptionHandler         Handles uncaught exceptions
    */
   public WorkerRequestServerHandler(ServerData<I, V, E> serverData,
       WorkerRequestReservedMap workerRequestReservedMap,
       ImmutableClassesGiraphConfiguration conf,
-      TaskInfo myTaskInfo) {
-    super(workerRequestReservedMap, conf, myTaskInfo);
+      TaskInfo myTaskInfo,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
+    super(workerRequestReservedMap, conf, myTaskInfo, exceptionHandler);
     this.serverData = serverData;
   }
 
@@ -80,9 +82,10 @@ public class WorkerRequestServerHandler<I extends WritableComparable,
     public RequestServerHandler newHandler(
         WorkerRequestReservedMap workerRequestReservedMap,
         ImmutableClassesGiraphConfiguration conf,
-        TaskInfo myTaskInfo) {
+        TaskInfo myTaskInfo,
+        Thread.UncaughtExceptionHandler exceptionHandler) {
       return new WorkerRequestServerHandler<I, V, E, Writable>(serverData,
-          workerRequestReservedMap, conf, myTaskInfo);
+          workerRequestReservedMap, conf, myTaskInfo, exceptionHandler);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java b/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
index c86a024..6f748c5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
@@ -49,13 +49,15 @@ public class GraphMapper<I extends WritableComparable, V extends Writable,
   @Override
   public void setup(Context context)
     throws IOException, InterruptedException {
-    // Setting the default handler for uncaught exceptions.
-    Thread.setDefaultUncaughtExceptionHandler(
-        new OverrideExceptionHandler());
-
     // Execute all Giraph-related role(s) assigned to this compute node.
     // Roles can include "master," "worker," "zookeeper," or . . . ?
     graphTaskManager = new GraphTaskManager<I, V, E>(context);
+
+    // Setting the default handler for uncaught exceptions.
+    Thread.setDefaultUncaughtExceptionHandler(
+        graphTaskManager.createUncaughtExceptionHandler());
+
+
     graphTaskManager.setup(
       DistributedCache.getLocalCacheArchives(context.getConfiguration()));
   }
@@ -96,6 +98,7 @@ public class GraphMapper<I extends WritableComparable, V extends Writable,
       // CHECKSTYLE: stop IllegalCatch
     } catch (RuntimeException e) {
       // CHECKSTYLE: resume IllegalCatch
+      LOG.error("Caught an unrecoverable exception " + e.getMessage(), e);
       graphTaskManager.zooKeeperCleanup();
       graphTaskManager.workerFailureCleanup();
       throw new IllegalStateException(
@@ -103,16 +106,4 @@ public class GraphMapper<I extends WritableComparable, V extends Writable,
     }
   }
 
-  /**
-    * Default handler for uncaught exceptions.
-    */
-  class OverrideExceptionHandler implements Thread.UncaughtExceptionHandler {
-    @Override
-    public void uncaughtException(final Thread t, final Throwable e) {
-      LOG.fatal(
-        "uncaughtException: OverrideExceptionHandler on thread " +
-         t.getName() + ", msg = " +  e.getMessage() + ", exiting...", e);
-      System.exit(1);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 e13eedd..b2a5c84 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
@@ -911,7 +911,37 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     }
   }
 
+  /**
+   * Creates exception handler that will terminate process gracefully in case
+   * of any uncaught exception.
+   * @return new exception handler object.
+   */
+  public Thread.UncaughtExceptionHandler createUncaughtExceptionHandler() {
+    return new OverrideExceptionHandler();
+  }
+
   public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
     return conf;
   }
+
+
+  /**
+   * Default handler for uncaught exceptions.
+   * It will do the best to clean up and then will terminate current giraph job.
+   */
+  class OverrideExceptionHandler implements Thread.UncaughtExceptionHandler {
+    @Override
+    public void uncaughtException(final Thread t, final Throwable e) {
+      try {
+        LOG.fatal(
+            "uncaughtException: OverrideExceptionHandler on thread " +
+                t.getName() + ", msg = " +  e.getMessage() + ", exiting...", e);
+
+        zooKeeperCleanup();
+        workerFailureCleanup();
+      } finally {
+        System.exit(1);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index 02d4f2b..0275395 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -883,11 +883,13 @@ public class BspServiceMaster<I extends WritableComparable,
 
           masterInfo = new MasterInfo();
           masterServer =
-              new NettyMasterServer(getConfiguration(), this, getContext());
+              new NettyMasterServer(getConfiguration(), this, getContext(),
+                  getGraphTaskManager().createUncaughtExceptionHandler());
           masterInfo.setInetSocketAddress(masterServer.getMyAddress());
           masterInfo.setTaskId(getTaskPartition());
           masterClient =
-              new NettyMasterClient(getContext(), getConfiguration(), this);
+              new NettyMasterClient(getContext(), getConfiguration(), this,
+                  getGraphTaskManager().createUncaughtExceptionHandler());
 
           if (LOG.isInfoEnabled()) {
             LOG.info("becomeMaster: I am now the master!");
@@ -1397,8 +1399,7 @@ public class BspServiceMaster<I extends WritableComparable,
 
       // Did a worker die?
       try {
-        if ((getSuperstep() > 0) &&
-            !superstepChosenWorkerAlive(
+        if (!superstepChosenWorkerAlive(
                 workerInfoHealthyPath,
                 workerInfoList)) {
           return false;

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/utils/ThreadUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ThreadUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ThreadUtils.java
new file mode 100644
index 0000000..a235ff4
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ThreadUtils.java
@@ -0,0 +1,54 @@
+/*
+ * 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.utils;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.util.concurrent.ThreadFactory;
+
+/**
+ * Utility class for thread related functions.
+ */
+public class ThreadUtils {
+
+  /**
+   * Utility class. Do not inherit or create objects.
+   */
+  private ThreadUtils() { }
+
+  /**
+   * Creates new thread factory with specified thread name format.
+   *
+   * @param nameFormat defines naming format for threads created by
+   *                   thread factory
+   * @param exceptionHandler handles uncaught exceptions in all threads
+   *                         produced created thread factory
+   * @return new thread factory with specified thread name format and
+   * exception handler.
+   */
+  public static ThreadFactory createThreadFactory(
+      String nameFormat,
+      Thread.UncaughtExceptionHandler exceptionHandler) {
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder().
+        setNameFormat(nameFormat);
+    if (exceptionHandler != null) {
+      builder.setUncaughtExceptionHandler(exceptionHandler);
+    }
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index dbe6a45..de7af28 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -197,10 +197,12 @@ public class BspServiceWorker<I extends WritableComparable,
     workerGraphPartitioner =
         getGraphPartitionerFactory().createWorkerGraphPartitioner();
     workerInfo = new WorkerInfo();
-    workerServer = new NettyWorkerServer<I, V, E>(conf, this, context);
+    workerServer = new NettyWorkerServer<I, V, E>(conf, this, context,
+        graphTaskManager.createUncaughtExceptionHandler());
     workerInfo.setInetSocketAddress(workerServer.getMyAddress());
     workerInfo.setTaskId(getTaskPartition());
-    workerClient = new NettyWorkerClient<I, V, E>(context, conf, this);
+    workerClient = new NettyWorkerClient<I, V, E>(context, conf, this,
+        graphTaskManager.createUncaughtExceptionHandler());
 
     workerAggregatorRequestProcessor =
         new NettyWorkerAggregatorRequestProcessor(getContext(), conf, this);

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/main/java/org/apache/giraph/yarn/GiraphYarnTask.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/yarn/GiraphYarnTask.java b/giraph-core/src/main/java/org/apache/giraph/yarn/GiraphYarnTask.java
index f4719cc..ccfc972 100644
--- a/giraph-core/src/main/java/org/apache/giraph/yarn/GiraphYarnTask.java
+++ b/giraph-core/src/main/java/org/apache/giraph/yarn/GiraphYarnTask.java
@@ -166,19 +166,6 @@ public class GiraphYarnTask<I extends WritableComparable, V extends Writable,
   }
 
   /**
-    * Default handler for uncaught exceptions.
-    */
-  class OverrideExceptionHandler implements Thread.UncaughtExceptionHandler {
-    @Override
-    public void uncaughtException(final Thread t, final Throwable e) {
-      LOG.fatal(
-        "uncaughtException: OverrideExceptionHandler on thread " +
-         t.getName() + ", msg = " +  e.getMessage() + ", exiting...", e);
-      System.exit(1);
-    }
-  }
-
-  /**
    * Task entry point.
    * @param args CLI arguments injected by GiraphApplicationMaster to hand off
    *             job, task, and attempt ID's to this (and every) Giraph task.

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
index e771e36..5bc9ef0 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
@@ -70,11 +70,12 @@ public class ConnectionTest {
     NettyServer server =
         new NettyServer(conf,
             new WorkerRequestServerHandler.Factory(serverData), workerInfo,
-            context);
+            context, new MockExceptionHandler());
     server.start();
     workerInfo.setInetSocketAddress(server.getMyAddress());
 
-    NettyClient client = new NettyClient(context, conf, new WorkerInfo());
+    NettyClient client = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client.connectAllAddresses(
         Lists.<WorkerInfo>newArrayList(workerInfo));
 
@@ -101,7 +102,8 @@ public class ConnectionTest {
     WorkerInfo workerInfo1 = new WorkerInfo();
     workerInfo1.setTaskId(1);
     NettyServer server1 =
-        new NettyServer(conf, requestServerHandlerFactory, workerInfo1, context);
+        new NettyServer(conf, requestServerHandlerFactory, workerInfo1,
+            context, new MockExceptionHandler());
     server1.start();
     workerInfo1.setInetSocketAddress(server1.getMyAddress());
 
@@ -109,7 +111,7 @@ public class ConnectionTest {
     workerInfo1.setTaskId(2);
     NettyServer server2 =
         new NettyServer(conf, requestServerHandlerFactory, workerInfo2,
-            context);
+            context, new MockExceptionHandler());
     server2.start();
     workerInfo2.setInetSocketAddress(server2.getMyAddress());
 
@@ -117,11 +119,12 @@ public class ConnectionTest {
     workerInfo1.setTaskId(3);
     NettyServer server3 =
         new NettyServer(conf, requestServerHandlerFactory, workerInfo3,
-            context);
+            context, new MockExceptionHandler());
     server3.start();
     workerInfo3.setInetSocketAddress(server3.getMyAddress());
 
-    NettyClient client = new NettyClient(context, conf, new WorkerInfo());
+    NettyClient client = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     List<WorkerInfo> addresses = Lists.<WorkerInfo>newArrayList(workerInfo1,
         workerInfo2, workerInfo3);
     client.connectAllAddresses(addresses);
@@ -148,16 +151,19 @@ public class ConnectionTest {
     WorkerInfo workerInfo = new WorkerInfo();
     NettyServer server = new NettyServer(conf,
         new WorkerRequestServerHandler.Factory(serverData), workerInfo,
-            context);
+            context, new MockExceptionHandler());
     server.start();
     workerInfo.setInetSocketAddress(server.getMyAddress());
 
     List<WorkerInfo> addresses = Lists.<WorkerInfo>newArrayList(workerInfo);
-    NettyClient client1 = new NettyClient(context, conf, new WorkerInfo());
+    NettyClient client1 = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client1.connectAllAddresses(addresses);
-    NettyClient client2 = new NettyClient(context, conf, new WorkerInfo());
+    NettyClient client2 = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client2.connectAllAddresses(addresses);
-    NettyClient client3 = new NettyClient(context, conf, new WorkerInfo());
+    NettyClient client3 = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client3.connectAllAddresses(addresses);
 
     client1.stop();

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/test/java/org/apache/giraph/comm/MockExceptionHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/MockExceptionHandler.java b/giraph-core/src/test/java/org/apache/giraph/comm/MockExceptionHandler.java
new file mode 100644
index 0000000..edd3fc0
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/MockExceptionHandler.java
@@ -0,0 +1,26 @@
+/*
+ * 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.comm;
+
+public class MockExceptionHandler implements Thread.UncaughtExceptionHandler{
+  @Override
+  public void uncaughtException(Thread t, Throwable e) {
+    throw new RuntimeException(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
index 157a543..572e290 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
@@ -160,10 +160,11 @@ public class RequestFailureTest {
     WorkerInfo workerInfo = new WorkerInfo();
     server = new NettyServer(conf,
         new WorkerRequestServerHandler.Factory(serverData), workerInfo,
-            context);
+            context, new MockExceptionHandler());
     server.start();
     workerInfo.setInetSocketAddress(server.getMyAddress());
-    client = new NettyClient(context, conf, new WorkerInfo());
+    client = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client.connectAllAddresses(
         Lists.<WorkerInfo>newArrayList(workerInfo));
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
index 32454f4..8037db9 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
@@ -95,10 +95,11 @@ public class RequestTest {
     workerInfo = new WorkerInfo();
     server = new NettyServer(conf,
         new WorkerRequestServerHandler.Factory(serverData), workerInfo,
-            context);
+            context, new MockExceptionHandler());
     server.start();
     workerInfo.setInetSocketAddress(server.getMyAddress());
-    client = new NettyClient(context, conf, new WorkerInfo());
+    client = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client.connectAllAddresses(
         Lists.<WorkerInfo>newArrayList(workerInfo));
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/61cb37ec/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
index c026cf8..96ce062 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
@@ -85,11 +85,13 @@ public class SaslConnectionTest {
             new WorkerRequestServerHandler.Factory(serverData),
             workerInfo,
             context,
-            mockedSaslServerFactory);
+            mockedSaslServerFactory,
+            new MockExceptionHandler());
     server.start();
     workerInfo.setInetSocketAddress(server.getMyAddress());
 
-    NettyClient client = new NettyClient(context, conf, new WorkerInfo());
+    NettyClient client = new NettyClient(context, conf, new WorkerInfo(),
+        new MockExceptionHandler());
     client.connectAllAddresses(Lists.<WorkerInfo>newArrayList(workerInfo));
 
     client.stop();


[31/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-945: Always use job Configuration to create Configuration (majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/f6845a37
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/f6845a37
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/f6845a37

Branch: refs/heads/release-1.1
Commit: f6845a374e39b91cb8d9292125a52048d71530a3
Parents: 4485e56
Author: Maja Kabiljo <ma...@fb.com>
Authored: Wed Sep 3 11:49:15 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Wed Sep 3 11:49:15 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                                     | 2 ++
 .../src/main/java/org/apache/giraph/job/GiraphJob.java        | 7 +++++--
 .../main/java/org/apache/giraph/utils/ConfigurationUtils.java | 4 +++-
 3 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/f6845a37/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index d5b284e..34db15c 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-945: Always use job Configuration to create Configuration (majakabiljo)
+
   GIRAPH-931: Provide a Strongly Connected Components algorithm (gianluca via majakabiljo)
 
   GIRAPH-933: Checkpointing improvements (edunov via majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/f6845a37/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
index 93aa679..491d3d2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
@@ -64,9 +64,12 @@ public class GiraphJob {
     /**
      * Constructor
      *
+     * @param conf Configuration
      * @throws IOException
      */
-    DelegatedJob() throws IOException { }
+    DelegatedJob(Configuration conf) throws IOException {
+      super(conf);
+    }
 
     @Override
     public Configuration getConfiguration() {
@@ -111,7 +114,7 @@ public class GiraphJob {
                    String jobName) throws IOException {
     this.jobName = jobName;
     this.giraphConfiguration = giraphConfiguration;
-    this.delegatedJob = new DelegatedJob();
+    this.delegatedJob = new DelegatedJob(giraphConfiguration);
   }
 
   public String getJobName() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f6845a37/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index 917f881..5e046cc 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -70,7 +70,8 @@ public final class ConfigurationUtils {
   /** Class logger */
   private static final Logger LOG =
     Logger.getLogger(ConfigurationUtils.class);
-  /** The base path for output dirs as saved in GiraphConfiguration */
+  /*if[PURE_YARN]
+  // The base path for output dirs as saved in GiraphConfiguration
   private static final Path BASE_OUTPUT_PATH;
   static {
     // whether local or remote, if there's no *-site.xml's to find, we're done
@@ -80,6 +81,7 @@ public final class ConfigurationUtils {
       throw new IllegalStateException("Error locating default base path!", ioe);
     }
   }
+  end[PURE_YARN]*/
   /** Maintains our accepted options in case the caller wants to add some */
   private static Options OPTIONS;
 


[30/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-944: Improve job tracking on command line


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/4485e563
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/4485e563
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/4485e563

Branch: refs/heads/release-1.1
Commit: 4485e563a6582afb1c848ea80888fdad50ada516
Parents: de0efb0
Author: Maja Kabiljo <ma...@fb.com>
Authored: Tue Aug 26 11:35:14 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Tue Aug 26 14:42:52 2014 -0700

----------------------------------------------------------------------
 giraph-core/pom.xml                             |  24 +-
 .../java/org/apache/giraph/bsp/BspService.java  |  12 +-
 .../apache/giraph/bsp/CentralizedService.java   |   9 +
 .../org/apache/giraph/conf/GiraphConstants.java |   2 +-
 .../apache/giraph/graph/GraphTaskManager.java   |  30 +++
 .../giraph/graph/JobProgressTrackerClient.java  |  33 +++
 .../graph/JobProgressTrackerClientNoOp.java     |  47 ++++
 .../RetryableJobProgressTrackerClient.java      | 175 ++++++++++++++
 .../java/org/apache/giraph/job/GiraphJob.java   |  16 +-
 .../apache/giraph/job/JobProgressTracker.java   | 155 +++----------
 .../giraph/job/JobProgressTrackerService.java   | 193 +++++++++++++++
 .../apache/giraph/master/BspServiceMaster.java  |  10 +-
 .../org/apache/giraph/master/MasterCompute.java |  30 ++-
 .../apache/giraph/scripting/ScriptLoader.java   |   6 +-
 .../java/org/apache/giraph/utils/FileUtils.java |   2 +-
 .../apache/giraph/worker/BspServiceWorker.java  |   3 +-
 .../org/apache/giraph/worker/WorkerContext.java |  10 +
 .../apache/giraph/worker/WorkerProgress.java    | 232 ++++++++++---------
 .../giraph/worker/WorkerProgressWriter.java     |  30 +--
 .../org/apache/giraph/zk/ZooKeeperManager.java  |   4 +-
 .../test/java/org/apache/giraph/BspCase.java    |   4 +-
 giraph-examples/pom.xml                         |   8 -
 .../java/org/apache/giraph/TestBspBasic.java    |   4 +-
 .../giraph/hive/jython/HiveJythonUtils.java     |   2 +-
 pom.xml                                         |  81 ++++++-
 25 files changed, 825 insertions(+), 297 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-core/pom.xml b/giraph-core/pom.xml
index b66ba1d..23f6666 100644
--- a/giraph-core/pom.xml
+++ b/giraph-core/pom.xml
@@ -289,14 +289,6 @@ under the License.
           </plugin>
         </plugins>
       </build>
-      <dependencies>
-        <dependency>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-          <version>${dep.oldnetty.version}</version>
-          <scope>test</scope>
-        </dependency>
-      </dependencies>
     </profile>
 
     <profile>
@@ -467,6 +459,22 @@ under the License.
   <dependencies>
     <!-- compile dependencies. sorted lexicographically. -->
     <dependency>
+      <groupId>com.facebook.nifty</groupId>
+      <artifactId>nifty-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-service</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.facebook.thirdparty.yourkit-api</groupId>
       <artifactId>yjp-controller-api-redist</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
----------------------------------------------------------------------
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 c418a89..2a50489 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
@@ -23,6 +23,7 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.graph.InputSplitEvents;
 import org.apache.giraph.graph.InputSplitPaths;
+import org.apache.giraph.job.JobProgressTracker;
 import org.apache.giraph.partition.GraphPartitionerFactory;
 import org.apache.giraph.utils.CheckpointingUtils;
 import org.apache.giraph.worker.WorkerInfo;
@@ -161,8 +162,6 @@ public abstract class BspService<I extends WritableComparable,
       "/_partitionExchangeDir";
   /** Denotes that the superstep is done */
   public static final String SUPERSTEP_FINISHED_NODE = "/_superstepFinished";
-  /** Stores progress info for workers */
-  public static final String WORKER_PROGRESSES = "/_workerProgresses";
   /** Denotes that computation should be halted */
   public static final String HALT_COMPUTATION_NODE = "/_haltComputation";
   /** User sets this flag to checkpoint and stop the job */
@@ -241,8 +240,6 @@ public abstract class BspService<I extends WritableComparable,
   protected final String savedCheckpointBasePath;
   /** Path to the master election path */
   protected final String masterElectionPath;
-  /** Stores progress info of this worker */
-  protected final String myProgressPath;
   /** If this path exists computation will be halted */
   protected final String haltComputationPath;
   /** Private ZooKeeper instance that implements the service */
@@ -363,7 +360,6 @@ public abstract class BspService<I extends WritableComparable,
         getCheckpointBasePath(getConfiguration(), getJobId());
 
     masterElectionPath = basePath + MASTER_ELECTION_DIR;
-    myProgressPath = basePath + WORKER_PROGRESSES + "/" + taskPartition;
     String serverPortList = conf.getZookeeperList();
     haltComputationPath = basePath + HALT_COMPUTATION_NODE;
     getContext().getCounter(GiraphConstants.ZOOKEEPER_HALT_NODE_COUNTER_GROUP,
@@ -404,7 +400,6 @@ public abstract class BspService<I extends WritableComparable,
           "BspService: Invalid superstep to restart - " +
               restartedSuperstep);
     }
-
   }
 
   /**
@@ -1258,6 +1253,11 @@ public abstract class BspService<I extends WritableComparable,
     return lastCheckpointedSuperstep;
   }
 
+  @Override
+  public JobProgressTracker getJobProgressTracker() {
+    return getGraphTaskManager().getJobProgressTracker();
+  }
+
   /**
    * Only get the finalized checkpoint files
    */

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
index 560f1fb..0cadfb7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
@@ -19,6 +19,8 @@
 package org.apache.giraph.bsp;
 
 import java.util.List;
+
+import org.apache.giraph.job.JobProgressTracker;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -56,4 +58,11 @@ public interface CentralizedService<I extends WritableComparable,
    * @return List of workers
    */
   List<WorkerInfo> getWorkerInfoList();
+
+  /**
+   * Get JobProgressTracker to report progress to
+   *
+   * @return JobProgressTrackerClient
+   */
+  JobProgressTracker getJobProgressTracker();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 da0a8db..d1fdf57 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
@@ -1119,7 +1119,7 @@ public interface GiraphConstants {
 
   /** Whether to track job progress on client or not */
   BooleanConfOption TRACK_JOB_PROGRESS_ON_CLIENT =
-      new BooleanConfOption("giraph.trackJobProgressOnClient", true,
+      new BooleanConfOption("giraph.trackJobProgressOnClient", false,
           "Whether to track job progress on client or not");
 
   /** Number of retries for creating the HDFS files */

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 8a97939..ba5d2fa 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
@@ -25,6 +25,7 @@ import org.apache.giraph.bsp.CheckpointStatus;
 import org.apache.giraph.comm.messages.MessageStore;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.job.JobProgressTracker;
 import org.apache.giraph.scripting.ScriptLoader;
 import org.apache.giraph.master.BspServiceMaster;
 import org.apache.giraph.master.MasterAggregatorUsage;
@@ -69,6 +70,7 @@ import java.util.List;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -122,6 +124,8 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   /** Superstep stats */
   private FinishedSuperstepStats finishedSuperstepStats =
       new FinishedSuperstepStats(0, false, 0, 0, false, CheckpointStatus.NONE);
+  /** Job progress tracker */
+  private JobProgressTrackerClient jobProgressTracker;
 
   // Per-Job Metrics
   /** Timer for WorkerContext#preApplication() */
@@ -194,6 +198,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     context.setStatus("setup: Beginning worker setup.");
     Configuration hadoopConf = context.getConfiguration();
     conf = new ImmutableClassesGiraphConfiguration<I, V, E>(hadoopConf);
+    initializeJobProgressTracker();
     // Write user's graph types (I,V,E,M) back to configuration parameters so
     // that they are set for quicker access later. These types are often
     // inferred from the Computation class used.
@@ -245,6 +250,26 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   }
 
   /**
+   * Create and connect a client to JobProgressTrackerService,
+   * or no-op implementation if progress shouldn't be tracked or something
+   * goes wrong
+   */
+  private void initializeJobProgressTracker() {
+    if (!conf.trackJobProgressOnClient()) {
+      jobProgressTracker = new JobProgressTrackerClientNoOp();
+    } else {
+      try {
+        jobProgressTracker = new RetryableJobProgressTrackerClient(conf);
+      } catch (InterruptedException | ExecutionException e) {
+        LOG.warn("createJobProgressClient: Exception occurred while trying to" +
+            " connect to JobProgressTracker - not reporting progress", e);
+        jobProgressTracker = new JobProgressTrackerClientNoOp();
+      }
+    }
+    jobProgressTracker.mapperStarted();
+  }
+
+  /**
   * Perform the work assigned to this compute node for this job run.
   * 1) Run checkpoint per frequency policy.
   * 2) For every vertex on this mapper, run the compute() function
@@ -485,6 +510,10 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     return serviceWorker.getWorkerContext();
   }
 
+  public JobProgressTracker getJobProgressTracker() {
+    return jobProgressTracker;
+  }
+
   /**
    * Copied from JobConf to get the location of this jar.  Workaround for
    * things like Oozie map-reduce jobs. NOTE: Pure YARN profile cannot
@@ -878,6 +907,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     if (LOG.isInfoEnabled()) {
       LOG.info("cleanup: Starting for " + getGraphFunctions());
     }
+    jobProgressTracker.cleanup();
     if (done) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClient.java b/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClient.java
new file mode 100644
index 0000000..c302d9a
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClient.java
@@ -0,0 +1,33 @@
+/*
+ * 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.giraph.job.JobProgressTracker;
+
+import java.io.IOException;
+
+/**
+ * Wrapper around JobProgressTracker which retries to connect and swallows
+ * exceptions so app wouldn't crash if something goes wrong with progress
+ * reports.
+ */
+public interface JobProgressTrackerClient extends JobProgressTracker {
+  /** Close the connections if any */
+  void cleanup() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClientNoOp.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClientNoOp.java b/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClientNoOp.java
new file mode 100644
index 0000000..d75fd42
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/JobProgressTrackerClientNoOp.java
@@ -0,0 +1,47 @@
+/*
+ * 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.giraph.worker.WorkerProgress;
+
+/**
+ * Class to use for JobProgressTracker client when progress shouldn't be
+ * tracked or something goes wrong
+ */
+public class JobProgressTrackerClientNoOp implements JobProgressTrackerClient {
+  @Override
+  public void cleanup() {
+  }
+
+  @Override
+  public void mapperStarted() {
+  }
+
+  @Override
+  public void logInfo(String logLine) {
+  }
+
+  @Override
+  public void logFailure(String reason) {
+  }
+
+  @Override
+  public void updateProgress(WorkerProgress workerProgress) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/graph/RetryableJobProgressTrackerClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/RetryableJobProgressTrackerClient.java b/giraph-core/src/main/java/org/apache/giraph/graph/RetryableJobProgressTrackerClient.java
new file mode 100644
index 0000000..f15a2e7
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/RetryableJobProgressTrackerClient.java
@@ -0,0 +1,175 @@
+/*
+ * 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.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.job.JobProgressTracker;
+import org.apache.giraph.worker.WorkerProgress;
+import org.apache.log4j.Logger;
+
+import com.facebook.nifty.client.FramedClientConnector;
+import com.facebook.swift.service.RuntimeTTransportException;
+import com.facebook.swift.service.ThriftClientManager;
+import com.google.common.io.Closeables;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Wrapper around JobProgressTracker which retires to connect and swallows
+ * exceptions so app wouldn't crash if something goes wrong with progress
+ * reports.
+ */
+public class RetryableJobProgressTrackerClient
+    implements JobProgressTrackerClient {
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(RetryableJobProgressTrackerClient.class);
+  /** Configuration */
+  private final GiraphConfiguration conf;
+  /** Thrift client manager to use to connect to job progress tracker */
+  private ThriftClientManager clientManager;
+  /** Job progress tracker */
+  private JobProgressTracker jobProgressTracker;
+
+  /**
+   * Constructor
+   *
+   * @param conf Giraph configuration
+   */
+  public RetryableJobProgressTrackerClient(GiraphConfiguration conf) throws
+      ExecutionException, InterruptedException {
+    this.conf = conf;
+    resetConnection();
+  }
+
+  /**
+   * Try to establish new connection to JobProgressTracker
+   */
+  private void resetConnection() throws ExecutionException,
+      InterruptedException {
+    clientManager = new ThriftClientManager();
+    FramedClientConnector connector =
+        new FramedClientConnector(new InetSocketAddress(
+            JOB_PROGRESS_SERVICE_HOST.get(conf),
+            JOB_PROGRESS_SERVICE_PORT.get(conf)));
+    jobProgressTracker =
+        clientManager.createClient(connector, JobProgressTracker.class).get();
+
+  }
+
+  @Override
+  public synchronized void cleanup() throws IOException {
+    Closeables.close(clientManager, true);
+    try {
+      clientManager.close();
+      // CHECKSTYLE: stop IllegalCatch
+    } catch (Exception e) {
+      // CHECKSTYLE: resume IllegalCatch
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Exception occurred while trying to close JobProgressTracker", e);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void mapperStarted() {
+    executeWithRetry(new Runnable() {
+      @Override
+      public void run() {
+        jobProgressTracker.mapperStarted();
+      }
+    });
+  }
+
+  @Override
+  public synchronized void logInfo(final String logLine) {
+    executeWithRetry(new Runnable() {
+      @Override
+      public void run() {
+        jobProgressTracker.logInfo(logLine);
+      }
+    });
+  }
+
+  @Override
+  public synchronized void logFailure(final String reason) {
+    executeWithRetry(new Runnable() {
+      @Override
+      public void run() {
+        jobProgressTracker.logFailure(reason);
+      }
+    });
+  }
+
+  @Override
+  public synchronized void updateProgress(final WorkerProgress workerProgress) {
+    executeWithRetry(new Runnable() {
+      @Override
+      public void run() {
+        jobProgressTracker.updateProgress(workerProgress);
+      }
+    });
+  }
+
+  /**
+   * Execute Runnable, if disconnected try to connect again and retry
+   *
+   * @param runnable Runnable to execute
+   */
+  private void executeWithRetry(Runnable runnable) {
+    try {
+      runnable.run();
+    } catch (RuntimeTTransportException te) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RuntimeTTransportException occurred while talking to " +
+            "JobProgressTracker server, trying to reconnect", te);
+      }
+      try {
+        try {
+          clientManager.close();
+          // CHECKSTYLE: stop IllegalCatch
+        } catch (Exception e) {
+          // CHECKSTYLE: resume IllegalCatch
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("");
+          }
+        }
+        resetConnection();
+        runnable.run();
+        // CHECKSTYLE: stop IllegalCatch
+      } catch (Exception e) {
+        // CHECKSTYLE: resume IllegalCatch
+        if (LOG.isInfoEnabled()) {
+          LOG.info("Exception occurred while talking to " +
+              "JobProgressTracker server, giving up", e);
+        }
+      }
+      // CHECKSTYLE: stop IllegalCatch
+    } catch (Exception e) {
+      // CHECKSTYLE: resume IllegalCatch
+      if (LOG.isInfoEnabled()) {
+        LOG.info("Exception occurred while talking to " +
+            "JobProgressTracker server, giving up", e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
index 436126b..93aa679 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
@@ -239,6 +239,9 @@ public class GiraphJob {
     int tryCount = 0;
     GiraphJobRetryChecker retryChecker = conf.getJobRetryChecker();
     while (true) {
+      JobProgressTrackerService jobProgressTrackerService =
+          JobProgressTrackerService.createJobProgressServer(conf);
+
       tryCount++;
       Job submittedJob = new Job(conf, jobName);
       if (submittedJob.getJar() == null) {
@@ -253,16 +256,17 @@ public class GiraphJob {
       jobObserver.launchingJob(submittedJob);
       submittedJob.submit();
       if (LOG.isInfoEnabled()) {
-        LOG.info("run: Tracking URL: " + submittedJob.getTrackingURL());
+        LOG.info("Tracking URL: " + submittedJob.getTrackingURL());
+        LOG.info(
+            "Waiting for resources... Job will start only when it gets all " +
+                (conf.getMinWorkers() + 1) + " mappers");
       }
-      HaltApplicationUtils.printHaltInfo(submittedJob, conf);
-      JobProgressTracker jobProgressTracker = conf.trackJobProgressOnClient() ?
-          new JobProgressTracker(submittedJob, conf) : null;
       jobObserver.jobRunning(submittedJob);
+      HaltApplicationUtils.printHaltInfo(submittedJob, conf);
 
       boolean passed = submittedJob.waitForCompletion(verbose);
-      if (jobProgressTracker != null) {
-        jobProgressTracker.stop();
+      if (jobProgressTrackerService != null) {
+        jobProgressTrackerService.stop(passed);
       }
       jobObserver.jobFinished(submittedJob, passed);
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTracker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTracker.java b/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTracker.java
index 6971174..95bc56d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTracker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTracker.java
@@ -18,138 +18,53 @@
 
 package org.apache.giraph.job;
 
-import org.apache.giraph.bsp.BspService;
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.utils.CounterUtils;
-import org.apache.giraph.utils.WritableUtils;
+import org.apache.giraph.conf.IntConfOption;
+import org.apache.giraph.conf.StrConfOption;
 import org.apache.giraph.worker.WorkerProgress;
-import org.apache.giraph.zk.ZooKeeperExt;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+import com.facebook.swift.service.ThriftMethod;
+import com.facebook.swift.service.ThriftService;
 
 /**
- * Class which tracks job's progress on client
+ * Interface for job progress tracker on job client
  */
-public class JobProgressTracker implements Watcher {
-  /** Class logger */
-  private static final Logger LOG = Logger.getLogger(JobProgressTracker.class);
-  /** How often to print job's progress */
-  private static final int UPDATE_MILLISECONDS = 5 * 1000;
-  /** Thread which periodically writes job's progress */
-  private Thread writerThread;
-  /** ZooKeeperExt */
-  private ZooKeeperExt zk;
-  /** Whether application is finished */
-  private volatile boolean finished = false;
+@ThriftService
+public interface JobProgressTracker {
+  /** Host on which job progress service runs */
+  StrConfOption JOB_PROGRESS_SERVICE_HOST =
+      new StrConfOption("giraph.jobProgressServiceHost", null,
+          "Host on which job progress service runs");
+  /** Port which job progress service uses */
+  IntConfOption JOB_PROGRESS_SERVICE_PORT =
+      new IntConfOption("giraph.jobProgressServicePort", -1,
+          "Port which job progress service uses");
+
+  /** Notify JobProgressTracker that mapper started */
+  @ThriftMethod
+  void mapperStarted();
 
   /**
-   * Constructor
+   * Call this when you want to log an info line from any mapper to command line
    *
-   * @param submittedJob Job to track
-   * @param conf Configuration
+   * @param logLine Line to log
    */
-  public JobProgressTracker(final Job submittedJob,
-      final GiraphConfiguration conf) throws IOException, InterruptedException {
-    String zkServer = CounterUtils.waitAndGetCounterNameFromGroup(
-        submittedJob, GiraphConstants.ZOOKEEPER_SERVER_PORT_COUNTER_GROUP);
-    final String basePath = CounterUtils.waitAndGetCounterNameFromGroup(
-        submittedJob, GiraphConstants.ZOOKEEPER_BASE_PATH_COUNTER_GROUP);
-    // Connect to ZooKeeper
-    if (zkServer != null && basePath != null) {
-      zk = new ZooKeeperExt(
-        zkServer,
-        conf.getZooKeeperSessionTimeout(),
-        conf.getZookeeperOpsMaxAttempts(),
-        conf.getZookeeperOpsRetryWaitMsecs(),
-        this,
-        new Progressable() {
-          @Override
-          public void progress() {
-          }
-        });
-      writerThread = new Thread(new Runnable() {
-        @Override
-        public void run() {
-          String workerProgressBasePath = basePath +
-            BspService.WORKER_PROGRESSES;
-          try {
-            while (!finished) {
-              if (zk.exists(workerProgressBasePath, false) != null) {
-                // Get locations of all worker progresses
-                List<String> workerProgressPaths = zk.getChildrenExt(
-                  workerProgressBasePath, false, false, true);
-                List<WorkerProgress> workerProgresses =
-                  new ArrayList<WorkerProgress>(workerProgressPaths.size());
-                // Read all worker progresses
-                for (String workerProgressPath : workerProgressPaths) {
-                  WorkerProgress workerProgress = new WorkerProgress();
-                  byte[] zkData = zk.getData(workerProgressPath, false, null);
-                  WritableUtils.readFieldsFromByteArray(zkData, workerProgress);
-                  workerProgresses.add(workerProgress);
-                }
-                // Combine and log
-                CombinedWorkerProgress combinedWorkerProgress =
-                  new CombinedWorkerProgress(workerProgresses);
-                if (LOG.isInfoEnabled()) {
-                  LOG.info(combinedWorkerProgress.toString());
-                }
-                // Check if application is done
-                if (combinedWorkerProgress.isDone(conf.getMaxWorkers())) {
-                  break;
-                }
-              }
-              Thread.sleep(UPDATE_MILLISECONDS);
-            }
-            // CHECKSTYLE: stop IllegalCatchCheck
-          } catch (Exception e) {
-            // CHECKSTYLE: resume IllegalCatchCheck
-            if (LOG.isInfoEnabled()) {
-              LOG.info("run: Exception occurred", e);
-            }
-          } finally {
-            try {
-              // Create a node so master knows we stopped communicating with
-              // ZooKeeper and it's safe to cleanup
-              zk.createExt(
-                basePath + BspService.CLEANED_UP_DIR + "/client",
-                null,
-                ZooDefs.Ids.OPEN_ACL_UNSAFE,
-                CreateMode.PERSISTENT,
-                true);
-              zk.close();
-              // CHECKSTYLE: stop IllegalCatchCheck
-            } catch (Exception e) {
-              // CHECKSTYLE: resume IllegalCatchCheck
-              if (LOG.isInfoEnabled()) {
-                LOG.info("run: Exception occurred", e);
-              }
-            }
-          }
-        }
-      });
-      writerThread.start();
-    }
-  }
+  @ThriftMethod
+  void logInfo(String logLine);
 
   /**
-   * Stop the thread which logs application progress
+   * Notify that job is failing
+   *
+   * @param reason Reason for failure
    */
-  public void stop() {
-    finished = true;
-  }
+  @ThriftMethod
+  void logFailure(String reason);
 
-  @Override
-  public void process(WatchedEvent event) {
-  }
+  /**
+   * Workers should call this method to update their progress
+   *
+   * @param workerProgress Progress of the worker
+   */
+  @ThriftMethod
+  void updateProgress(WorkerProgress workerProgress);
 }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java b/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
new file mode 100644
index 0000000..3a896e2
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
@@ -0,0 +1,193 @@
+/*
+ * 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.job;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.worker.WorkerProgress;
+import org.apache.log4j.Logger;
+
+import com.facebook.swift.codec.ThriftCodecManager;
+import com.facebook.swift.service.ThriftEventHandler;
+import com.facebook.swift.service.ThriftServer;
+import com.facebook.swift.service.ThriftServerConfig;
+import com.facebook.swift.service.ThriftServiceProcessor;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Implementation of job progress tracker service on job client
+ */
+public class JobProgressTrackerService implements JobProgressTracker {
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(JobProgressTrackerService.class);
+  /** How often to print job's progress */
+  private static final int UPDATE_MILLISECONDS = 10 * 1000;
+
+  /** Configuration */
+  private final GiraphConfiguration conf;
+  /** Thread which periodically writes job's progress */
+  private Thread writerThread;
+  /** Whether application is finished */
+  private volatile boolean finished = false;
+  /** Server which uses this service */
+  private ThriftServer server;
+  /** Number of mappers which the job got */
+  private int mappersStarted;
+  /** Last time number of mappers started was logged */
+  private long lastTimeMappersStartedLogged;
+  /** Map of worker progresses */
+  private final Map<Integer, WorkerProgress> workerProgresses =
+      new ConcurrentHashMap<>();
+
+  /**
+   * Constructor
+   *
+   * @param conf Configuration
+   */
+  public JobProgressTrackerService(GiraphConfiguration conf) {
+    this.conf = conf;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Waiting for job to start... (this may take a minute)");
+    }
+    startWriterThread();
+  }
+
+  /**
+   * Start the thread which writes progress periodically
+   */
+  private void startWriterThread() {
+    writerThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        while (!finished) {
+          if (mappersStarted == conf.getMaxWorkers() + 1 &&
+              !workerProgresses.isEmpty()) {
+            // Combine and log
+            CombinedWorkerProgress combinedWorkerProgress =
+                new CombinedWorkerProgress(workerProgresses.values());
+            if (LOG.isInfoEnabled()) {
+              LOG.info(combinedWorkerProgress.toString());
+            }
+            // Check if application is done
+            if (combinedWorkerProgress.isDone(conf.getMaxWorkers())) {
+              break;
+            }
+          }
+          try {
+            Thread.sleep(UPDATE_MILLISECONDS);
+          } catch (InterruptedException e) {
+            if (LOG.isInfoEnabled()) {
+              LOG.info("Progress thread interrupted");
+            }
+            break;
+          }
+        }
+      }
+    });
+    writerThread.start();
+  }
+
+  @Override
+  public synchronized void mapperStarted() {
+    mappersStarted++;
+    if (LOG.isInfoEnabled()) {
+      if (mappersStarted == conf.getMaxWorkers() + 1) {
+        LOG.info("Got all " + mappersStarted + " mappers");
+      } else {
+        if (System.currentTimeMillis() - lastTimeMappersStartedLogged >
+            UPDATE_MILLISECONDS) {
+          lastTimeMappersStartedLogged = System.currentTimeMillis();
+          LOG.info("Got " + mappersStarted + " but needs " +
+              (conf.getMaxWorkers() + 1) + " mappers");
+        }
+      }
+    }
+  }
+
+  @Override
+  public void logInfo(String logLine) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(logLine);
+    }
+  }
+
+  @Override
+  public void logFailure(String reason) {
+    LOG.fatal(reason);
+    finished = true;
+    writerThread.interrupt();
+  }
+
+  @Override
+  public void updateProgress(WorkerProgress workerProgress) {
+    workerProgresses.put(workerProgress.getTaskId(), workerProgress);
+  }
+
+  /**
+   * Stop the thread which logs application progress and server
+   *
+   * @param succeeded Whether job succeeded or not
+   */
+  public void stop(boolean succeeded) {
+    finished = true;
+    writerThread.interrupt();
+    server.close();
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Job " + (succeeded ? "finished successfully" : "failed") +
+          ", cleaning up...");
+    }
+  }
+
+  /**
+   * Create job progress server on job client, and update configuration with
+   * its hostname and port so mappers would know what to connect to. Returns
+   * null if progress shouldn't be tracked
+   *
+   * @param conf Configuration
+   * @return JobProgressTrackerService
+   */
+  public static JobProgressTrackerService createJobProgressServer(
+      GiraphConfiguration conf) {
+    if (!conf.trackJobProgressOnClient()) {
+      return null;
+    }
+    try {
+      JobProgressTrackerService service = new JobProgressTrackerService(conf);
+      ThriftServiceProcessor processor =
+          new ThriftServiceProcessor(new ThriftCodecManager(),
+              new ArrayList<ThriftEventHandler>(), service);
+      service.server = new ThriftServer(processor, new ThriftServerConfig());
+      service.server.start();
+      JOB_PROGRESS_SERVICE_HOST.set(conf,
+          InetAddress.getLocalHost().getHostName());
+      JOB_PROGRESS_SERVICE_PORT.set(conf, service.server.getPort());
+      return service;
+      // CHECKSTYLE: stop IllegalCatch
+    } catch (Exception e) {
+      // CHECKSTYLE: resume IllegalCatch
+      LOG.warn("Exception occurred while trying to create " +
+          "JobProgressTrackerService - not using progress reporting", e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index 671df23..efa5b87 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -307,6 +307,7 @@ public class BspServiceMaster<I extends WritableComparable,
    * @param reason The reason the job failed
    */
   private void setJobStateFailed(String reason) {
+    getGraphTaskManager().getJobProgressTracker().logFailure(reason);
     setJobState(ApplicationState.FAILED, -1, -1, false);
     failJob(new IllegalStateException(reason));
   }
@@ -644,7 +645,8 @@ public class BspServiceMaster<I extends WritableComparable,
           "check input of " + inputFormat.getClass().getName() + "!");
       getContext().setStatus("Failing job due to 0 input splits, " +
           "check input of " + inputFormat.getClass().getName() + "!");
-      setJobStateFailed("0 input splits");
+      setJobStateFailed("Please check your input tables - partitions which " +
+          "you specified are missing. Failing the job!!!");
     }
     if (minSplitCountHint > splitList.size()) {
       LOG.warn(logPrefix + ": Number of inputSplits=" +
@@ -885,7 +887,7 @@ public class BspServiceMaster<I extends WritableComparable,
               getContext());
           aggregatorHandler.initialize(this);
           masterCompute = getConfiguration().createMasterCompute();
-          masterCompute.setMasterAggregatorUsage(aggregatorHandler);
+          masterCompute.setMasterService(this);
 
           masterInfo = new MasterInfo();
           masterServer =
@@ -1790,10 +1792,6 @@ public class BspServiceMaster<I extends WritableComparable,
         GraphFunctions.ALL_EXCEPT_ZOOKEEPER)) {
       maxTasks *= 2;
     }
-    if (getConfiguration().trackJobProgressOnClient()) {
-      // For job client
-      maxTasks++;
-    }
     List<String> cleanedUpChildrenList = null;
     while (true) {
       try {

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
index c2a1f9a..552cca9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
@@ -19,6 +19,7 @@
 package org.apache.giraph.master;
 
 import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.Computation;
@@ -46,7 +47,7 @@ public abstract class MasterCompute
   /** If true, do not do anymore computation on this vertex. */
   private boolean halt = false;
   /** Master aggregator usage */
-  private MasterAggregatorUsage masterAggregatorUsage;
+  private CentralizedServiceMaster serviceMaster;
   /** Graph state */
   private GraphState graphState;
   /**
@@ -192,14 +193,16 @@ public abstract class MasterCompute
   public final <A extends Writable> boolean registerAggregator(
     String name, Class<? extends Aggregator<A>> aggregatorClass)
     throws InstantiationException, IllegalAccessException {
-    return masterAggregatorUsage.registerAggregator(name, aggregatorClass);
+    return serviceMaster.getAggregatorHandler().registerAggregator(
+        name, aggregatorClass);
   }
 
   @Override
   public final <A extends Writable> boolean registerAggregator(
     String name, WritableFactory<? extends Aggregator<A>> aggregator)
     throws InstantiationException, IllegalAccessException {
-    return masterAggregatorUsage.registerAggregator(name, aggregator);
+    return serviceMaster.getAggregatorHandler().registerAggregator(
+        name, aggregator);
   }
 
   @Override
@@ -207,28 +210,37 @@ public abstract class MasterCompute
       String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
-    return masterAggregatorUsage.registerPersistentAggregator(
+    return serviceMaster.getAggregatorHandler().registerPersistentAggregator(
         name, aggregatorClass);
   }
 
   @Override
   public final <A extends Writable> A getAggregatedValue(String name) {
-    return masterAggregatorUsage.<A>getAggregatedValue(name);
+    return serviceMaster.getAggregatorHandler().<A>getAggregatedValue(name);
   }
 
   @Override
   public final <A extends Writable> void setAggregatedValue(
       String name, A value) {
-    masterAggregatorUsage.setAggregatedValue(name, value);
+    serviceMaster.getAggregatorHandler().setAggregatedValue(name, value);
+  }
+
+  /**
+   * Call this to log a line to command line of the job. Use in moderation -
+   * it's a synchronous call to Job client
+   *
+   * @param line Line to print
+   */
+  public void logToCommandLine(String line) {
+    serviceMaster.getJobProgressTracker().logInfo(line);
   }
 
   final void setGraphState(GraphState graphState) {
     this.graphState = graphState;
   }
 
-  final void setMasterAggregatorUsage(MasterAggregatorUsage
-      masterAggregatorUsage) {
-    this.masterAggregatorUsage = masterAggregatorUsage;
+  final void setMasterService(CentralizedServiceMaster serviceMaster) {
+    this.serviceMaster = serviceMaster;
   }
 
   final void setSuperstepClasses(SuperstepClasses superstepClasses) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/scripting/ScriptLoader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/scripting/ScriptLoader.java b/giraph-core/src/main/java/org/apache/giraph/scripting/ScriptLoader.java
index 2b30739..f78b1a0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/scripting/ScriptLoader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/scripting/ScriptLoader.java
@@ -150,7 +150,7 @@ public class ScriptLoader {
    *
    * @param conf Configuration
    */
-  public static void loadScripts(Configuration conf) {
+  public static void loadScripts(Configuration conf) throws IOException {
     List<DeployedScript> deployedScripts = getScriptsToLoad(conf);
     if (deployedScripts == null) {
       return;
@@ -167,7 +167,7 @@ public class ScriptLoader {
    * @param deployedScript the deployed script
    */
   public static void loadScript(Configuration conf,
-      DeployedScript deployedScript) {
+      DeployedScript deployedScript) throws IOException {
     InputStream stream = openScriptInputStream(conf, deployedScript);
     switch (deployedScript.getLanguage()) {
     case JYTHON:
@@ -180,7 +180,7 @@ public class ScriptLoader {
     }
 
     LOADED_SCRIPTS.add(deployedScript);
-    Closeables.closeQuietly(stream);
+    Closeables.close(stream, true);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/utils/FileUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/FileUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/FileUtils.java
index 6e8b1e3..0f9a08a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/FileUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/FileUtils.java
@@ -129,7 +129,7 @@ public class FileUtils {
         writer.write('\n');
       }
     } finally {
-      Closeables.closeQuietly(writer);
+      Closeables.close(writer, true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index 447bb6f..120678f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -229,7 +229,8 @@ public class BspServiceWorker<I extends WritableComparable,
 
     WorkerProgress.get().setTaskId(getTaskPartition());
     workerProgressWriter = conf.trackJobProgressOnClient() ?
-        new WorkerProgressWriter(myProgressPath, getZkExt()) : null;
+        new WorkerProgressWriter(graphTaskManager.getJobProgressTracker()) :
+        null;
 
     GiraphMetrics.get().addSuperstepResetObserver(this);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
index aca9944..7a55d56 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
@@ -206,6 +206,16 @@ public abstract class WorkerContext
     return workerAggregatorUsage.<A>getAggregatedValue(name);
   }
 
+  /**
+   * Call this to log a line to command line of the job. Use in moderation -
+   * it's a synchronous call to Job client
+   *
+   * @param line Line to print
+   */
+  public void logToCommandLine(String line) {
+    serviceWorker.getJobProgressTracker().logInfo(line);
+  }
+
   @Override
   public void write(DataOutput dataOutput) throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgress.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgress.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgress.java
index 1a2a6ee..24f791b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgress.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgress.java
@@ -19,17 +19,9 @@
 package org.apache.giraph.worker;
 
 import org.apache.giraph.utils.MemoryUtils;
-import org.apache.giraph.utils.WritableUtils;
-import org.apache.giraph.zk.ZooKeeperExt;
-import org.apache.hadoop.io.Writable;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
+
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
 
 import javax.annotation.concurrent.ThreadSafe;
 
@@ -38,9 +30,8 @@ import javax.annotation.concurrent.ThreadSafe;
  * ZooKeeper with {@link WorkerProgressWriter}.
  */
 @ThreadSafe
-public class WorkerProgress implements Writable {
-  /** Class logger */
-  private static final Logger LOG = Logger.getLogger(WorkerProgress.class);
+@ThriftStruct
+public class WorkerProgress {
   /** Singleton instance for everyone to use */
   private static final WorkerProgress INSTANCE = new WorkerProgress();
 
@@ -99,45 +90,6 @@ public class WorkerProgress implements Writable {
   }
 
   /**
-   * Write worker's progress to znode
-   *
-   * @param zk ZooKeeperExt
-   * @param myProgressPath Path to write the progress to
-   */
-  public static void writeToZnode(ZooKeeperExt zk, String myProgressPath) {
-    byte[] byteArray = WritableUtils.writeToByteArray(get());
-    try {
-      zk.createOrSetExt(myProgressPath,
-          byteArray,
-          ZooDefs.Ids.OPEN_ACL_UNSAFE,
-          CreateMode.PERSISTENT,
-          true,
-          -1);
-    } catch (KeeperException | InterruptedException e) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("writeToZnode: " + e.getClass().getName() +
-            " exception occurred", e);
-      }
-    }
-  }
-
-  public synchronized boolean isLoadingVerticesDone() {
-    return loadingVerticesDone;
-  }
-
-  public synchronized boolean isLoadingEdgesDone() {
-    return loadingEdgesDone;
-  }
-
-  public synchronized boolean isComputationDone() {
-    return computationDone;
-  }
-
-  public synchronized boolean isStoringDone() {
-    return storingDone;
-  }
-
-  /**
    * Add number of vertices loaded
    *
    * @param verticesLoaded How many vertices were loaded since the last
@@ -188,8 +140,8 @@ public class WorkerProgress implements Writable {
   /**
    * Notify this class that next computation superstep is starting
    *
-   * @param superstep Superstep which is starting
-   * @param verticesToCompute How many vertices are there to compute
+   * @param superstep           Superstep which is starting
+   * @param verticesToCompute   How many vertices are there to compute
    * @param partitionsToCompute How many partitions are there to compute
    */
   public synchronized void startSuperstep(long superstep,
@@ -221,7 +173,7 @@ public class WorkerProgress implements Writable {
   /**
    * Notify this class that worker is starting to store data
    *
-   * @param verticesToStore How many vertices should be stored
+   * @param verticesToStore   How many vertices should be stored
    * @param partitionsToStore How many partitions should be stored
    */
   public synchronized void startStoring(long verticesToStore,
@@ -260,10 +212,6 @@ public class WorkerProgress implements Writable {
     storingDone = true;
   }
 
-  public synchronized void setTaskId(int taskId) {
-    this.taskId = taskId;
-  }
-
   /**
    * Update memory info
    */
@@ -271,58 +219,101 @@ public class WorkerProgress implements Writable {
     freeMemoryMB = MemoryUtils.freeMemoryMB();
   }
 
+  @ThriftField(1)
   public synchronized long getCurrentSuperstep() {
     return currentSuperstep;
   }
 
+  @ThriftField(2)
   public synchronized long getVerticesLoaded() {
     return verticesLoaded;
   }
 
+  @ThriftField(3)
   public synchronized int getVertexInputSplitsLoaded() {
     return vertexInputSplitsLoaded;
   }
 
+  @ThriftField(4)
+  public synchronized boolean isLoadingVerticesDone() {
+    return loadingVerticesDone;
+  }
+
+  @ThriftField(5)
   public synchronized long getEdgesLoaded() {
     return edgesLoaded;
   }
 
+  @ThriftField(6)
   public synchronized int getEdgeInputSplitsLoaded() {
     return edgeInputSplitsLoaded;
   }
 
+  @ThriftField(7)
+  public synchronized boolean isLoadingEdgesDone() {
+    return loadingEdgesDone;
+  }
+
+  @ThriftField(8)
   public synchronized long getVerticesToCompute() {
     return verticesToCompute;
   }
 
+  @ThriftField(9)
   public synchronized long getVerticesComputed() {
     return verticesComputed;
   }
 
+  @ThriftField(10)
   public synchronized int getPartitionsToCompute() {
     return partitionsToCompute;
   }
 
+  @ThriftField(11)
   public synchronized int getPartitionsComputed() {
     return partitionsComputed;
   }
 
+  @ThriftField(12)
+  public synchronized boolean isComputationDone() {
+    return computationDone;
+  }
+
+  @ThriftField(13)
   public synchronized long getVerticesToStore() {
     return verticesToStore;
   }
 
+  @ThriftField(14)
   public synchronized long getVerticesStored() {
     return verticesStored;
   }
 
+  @ThriftField(15)
   public synchronized int getPartitionsToStore() {
     return partitionsToStore;
   }
 
+  @ThriftField(16)
   public synchronized int getPartitionsStored() {
     return partitionsStored;
   }
 
+  @ThriftField(17)
+  public synchronized boolean isStoringDone() {
+    return storingDone;
+  }
+
+  @ThriftField(18)
+  public synchronized int getTaskId() {
+    return taskId;
+  }
+
+  @ThriftField(19)
+  public synchronized double getFreeMemoryMB() {
+    return freeMemoryMB;
+  }
+
   public synchronized boolean isInputSuperstep() {
     return currentSuperstep == -1;
   }
@@ -335,69 +326,98 @@ public class WorkerProgress implements Writable {
     return currentSuperstep == Long.MAX_VALUE;
   }
 
-  public synchronized int getTaskId() {
-    return taskId;
+  @ThriftField
+  public void setCurrentSuperstep(long currentSuperstep) {
+    this.currentSuperstep = currentSuperstep;
   }
 
-  public synchronized double getFreeMemoryMB() {
-    return freeMemoryMB;
+  @ThriftField
+  public void setVerticesLoaded(long verticesLoaded) {
+    this.verticesLoaded = verticesLoaded;
   }
 
-  @Override
-  public synchronized void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeLong(currentSuperstep);
+  @ThriftField
+  public void setVertexInputSplitsLoaded(int vertexInputSplitsLoaded) {
+    this.vertexInputSplitsLoaded = vertexInputSplitsLoaded;
+  }
 
-    dataOutput.writeLong(verticesLoaded);
-    dataOutput.writeInt(vertexInputSplitsLoaded);
-    dataOutput.writeBoolean(loadingVerticesDone);
-    dataOutput.writeLong(edgesLoaded);
-    dataOutput.writeInt(edgeInputSplitsLoaded);
-    dataOutput.writeBoolean(loadingEdgesDone);
+  @ThriftField
+  public void setLoadingVerticesDone(boolean loadingVerticesDone) {
+    this.loadingVerticesDone = loadingVerticesDone;
+  }
 
-    dataOutput.writeLong(verticesToCompute);
-    dataOutput.writeLong(verticesComputed);
-    dataOutput.writeInt(partitionsToCompute);
-    dataOutput.writeInt(partitionsComputed);
+  @ThriftField
+  public void setEdgesLoaded(long edgesLoaded) {
+    this.edgesLoaded = edgesLoaded;
+  }
 
-    dataOutput.writeBoolean(computationDone);
+  @ThriftField
+  public void setEdgeInputSplitsLoaded(int edgeInputSplitsLoaded) {
+    this.edgeInputSplitsLoaded = edgeInputSplitsLoaded;
+  }
 
-    dataOutput.writeLong(verticesToStore);
-    dataOutput.writeLong(verticesStored);
-    dataOutput.writeInt(partitionsToStore);
-    dataOutput.writeInt(partitionsStored);
-    dataOutput.writeBoolean(storingDone);
+  @ThriftField
+  public void setLoadingEdgesDone(boolean loadingEdgesDone) {
+    this.loadingEdgesDone = loadingEdgesDone;
+  }
 
-    dataOutput.writeInt(taskId);
+  @ThriftField
+  public void setVerticesToCompute(long verticesToCompute) {
+    this.verticesToCompute = verticesToCompute;
+  }
 
-    dataOutput.writeDouble(freeMemoryMB);
+  @ThriftField
+  public void setVerticesComputed(long verticesComputed) {
+    this.verticesComputed = verticesComputed;
   }
 
-  @Override
-  public synchronized void readFields(DataInput dataInput) throws IOException {
-    currentSuperstep = dataInput.readLong();
+  @ThriftField
+  public void setPartitionsToCompute(int partitionsToCompute) {
+    this.partitionsToCompute = partitionsToCompute;
+  }
 
-    verticesLoaded = dataInput.readLong();
-    vertexInputSplitsLoaded = dataInput.readInt();
-    loadingVerticesDone = dataInput.readBoolean();
-    edgesLoaded = dataInput.readLong();
-    edgeInputSplitsLoaded = dataInput.readInt();
-    loadingEdgesDone = dataInput.readBoolean();
+  @ThriftField
+  public void setPartitionsComputed(int partitionsComputed) {
+    this.partitionsComputed = partitionsComputed;
+  }
 
-    verticesToCompute = dataInput.readLong();
-    verticesComputed = dataInput.readLong();
-    partitionsToCompute = dataInput.readInt();
-    partitionsComputed = dataInput.readInt();
+  @ThriftField
+  public void setComputationDone(boolean computationDone) {
+    this.computationDone = computationDone;
+  }
 
-    computationDone = dataInput.readBoolean();
+  @ThriftField
+  public void setVerticesToStore(long verticesToStore) {
+    this.verticesToStore = verticesToStore;
+  }
 
-    verticesToStore = dataInput.readLong();
-    verticesStored = dataInput.readLong();
-    partitionsToStore = dataInput.readInt();
-    partitionsStored = dataInput.readInt();
-    storingDone = dataInput.readBoolean();
+  @ThriftField
+  public void setVerticesStored(long verticesStored) {
+    this.verticesStored = verticesStored;
+  }
 
-    taskId = dataInput.readInt();
+  @ThriftField
+  public void setPartitionsToStore(int partitionsToStore) {
+    this.partitionsToStore = partitionsToStore;
+  }
+
+  @ThriftField
+  public void setPartitionsStored(int partitionsStored) {
+    this.partitionsStored = partitionsStored;
+  }
+
+  @ThriftField
+  public void setStoringDone(boolean storingDone) {
+    this.storingDone = storingDone;
+  }
 
-    freeMemoryMB = dataInput.readDouble();
+  @ThriftField
+  public void setFreeMemoryMB(double freeMemoryMB) {
+    this.freeMemoryMB = freeMemoryMB;
+  }
+
+  @ThriftField
+  public synchronized void setTaskId(int taskId) {
+    this.taskId = taskId;
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgressWriter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgressWriter.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgressWriter.java
index 4ff5bb1..dae9963 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgressWriter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerProgressWriter.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.worker;
 
-import org.apache.giraph.zk.ZooKeeperExt;
+import org.apache.giraph.job.JobProgressTracker;
 import org.apache.log4j.Logger;
 
 /**
@@ -31,33 +31,27 @@ public class WorkerProgressWriter {
   /** How often to update worker's progress */
   private static final int WRITE_UPDATE_PERIOD_MILLISECONDS = 10 * 1000;
 
+  /** Job progress tracker */
+  private final JobProgressTracker jobProgressTracker;
   /** Thread which writes worker's progress */
   private final Thread writerThread;
   /** Whether worker finished application */
   private volatile boolean finished = false;
-  /** Path where this worker's progress should be stored */
-  private final String myProgressPath;
-  /** ZooKeeperExt */
-  private final ZooKeeperExt zk;
 
   /**
    * Constructor, starts separate thread to periodically update worker's
    * progress
    *
-   * @param myProgressPath Path where this worker's progress should be stored
-   * @param zk ZooKeeperExt
+   * @param jobProgressTracker JobProgressTracker to report job progress to
    */
-  public WorkerProgressWriter(String myProgressPath, ZooKeeperExt zk) {
-    this.myProgressPath = myProgressPath;
-    this.zk = zk;
+  public WorkerProgressWriter(JobProgressTracker jobProgressTracker) {
+    this.jobProgressTracker = jobProgressTracker;
     writerThread = new Thread(new Runnable() {
       @Override
       public void run() {
         try {
           while (!finished) {
-            WorkerProgress.get().updateMemory();
-            WorkerProgress.writeToZnode(WorkerProgressWriter.this.zk,
-                WorkerProgressWriter.this.myProgressPath);
+            updateAndSendProgress();
             double factor = 1 + Math.random();
             Thread.sleep((long) (WRITE_UPDATE_PERIOD_MILLISECONDS * factor));
           }
@@ -73,12 +67,20 @@ public class WorkerProgressWriter {
   }
 
   /**
+   * Update worker progress and send it
+   */
+  private void updateAndSendProgress() {
+    WorkerProgress.get().updateMemory();
+    jobProgressTracker.updateProgress(WorkerProgress.get());
+  }
+
+  /**
    * Stop the thread which writes worker's progress
    */
   public void stop() throws InterruptedException {
     finished = true;
     writerThread.interrupt();
     writerThread.join();
-    WorkerProgress.writeToZnode(zk, myProgressPath);
+    updateAndSendProgress();
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperManager.java
----------------------------------------------------------------------
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 73ef97b..b5816d7 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
@@ -619,13 +619,13 @@ public class ZooKeeperManager {
                 myidWriter = new FileWriter(zkDir + "/myid");
                 myidWriter.write(i + "\n");
               } finally {
-                Closeables.closeQuietly(myidWriter);
+                Closeables.close(myidWriter, true);
               }
             }
           }
         }
       } finally {
-        Closeables.closeQuietly(writer);
+        Closeables.close(writer, true);
       }
     } catch (IOException e) {
       throw new IllegalStateException(

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-core/src/test/java/org/apache/giraph/BspCase.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/BspCase.java b/giraph-core/src/test/java/org/apache/giraph/BspCase.java
index dd2369a..b372ab7 100644
--- a/giraph-core/src/test/java/org/apache/giraph/BspCase.java
+++ b/giraph-core/src/test/java/org/apache/giraph/BspCase.java
@@ -268,8 +268,8 @@ public class BspCase implements Watcher {
           numResults++;
         }
       } finally {
-        Closeables.closeQuietly(in);
-        Closeables.closeQuietly(reader);
+        Closeables.close(in, true);
+        Closeables.close(reader, true);
       }
     }
     return numResults;

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-examples/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-examples/pom.xml b/giraph-examples/pom.xml
index f8304a1..f95edcb 100644
--- a/giraph-examples/pom.xml
+++ b/giraph-examples/pom.xml
@@ -226,14 +226,6 @@ under the License.
           </plugin>
         </plugins>
       </build>
-      <dependencies>
-        <dependency>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-          <version>${dep.oldnetty.version}</version>
-          <scope>test</scope>
-        </dependency>
-      </dependencies>
     </profile>
 
     <profile>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
index 5612e5f..488e1ea 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
@@ -461,8 +461,8 @@ public class
           assertEquals(numVertices, (long) vertexCounts.get(maxSuperstep));
 
         } finally {
-          Closeables.closeQuietly(in);
-          Closeables.closeQuietly(reader);
+          Closeables.close(in, true);
+          Closeables.close(reader, true);
         }
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/giraph-hive/src/main/java/org/apache/giraph/hive/jython/HiveJythonUtils.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/jython/HiveJythonUtils.java b/giraph-hive/src/main/java/org/apache/giraph/hive/jython/HiveJythonUtils.java
index 7ae8bc3..517901a 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/jython/HiveJythonUtils.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/jython/HiveJythonUtils.java
@@ -176,7 +176,7 @@ public class HiveJythonUtils {
       jythonJob = parseJythonStreams(interpreter, streams);
     } finally {
       for (InputStream stream : streams) {
-        Closeables.closeQuietly(stream);
+        Closeables.close(stream, true);
       }
     }
     return jythonJob;

http://git-wip-us.apache.org/repos/asf/giraph/blob/4485e563/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b4d78ae..2e3eb63 100644
--- a/pom.xml
+++ b/pom.xml
@@ -289,6 +289,7 @@ under the License.
     <dep.commons-logging.version>1.1.1</dep.commons-logging.version>
     <dep.commons-io.version>2.1</dep.commons-io.version>
     <dep.commons-net.version>3.1</dep.commons-net.version>
+    <dep.facebook-swift.version>0.13.1</dep.facebook-swift.version>
     <dep.fasterxml-jackson.version>2.1.2</dep.fasterxml-jackson.version>
     <dep.fastutil.version>6.5.4</dep.fastutil.version>
     <dep.google.findbugs.version>2.0.2</dep.google.findbugs.version>
@@ -309,7 +310,7 @@ under the License.
     <!-- note: old version of netty is required by hadoop_facebook for tests to succeed -->
     <dep.oldnetty.version>3.2.2.Final</dep.oldnetty.version>
     <dep.netty.version>4.0.14.Final</dep.netty.version>
-    <dep.paranamer.version>2.3</dep.paranamer.version>
+    <dep.paranamer.version>2.5.2</dep.paranamer.version>
     <dep.slf4j.version>1.7.5</dep.slf4j.version>
     <dep.tinkerpop.rexter.version>2.4.0</dep.tinkerpop.rexter.version>
     <dep.typetools.version>0.2.1</dep.typetools.version>
@@ -1389,6 +1390,14 @@ under the License.
             <groupId>commons-beanutils</groupId>
             <artifactId>commons-beanutils-core</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>io.airlift</groupId>
+            <artifactId>units</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>javax.validation</groupId>
+            <artifactId>validation-api</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>
@@ -1404,6 +1413,14 @@ under the License.
             <groupId>org.apache.zookeeper</groupId>
             <artifactId>zookeeper</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>io.airlift</groupId>
+            <artifactId>units</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>javax.validation</groupId>
+            <artifactId>validation-api</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>
@@ -1594,6 +1611,60 @@ under the License.
         <version>${dep.yourkit-api.version}</version>
       </dependency>
       <dependency>
+        <groupId>com.facebook.nifty</groupId>
+        <artifactId>nifty-client</artifactId>
+        <version>${dep.facebook-swift.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>com.facebook.swift</groupId>
+        <artifactId>swift-codec</artifactId>
+        <version>${dep.facebook-swift.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.ow2.asm</groupId>
+            <artifactId>asm-all</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>com.facebook.swift</groupId>
+        <artifactId>swift-annotations</artifactId>
+        <version>${dep.facebook-swift.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.facebook.swift</groupId>
+        <artifactId>swift-service</artifactId>
+        <version>${dep.facebook-swift.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-beanutils</groupId>
+            <artifactId>commons-beanutils-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
         <groupId>log4j</groupId>
         <artifactId>log4j</artifactId>
         <version>${dep.log4j.version}</version>
@@ -2086,6 +2157,14 @@ under the License.
             <groupId>commons-beanutils</groupId>
             <artifactId>commons-beanutils-core</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>io.airlift</groupId>
+            <artifactId>units</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>javax.validation</groupId>
+            <artifactId>validation-api</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>


[40/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Update Giraph-Gora documentation and make it available from giraph.a.o


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/46d7d901
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/46d7d901
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/46d7d901

Branch: refs/heads/release-1.1
Commit: 46d7d901e35e83e4b95f95edda3217d08fb5b690
Parents: f43f450
Author: Lewis John McGibbney <le...@jpl.nasa.gov>
Authored: Sat Oct 11 18:30:36 2014 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Sun Oct 19 16:51:13 2014 -0700

----------------------------------------------------------------------
 src/site/site.xml | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/46d7d901/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index de649e0..8fd1490 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -94,11 +94,10 @@
         <item name="Out-of-core" href="ooc.html"/>
         <item name="Implementation" href="implementation.html"/>
         <item name="Page Rank Example" href="pagerank.html"/>
-        <item name="Input/Output in Giraph" href="io.html">
-          <item name="Hive" href="hive.html"/>
-	  <item name="Gora" href="gora.html"/>
-    	  <item name="Rexster I/O" href="rexster.html"/>
-        </item>
+        <item name="Input/Output in Giraph" href="io.html"/>
+        <item name="Hive I/O" href="hive.html"/>
+	<item name="Gora I/O" href="gora.html"/>
+    	<item name="Rexster I/O" href="rexster.html"/>
       </item>
       <item name="How to generate patches?" href="generating_patches.html" />
       <item name="How to build this site?" href="build_site.html" />


[23/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-893: Implement preLoad & postSave on workerObservers (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/fc267734
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/fc267734
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/fc267734

Branch: refs/heads/release-1.1
Commit: fc267734887dddd7c379144d6d5499fe3d541db8
Parents: d455270
Author: Pavan Kumar <pa...@fb.com>
Authored: Tue Aug 5 14:35:04 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Tue Aug 5 14:35:32 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 +
 .../apache/giraph/graph/GraphTaskManager.java   | 22 ++++++++++
 .../apache/giraph/utils/JMapHistoDumper.java    | 12 +++++-
 .../org/apache/giraph/utils/LogVersions.java    |  6 +++
 .../giraph/utils/ReactiveJMapHistoDumper.java   | 12 +++++-
 .../giraph/worker/DefaultWorkerObserver.java    | 16 +++++--
 .../apache/giraph/worker/WorkerObserver.java    | 44 +++++++++++++-------
 7 files changed, 90 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 3c8e155..300215a 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-893: Implement preLoad & postSave on workerObservers (pavanka)
+
   GIRAPH-936: AsyncMessageStoreWrapper threads are not daemonized (edunov via majakabiljo)
 
   GIRAPH-934: Allow having state in aggregators (ikabiljo via majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 684f4eb..6ebb002 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
@@ -255,6 +255,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     if (checkTaskState()) {
       return;
     }
+    preLoadOnWorkerObservers();
     finishedSuperstepStats = serviceWorker.setup();
     if (collectInputSuperstepStats(finishedSuperstepStats)) {
       return;
@@ -830,6 +831,26 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   }
 
   /**
+   * Executes preLoad() on worker observers.
+   */
+  private void preLoadOnWorkerObservers() {
+    for (WorkerObserver obs : serviceWorker.getWorkerObservers()) {
+      obs.preLoad();
+      context.progress();
+    }
+  }
+
+  /**
+   * Executes postSave() on worker observers.
+   */
+  private void postSaveOnWorkerObservers() {
+    for (WorkerObserver obs : serviceWorker.getWorkerObservers()) {
+      obs.postSave();
+      context.progress();
+    }
+  }
+
+  /**
    * Called by owner of this GraphTaskManager object on each compute node
    */
   public void cleanup()
@@ -843,6 +864,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
 
     if (serviceWorker != null) {
       serviceWorker.cleanup(finishedSuperstepStats);
+      postSaveOnWorkerObservers();
     }
     try {
       if (masterThread != null) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/giraph-core/src/main/java/org/apache/giraph/utils/JMapHistoDumper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/JMapHistoDumper.java b/giraph-core/src/main/java/org/apache/giraph/utils/JMapHistoDumper.java
index 3bcf42e..f90337f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/JMapHistoDumper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/JMapHistoDumper.java
@@ -45,17 +45,25 @@ public class JMapHistoDumper implements MasterObserver, WorkerObserver {
   private boolean stop = false;
 
   @Override
-  public void preApplication() {
+  public void preLoad() {
     // This is called by both WorkerObserver and MasterObserver
     startJMapThread();
   }
 
   @Override
-  public void postApplication() {
+  public void postSave() {
     // This is called by both WorkerObserver and MasterObserver
     joinJMapThread();
   }
 
+  @Override
+  public void preApplication() {
+  }
+
+  @Override
+  public void postApplication() {
+  }
+
   /**
    * Join the jmap thread
    */

http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/giraph-core/src/main/java/org/apache/giraph/utils/LogVersions.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/LogVersions.java b/giraph-core/src/main/java/org/apache/giraph/utils/LogVersions.java
index 8305df7..5bdad87 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/LogVersions.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/LogVersions.java
@@ -37,6 +37,12 @@ public class LogVersions implements WorkerObserver, MasterObserver {
   }
 
   @Override
+  public void preLoad() { }
+
+  @Override
+  public void postSave() { }
+
+  @Override
   public void preApplication() {
     GiraphDepVersions.get().logVersionsUsed();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java b/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
index 68369d9..844f929 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ReactiveJMapHistoDumper.java
@@ -52,17 +52,25 @@ public class ReactiveJMapHistoDumper extends
   private volatile boolean stop = false;
 
   @Override
-  public void preApplication() {
+  public void preLoad() {
     // This is called by both WorkerObserver and MasterObserver
     startSupervisorThread();
   }
 
   @Override
-  public void postApplication() {
+  public void postSave() {
     // This is called by both WorkerObserver and MasterObserver
     joinSupervisorThread();
   }
 
+  @Override
+  public void preApplication() {
+  }
+
+  @Override
+  public void postApplication() {
+  }
+
   /**
    * Join the supervisor thread
    */

http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/giraph-core/src/main/java/org/apache/giraph/worker/DefaultWorkerObserver.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/DefaultWorkerObserver.java b/giraph-core/src/main/java/org/apache/giraph/worker/DefaultWorkerObserver.java
index 5c8c94a..694c4ed 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/DefaultWorkerObserver.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/DefaultWorkerObserver.java
@@ -18,18 +18,26 @@
 
 package org.apache.giraph.worker;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 
 /**
  * Default Observer for Worker that does nothing.
  */
-public class DefaultWorkerObserver implements WorkerObserver,
-    ImmutableClassesGiraphConfigurable {
-  /** The configuration stored here */
+public class DefaultWorkerObserver implements WorkerObserver {
+  /**
+   * The configuration stored here
+   */
   private ImmutableClassesGiraphConfiguration conf;
 
   @Override
+  public void preLoad() {
+  }
+
+  @Override
+  public void postSave() {
+  }
+
+  @Override
   public void preApplication() {
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/fc267734/giraph-core/src/main/java/org/apache/giraph/worker/WorkerObserver.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerObserver.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerObserver.java
index fc62629..b1b40db 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerObserver.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerObserver.java
@@ -18,39 +18,51 @@
 
 package org.apache.giraph.worker;
 
+import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+
 /**
- * Observer for Worker
+ * Observer for worker. The user can subclass and register an observer with the
+ * Giraph framework. The framework will execute methods of the observer at
+ * designated moments of computation on each worker.
  */
-public interface WorkerObserver {
+public interface WorkerObserver extends ImmutableClassesGiraphConfigurable {
   /**
-   * Initialize the WorkerContext.
-   * This method is executed once on each Worker before the first
-   * superstep starts.
+   * Initialize the observer. This method is executed once on each worker before
+   * loading.
    */
-  void preApplication();
+  void preLoad();
 
   /**
-   * Finalize the WorkerContext.
-   * This method is executed once on each Worker after the last
-   * superstep ends.
+   * Initialize the observer. This method is executed once on each worker after
+   * loading before the first superstep starts.
    */
-  void postApplication();
+  void preApplication();
 
   /**
-   * Execute user code.
-   * This method is executed once on each Worker before each
-   * superstep starts.
+   * Execute the observer. This method is executed once on each worker before
+   * each superstep starts.
    *
    * @param superstep number of superstep
    */
   void preSuperstep(long superstep);
 
   /**
-   * Execute user code.
-   * This method is executed once on each Worker after each
-   * superstep ends.
+   * Execute the observer. This method is executed once on each worker after
+   * each superstep ends.
    *
    * @param superstep number of superstep
    */
   void postSuperstep(long superstep);
+
+  /**
+   * Finalize the observer. This method is executed once on each worker after
+   * the last superstep ends before saving.
+   */
+  void postApplication();
+
+  /**
+   * Finalize the observer. This method is executed once on each worker after
+   * saving.
+   */
+  void postSave();
 }


[07/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-916: Wrong number of vertices stored reported to command line (majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/441ec8be
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/441ec8be
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/441ec8be

Branch: refs/heads/release-1.1
Commit: 441ec8bef03644c9d629f871993c14ff5442409a
Parents: 9e1a5a0
Author: Maja Kabiljo <ma...@fb.com>
Authored: Tue Jun 24 09:55:28 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Tue Jun 24 09:55:28 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                                          | 2 ++
 .../src/main/java/org/apache/giraph/worker/BspServiceWorker.java   | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/441ec8be/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 8da555d..2929c66 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-916: Wrong number of vertices stored reported to command line (majakabiljo)
+
   GIRAPH-919: Add worker to worker communication (majakabiljo)
 
   GIRAPH-922: SimpleEdgeStore has a bug causing NPE (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/441ec8be/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index 8dcf19a..dbe6a45 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -1109,7 +1109,7 @@ public class BspServiceWorker<I extends WritableComparable,
             vertexWriter.setConf(getConfiguration());
             vertexWriter.initialize(getContext());
             long nextPrintVertices = 0;
-            long nextUpdateProgressVertices = 0;
+            long nextUpdateProgressVertices = VERTICES_TO_UPDATE_PROGRESS;
             long nextPrintMsecs = System.currentTimeMillis() + 15000;
             int partitionIndex = 0;
             int numPartitions = getPartitionStore().getNumPartitions();


[25/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-935: Loosen modifiers when needed (ikabiljo via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/b914fec7
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/b914fec7
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/b914fec7

Branch: refs/heads/release-1.1
Commit: b914fec7e2bc669c20531ae7c2fbf340f8de9a8e
Parents: d8b4439
Author: Maja Kabiljo <ma...@fb.com>
Authored: Thu Aug 7 15:36:00 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Thu Aug 7 15:36:56 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 +
 .../comm/messages/OneMessagePerVertexStore.java |  2 +-
 .../conf/GiraphConfigurationSettable.java       | 39 ++++++++++++++++++++
 .../ImmutableClassesGiraphConfigurable.java     | 10 +----
 .../ImmutableClassesGiraphConfiguration.java    |  9 +++--
 .../org/apache/giraph/edge/DefaultEdge.java     |  2 +-
 .../java/org/apache/giraph/utils/TestGraph.java | 21 ++++++-----
 7 files changed, 62 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index be3c59f..08abaaa 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-935: Loosen modifiers when needed (ikabiljo via majakabiljo)
+
   GIRAPH-937 - Upgrade Hive I/O to 0.26. (aching)
 
   GIRAPH-893: Implement preLoad & postSave on workerObservers (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
index d3942d4..349e58b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
@@ -52,7 +52,7 @@ public class OneMessagePerVertexStore<I extends WritableComparable,
    * @param messageCombiner MessageCombiner for messages
    * @param config Hadoop configuration
    */
-  OneMessagePerVertexStore(
+  public OneMessagePerVertexStore(
       MessageValueFactory<M> messageValueFactory,
       CentralizedServiceWorker<I, ?, ?> service,
       MessageCombiner<I, M> messageCombiner,

http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfigurationSettable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfigurationSettable.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfigurationSettable.java
new file mode 100644
index 0000000..ce8384c
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfigurationSettable.java
@@ -0,0 +1,39 @@
+/*
+ * 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.conf;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Can be instantiated with ImmutableClassesGiraphConfiguration
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ */
+public interface GiraphConfigurationSettable<
+    I extends WritableComparable, V extends Writable, E extends Writable> {
+  /**
+   * Set the configuration to be used by this object.
+   *
+   * @param configuration Set configuration
+   */
+  void setConf(ImmutableClassesGiraphConfiguration<I, V, E> configuration);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
index dbffbc7..4291c86 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
@@ -29,14 +29,8 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <E> Edge data
  */
 public interface ImmutableClassesGiraphConfigurable<
-    I extends WritableComparable, V extends Writable, E extends Writable> {
-  /**
-   * Set the configuration to be used by this object.
-   *
-   * @param configuration Set configuration
-   */
-  void setConf(ImmutableClassesGiraphConfiguration<I, V, E> configuration);
-
+    I extends WritableComparable, V extends Writable, E extends Writable>
+    extends GiraphConfigurationSettable<I, V, E> {
   /**
    * Return the configuration used by this object.
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/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 32d08f8..8b67490 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,13 +18,13 @@
 
 package org.apache.giraph.conf;
 
-import com.google.common.base.Preconditions;
 import io.netty.handler.codec.ByteToMessageDecoder;
 import io.netty.handler.codec.MessageToByteEncoder;
 import io.netty.handler.codec.compression.JdkZlibDecoder;
 import io.netty.handler.codec.compression.JdkZlibEncoder;
 import io.netty.handler.codec.compression.SnappyFramedDecoder;
 import io.netty.handler.codec.compression.SnappyFramedEncoder;
+
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.edge.Edge;
@@ -89,6 +89,8 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.util.Progressable;
 
+import com.google.common.base.Preconditions;
+
 
 /**
  * The classes set here are immutable, the remaining configuration is mutable.
@@ -150,8 +152,8 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    * @param obj Object
    */
   public void configureIfPossible(Object obj) {
-    if (obj instanceof ImmutableClassesGiraphConfigurable) {
-      ((ImmutableClassesGiraphConfigurable) obj).setConf(this);
+    if (obj instanceof GiraphConfigurationSettable) {
+      ((GiraphConfigurationSettable) obj).setConf(this);
     }
   }
 
@@ -520,6 +522,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return User's combiner class
    */
+  @Override
   public Class<? extends MessageCombiner<I, ? extends Writable>>
   getMessageCombinerClass() {
     return classes.getMessageCombinerClass();

http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/giraph-core/src/main/java/org/apache/giraph/edge/DefaultEdge.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/DefaultEdge.java b/giraph-core/src/main/java/org/apache/giraph/edge/DefaultEdge.java
index d7cfb2f..0a558db 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/DefaultEdge.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/DefaultEdge.java
@@ -47,7 +47,7 @@ public class DefaultEdge<I extends WritableComparable, E extends Writable>
    * @param targetVertexId Desination vertex id.
    * @param value Value of the edge.
    */
-  DefaultEdge(I targetVertexId, E value) {
+  protected DefaultEdge(I targetVertexId, E value) {
     this.targetVertexId = targetVertexId;
     this.value = value;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/b914fec7/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java b/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
index 183e90b..7a37167 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
@@ -18,6 +18,11 @@
 
 package org.apache.giraph.utils;
 
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
@@ -30,11 +35,6 @@ import com.google.common.base.Objects;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
 /**
  * TestGraph class for in-memory testing.
  *
@@ -47,9 +47,9 @@ public class TestGraph<I extends WritableComparable,
                        E extends Writable>
                        implements Iterable<Vertex<I, V, E>> {
   /** The vertex values */
-  private final HashMap<I, Vertex<I, V, E>> vertices = Maps.newHashMap();
+  protected HashMap<I, Vertex<I, V, E>> vertices = Maps.newHashMap();
   /** The configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
+  protected ImmutableClassesGiraphConfiguration<I, V, E> conf;
 
   /**
    * Constructor requiring classes
@@ -112,7 +112,7 @@ public class TestGraph<I extends WritableComparable,
       vertices.put(vertexId, v);
     }
     vertices.get(vertexId)
-      .addEdge((Edge<I, E>) EdgeFactory.create(edgePair.getKey(),
+      .addEdge(EdgeFactory.create(edgePair.getKey(),
                                                edgePair.getValue()));
     return this;
   }
@@ -132,7 +132,7 @@ public class TestGraph<I extends WritableComparable,
       vertices.put(vertexId, v);
     }
     vertices.get(vertexId)
-      .addEdge((Edge<I, E>) EdgeFactory.create(toVertex, edgeValue));
+      .addEdge(EdgeFactory.create(toVertex, edgeValue));
     return this;
   }
   /**
@@ -149,6 +149,7 @@ public class TestGraph<I extends WritableComparable,
    *
    * @return the iterator
    */
+  @Override
   public Iterator<Vertex<I, V, E>> iterator() {
     return vertices.values().iterator();
   }
@@ -173,7 +174,7 @@ public class TestGraph<I extends WritableComparable,
   createEdges(Entry<I, E>... destEdgess) {
     List<Edge<I, E>> edgesList = Lists.newArrayList();
     for (Entry<I, E> e: destEdgess) {
-      edgesList.add((Edge<I, E>) EdgeFactory.create(e.getKey(), e.getValue()));
+      edgesList.add(EdgeFactory.create(e.getKey(), e.getValue()));
     }
     return edgesList;
   }


[06/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-919: Add worker to worker communication (majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/9e1a5a05
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/9e1a5a05
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/9e1a5a05

Branch: refs/heads/release-1.1
Commit: 9e1a5a05368a1543aaf6bea0176b9d79058e03b9
Parents: b218d72
Author: Maja Kabiljo <ma...@fb.com>
Authored: Tue Jun 24 09:51:40 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Tue Jun 24 09:51:40 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 +
 .../java/org/apache/giraph/comm/ServerData.java | 35 +++++++++
 .../giraph/comm/requests/RequestType.java       |  4 +-
 .../SendWorkerToWorkerMessageRequest.java       | 80 ++++++++++++++++++++
 .../apache/giraph/graph/GraphTaskManager.java   |  1 +
 .../apache/giraph/master/BspServiceMaster.java  | 11 ++-
 .../org/apache/giraph/worker/WorkerContext.java | 73 +++++++++++++++++-
 7 files changed, 202 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index d315a9f..8da555d 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-919: Add worker to worker communication (majakabiljo)
+
   GIRAPH-922: SimpleEdgeStore has a bug causing NPE (pavanka)
 
   GIRAPH-915: With BigDataIO some messages can get ignored (majakabiljo via pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
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 f0ecca2..b3f8733 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
@@ -36,6 +36,9 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
 /**
@@ -84,6 +87,13 @@ public class ServerData<I extends WritableComparable,
   /** Service worker */
   private final CentralizedServiceWorker<I, V, E> serviceWorker;
 
+  /** Store for current messages from other workers to this worker */
+  private volatile List<Writable> currentWorkerToWorkerMessages =
+      Collections.synchronizedList(new ArrayList<Writable>());
+  /** Store for message from other workers to this worker for next superstep */
+  private volatile List<Writable> incomingWorkerToWorkerMessages =
+      Collections.synchronizedList(new ArrayList<Writable>());
+
   /**
    * Constructor.
    *
@@ -166,6 +176,10 @@ public class ServerData<I extends WritableComparable,
             messageStoreFactory.newStore(conf.getIncomingMessageValueFactory());
     incomingMessageStore =
         messageStoreFactory.newStore(conf.getOutgoingMessageValueFactory());
+
+    currentWorkerToWorkerMessages = incomingWorkerToWorkerMessages;
+    incomingWorkerToWorkerMessages =
+        Collections.synchronizedList(new ArrayList<Writable>());
   }
 
   /**
@@ -204,4 +218,25 @@ public class ServerData<I extends WritableComparable,
   public CentralizedServiceWorker<I, V, E> getServiceWorker() {
     return this.serviceWorker;
   }
+
+  /**
+   * Get and clear worker to worker messages for this superstep. Can be
+   * called only once per superstep.
+   *
+   * @return List of messages for this worker
+   */
+  public List<Writable> getAndClearCurrentWorkerToWorkerMessages() {
+    List<Writable> ret = currentWorkerToWorkerMessages;
+    currentWorkerToWorkerMessages = null;
+    return ret;
+  }
+
+  /**
+   * Add incoming message to this worker for next superstep. Thread-safe.
+   *
+   * @param message Message received
+   */
+  public void addIncomingWorkerToWorkerMessage(Writable message) {
+    incomingWorkerToWorkerMessages.add(message);
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
index 7fe2ae7..408295c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
@@ -58,7 +58,9 @@ public enum RequestType {
   /** Send aggregators from master to worker owners */
   SEND_AGGREGATORS_TO_OWNER_REQUEST(SendAggregatorsToOwnerRequest.class),
   /** Send aggregators from worker owner to other workers */
-  SEND_AGGREGATORS_TO_WORKER_REQUEST(SendAggregatorsToWorkerRequest.class);
+  SEND_AGGREGATORS_TO_WORKER_REQUEST(SendAggregatorsToWorkerRequest.class),
+  /** Send message from worker to worker */
+  SEND_WORKER_TO_WORKER_MESSAGE_REQUEST(SendWorkerToWorkerMessageRequest.class);
 
   /** Class of request which this type corresponds to */
   private final Class<? extends WritableRequest> requestClass;

http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerToWorkerMessageRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerToWorkerMessageRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerToWorkerMessageRequest.java
new file mode 100644
index 0000000..a2505ef
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerToWorkerMessageRequest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.comm.requests;
+
+import org.apache.giraph.comm.ServerData;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/** Request which sends any Writable message from one worker to another */
+public class SendWorkerToWorkerMessageRequest extends WritableRequest
+    implements WorkerRequest<WritableComparable, Writable, Writable> {
+  /** Message sent */
+  private Writable message;
+
+  /**
+   * Default constructor, for reflection
+   */
+  public SendWorkerToWorkerMessageRequest() {
+  }
+
+  /**
+   * Constructor with message
+   *
+   * @param message Message sent
+   */
+  public SendWorkerToWorkerMessageRequest(Writable message) {
+    this.message = message;
+  }
+
+  @Override
+  public RequestType getType() {
+    return RequestType.SEND_WORKER_TO_WORKER_MESSAGE_REQUEST;
+  }
+
+  @Override
+  void writeRequest(DataOutput output) throws IOException {
+    Text.writeString(output, message.getClass().getName());
+    message.write(output);
+  }
+
+  @Override
+  void readFieldsRequest(DataInput input) throws IOException {
+    String className = Text.readString(input);
+    try {
+      message = (Writable) Class.forName(className).newInstance();
+      message.readFields(input);
+    } catch (InstantiationException | IllegalAccessException |
+        ClassNotFoundException e) {
+      throw new IllegalStateException(
+          "readFieldsRequest: Exception occurred", e);
+    }
+  }
+
+  @Override
+  public void doRequest(
+      ServerData<WritableComparable, Writable, Writable> serverData) {
+    serverData.addIncomingWorkerToWorkerMessage(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 ad5fc91..e13eedd 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
@@ -432,6 +432,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     serviceWorker.prepareSuperstep();
 
     serviceWorker.getWorkerContext().setGraphState(graphState);
+    serviceWorker.getWorkerContext().setupSuperstep(serviceWorker);
     GiraphTimerContext preSuperstepTimer = wcPreSuperstepTimer.time();
     serviceWorker.getWorkerContext().preSuperstep();
     preSuperstepTimer.stop();

http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index ad7e045..02d4f2b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -98,6 +98,8 @@ import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -175,7 +177,7 @@ public class BspServiceMaster<I extends WritableComparable,
   private MasterServer masterServer;
   /** Master info */
   private MasterInfo masterInfo;
-  /** List of workers in current superstep */
+  /** List of workers in current superstep, sorted by task id */
   private List<WorkerInfo> chosenWorkerInfoList = Lists.newArrayList();
   /** Limit locality information added to each InputSplit znode */
   private final int localityLimit = 5;
@@ -1555,6 +1557,13 @@ public class BspServiceMaster<I extends WritableComparable,
       setJobStateFailed("coordinateSuperstep: Not enough healthy workers for " +
                     "superstep " + getSuperstep());
     } else {
+      // Sort this list, so order stays the same over supersteps
+      Collections.sort(chosenWorkerInfoList, new Comparator<WorkerInfo>() {
+        @Override
+        public int compare(WorkerInfo wi1, WorkerInfo wi2) {
+          return Integer.compare(wi1.getTaskId(), wi2.getTaskId());
+        }
+      });
       for (WorkerInfo workerInfo : chosenWorkerInfoList) {
         String workerInfoHealthyPath =
             getWorkerInfoHealthyPath(getApplicationAttempt(),

http://git-wip-us.apache.org/repos/asf/giraph/blob/9e1a5a05/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
index 17347db..29835c5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
@@ -18,24 +18,36 @@
 
 package org.apache.giraph.worker;
 
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.requests.SendWorkerToWorkerMessageRequest;
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.GraphState;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
 
+import java.util.List;
+
 /**
  * WorkerContext allows for the execution of user code
  * on a per-worker basis. There's one WorkerContext per worker.
  */
 @SuppressWarnings("rawtypes")
 public abstract class WorkerContext
-  extends DefaultImmutableClassesGiraphConfigurable
-  implements WorkerAggregatorUsage {
+    extends DefaultImmutableClassesGiraphConfigurable
+    implements WorkerAggregatorUsage {
+
   /** Global graph state */
   private GraphState graphState;
   /** Worker aggregator usage */
   private WorkerAggregatorUsage workerAggregatorUsage;
 
+  /** Service worker */
+  private CentralizedServiceWorker serviceWorker;
+  /** Sorted list of other participating workers */
+  private List<WorkerInfo> workerList;
+  /** Index of this worker within workerList */
+  private int myWorkerIndex;
+
   /**
    * Set the graph state.
    *
@@ -46,6 +58,17 @@ public abstract class WorkerContext
   }
 
   /**
+   * Setup superstep.
+   *
+   * @param serviceWorker Service worker containing all the information
+   */
+  public void setupSuperstep(CentralizedServiceWorker<?, ?, ?> serviceWorker) {
+    this.serviceWorker = serviceWorker;
+    workerList = serviceWorker.getWorkerInfoList();
+    myWorkerIndex = workerList.indexOf(serviceWorker.getWorkerInfo());
+  }
+
+  /**
    * Set worker aggregator usage
    *
    * @param workerAggregatorUsage Worker aggregator usage
@@ -81,6 +104,52 @@ public abstract class WorkerContext
   public abstract void preSuperstep();
 
   /**
+   * Get number of workers
+   *
+   * @return Number of workers
+   */
+  public int getWorkerCount() {
+    return workerList.size();
+  }
+
+  /**
+   * Get index for this worker
+   *
+   * @return Index of this worker
+   */
+  public int getMyWorkerIndex() {
+    return myWorkerIndex;
+  }
+
+  /**
+   * Get messages which other workers sent to this worker and clear them (can
+   * be called once per superstep)
+   *
+   * @return Messages received
+   */
+  public List<Writable> getAndClearMessagesFromOtherWorkers() {
+    return serviceWorker.getServerData().
+        getAndClearCurrentWorkerToWorkerMessages();
+  }
+
+  /**
+   * Send message to another worker
+   *
+   * @param message Message to send
+   * @param workerIndex Index of the worker to send the message to
+   */
+  public void sendMessageToWorker(Writable message, int workerIndex) {
+    SendWorkerToWorkerMessageRequest request =
+        new SendWorkerToWorkerMessageRequest(message);
+    if (workerIndex == myWorkerIndex) {
+      request.doRequest(serviceWorker.getServerData());
+    } else {
+      serviceWorker.getWorkerClient().sendWritableRequest(
+          workerList.get(workerIndex).getTaskId(), request);
+    }
+  }
+
+  /**
    * Execute user code.
    * This method is executed once on each Worker after each
    * superstep ends.


[41/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-947: Compilation error with hadoop 2.5.x fixed.

1. Munge symbol 'STATIC_SASL_SYMBOL' was added.
2. STATIC_SASL_SYMBOL was added to all build profiles, except hadoop_2 and hadoop_snapshot. Profile 2.1.1-SNAPSHOT is not munge-free anymore.
3. Default hadoop version of profile hadoop_2 was changed: 2.2.0 -> 2.5.0


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/0bcc76de
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/0bcc76de
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/0bcc76de

Branch: refs/heads/release-1.1
Commit: 0bcc76de532636985a020ed4f41129041bffa850
Parents: 46d7d90
Author: dongjinleekr <do...@gmail.com>
Authored: Wed Sep 24 18:03:51 2014 +0900
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Sun Oct 19 16:52:45 2014 -0700

----------------------------------------------------------------------
 .../giraph/comm/netty/SaslNettyClient.java      | 22 ++++++++++++++++---
 .../giraph/comm/netty/SaslNettyServer.java      | 23 +++++++++++++++++---
 pom.xml                                         | 23 ++++++++++----------
 3 files changed, 51 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/0bcc76de/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyClient.java
index 840fee4..431691a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyClient.java
@@ -24,6 +24,9 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.security.Credentials;
+/*if_not[STATIC_SASL_SYMBOL]*/
+import org.apache.hadoop.security.SaslPropertiesResolver;
+/*end[STATIC_SASL_SYMBOL]*/
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.token.Token;
@@ -74,9 +77,22 @@ public class SaslNettyClient {
             AuthMethod.DIGEST.getMechanismName() +
             " client to authenticate to service at " + token.getService());
       }
-      saslClient = Sasl.createSaslClient(new String[] { AuthMethod.DIGEST
-          .getMechanismName() }, null, null, SaslRpcServer.SASL_DEFAULT_REALM,
-          SaslRpcServer.SASL_PROPS, new SaslClientCallbackHandler(token));
+      /*if[STATIC_SASL_SYMBOL]
+      saslClient =
+          Sasl.createSaslClient(
+              new String[] { AuthMethod.DIGEST.getMechanismName() }, null,
+              null, SaslRpcServer.SASL_DEFAULT_REALM, SaslRpcServer.SASL_PROPS,
+              new SaslClientCallbackHandler(token));
+      else[STATIC_SASL_SYMBOL]*/
+      SaslPropertiesResolver saslPropsResolver =
+          SaslPropertiesResolver.getInstance(new Configuration());
+      saslClient =
+          Sasl.createSaslClient(
+              new String[] { AuthMethod.DIGEST.getMechanismName() }, null,
+              null, SaslRpcServer.SASL_DEFAULT_REALM,
+              saslPropsResolver.getDefaultProperties(),
+              new SaslClientCallbackHandler(token));
+      /*end[STATIC_SASL_SYMBOL]*/
     } catch (IOException e) {
       LOG.error("SaslNettyClient: Could not obtain job token for Netty " +
           "Client to use to authenticate with a Netty Server.");

http://git-wip-us.apache.org/repos/asf/giraph/blob/0bcc76de/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyServer.java
index 9039141..a1d410f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/SaslNettyServer.java
@@ -19,12 +19,18 @@ package org.apache.giraph.comm.netty;
 
 import org.apache.commons.net.util.Base64;
 import org.apache.hadoop.classification.InterfaceStability;
+/*if_not[STATIC_SASL_SYMBOL]*/
+import org.apache.hadoop.conf.Configuration;
+/*end[STATIC_SASL_SYMBOL]*/
 /*if[HADOOP_1_SECURITY]
 else[HADOOP_1_SECURITY]*/
 import org.apache.hadoop.ipc.StandbyException;
 /*end[HADOOP_1_SECURITY]*/
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+/*if_not[STATIC_SASL_SYMBOL]*/
+import org.apache.hadoop.security.SaslPropertiesResolver;
+/*end[STATIC_SASL_SYMBOL]*/
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.log4j.Logger;
 
@@ -92,9 +98,20 @@ else[HADOOP_1_SECRET_MANAGER]*/
     try {
       SaslDigestCallbackHandler ch =
           new SaslNettyServer.SaslDigestCallbackHandler(secretManager);
-      saslServer = Sasl.createSaslServer(SaslNettyServer.AuthMethod.DIGEST
-          .getMechanismName(), null, SaslRpcServer.SASL_DEFAULT_REALM,
-          SaslRpcServer.SASL_PROPS, ch);
+      /*if[STATIC_SASL_SYMBOL]
+      saslServer =
+          Sasl.createSaslServer(
+              SaslNettyServer.AuthMethod.DIGEST.getMechanismName(), null,
+              SaslRpcServer.SASL_DEFAULT_REALM, SaslRpcServer.SASL_PROPS, ch);
+      else[STATIC_SASL_SYMBOL]*/
+      SaslPropertiesResolver saslPropsResolver =
+          SaslPropertiesResolver.getInstance(new Configuration());
+      saslServer =
+          Sasl.createSaslServer(
+              SaslNettyServer.AuthMethod.DIGEST.getMechanismName(), null,
+              SaslRpcServer.SASL_DEFAULT_REALM,
+              saslPropsResolver.getDefaultProperties(), ch);
+      /*end[STATIC_SASL_SYMBOL]*/
     } catch (SaslException e) {
       LOG.error("SaslNettyServer: Could not create SaslServer: " + e);
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/0bcc76de/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2e3eb63..e654ff3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -930,7 +930,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>0.20.203.0</hadoop.version>
-        <munge.symbols>HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_1_SECURITY,HADOOP_1_SECRET_MANAGER</munge.symbols>
+        <munge.symbols>HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_1_SECURITY,HADOOP_1_SECRET_MANAGER,STATIC_SASL_SYMBOL</munge.symbols>
       </properties>
       <dependencies>
         <!-- sorted lexicographically -->
@@ -985,7 +985,7 @@ under the License.
       </activation>
       <properties>
         <hadoop.version>1.2.1</hadoop.version>
-        <munge.symbols>HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_1_SECURITY,HADOOP_1_SECRET_MANAGER</munge.symbols>
+        <munge.symbols>HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_1_SECURITY,HADOOP_1_SECRET_MANAGER,STATIC_SASL_SYMBOL</munge.symbols>
       </properties>
       <dependencies>
         <!-- sorted lexicographically -->
@@ -1036,7 +1036,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>0.20.2</hadoop.version>
-        <munge.symbols>HADOOP_NON_SECURE,HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_NON_COMMIT_JOB</munge.symbols>
+        <munge.symbols>HADOOP_NON_SECURE,HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_NON_COMMIT_JOB,STATIC_SASL_SYMBOL</munge.symbols>
       </properties>
       <dependencies>
         <!-- sorted lexicographically -->
@@ -1060,7 +1060,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>0.20.0</hadoop.version>
-        <munge.symbols>HADOOP_NON_SECURE,HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_JOB_ID_AVAILABLE</munge.symbols>
+        <munge.symbols>HADOOP_NON_SECURE,HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_JOB_ID_AVAILABLE,STATIC_SASL_SYMBOL</munge.symbols>
       </properties>
       <dependencies>
         <!-- sorted lexicographically -->
@@ -1091,7 +1091,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>0.23.1</hadoop.version>
-        <munge.symbols>HADOOP_1_SECRET_MANAGER</munge.symbols>
+        <munge.symbols>HADOOP_1_SECRET_MANAGER,STATIC_SASL_SYMBOL</munge.symbols>
         <!-- TODO: add these checks eventually -->
         <project.enforcer.skip>true</project.enforcer.skip>
         <giraph.maven.dependency.plugin.skip>true</giraph.maven.dependency.plugin.skip>
@@ -1125,7 +1125,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>2.0.0-cdh4.1.2</hadoop.version> 
-        <munge.symbols>HADOOP_1_SECRET_MANAGER</munge.symbols>
+        <munge.symbols>HADOOP_1_SECRET_MANAGER,STATIC_SASL_SYMBOL</munge.symbols>
         <!-- TODO: add these checks eventually -->
         <project.enforcer.skip>true</project.enforcer.skip>
         <giraph.maven.dependency.plugin.skip>true</giraph.maven.dependency.plugin.skip>
@@ -1191,7 +1191,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>SET_HADOOP_VERSION_USING_MVN_DASH_D_OPTION</hadoop.version>
-        <munge.symbols>PURE_YARN</munge.symbols>
+        <munge.symbols>PURE_YARN,STATIC_SASL_SYMBOL</munge.symbols>
         <!-- TODO: add these checks eventually -->
         <project.enforcer.skip>true</project.enforcer.skip>
         <giraph.maven.dependency.plugin.skip>true</giraph.maven.dependency.plugin.skip>
@@ -1240,9 +1240,6 @@ under the License.
       </dependencies>
     </profile>
 
-    <!-- Help keep future Hadoop versions munge-free:
-         All profiles below are munge-free: avoid introducing any munge
-         flags on any of the following profiles. -->
     <profile>
        <id>2.1.1-SNAPSHOT</id>
        <modules>
@@ -1250,6 +1247,7 @@ under the License.
        </modules>
        <properties>
          <hadoop.version>2.1.1-SNAPSHOT</hadoop.version>
+         <munge.symbols>STATIC_SASL_SYMBOL</munge.symbols>
          <!-- TODO: add these checks eventually -->
          <project.enforcer.skip>true</project.enforcer.skip>
          <giraph.maven.dependency.plugin.skip>true</giraph.maven.dependency.plugin.skip>
@@ -1284,6 +1282,9 @@ under the License.
        </dependencies>
      </profile>
 
+    <!-- Help keep future Hadoop versions munge-free:
+         All profiles below are munge-free: avoid introducing any munge
+         flags on any of the following profiles. -->
     <profile>
        <id>hadoop_2</id>
       <modules>
@@ -1296,7 +1297,7 @@ under the License.
         <module>giraph-dist</module>
       </modules>
        <properties>
-         <hadoop.version>2.2.0</hadoop.version>
+         <hadoop.version>2.5.1</hadoop.version>
          <!-- TODO: add these checks eventually -->
          <project.enforcer.skip>true</project.enforcer.skip>
          <giraph.maven.dependency.plugin.skip>true</giraph.maven.dependency.plugin.skip>


[37/47] Reduce/broadcast API

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java b/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
new file mode 100644
index 0000000..9f821b4
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
@@ -0,0 +1,110 @@
+/*
+ * 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.reducers;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Object responsible for performing reducing operation.
+ * Simple wrapper of ReduceOperation object and current value holding
+ * partially reduced result.
+ *
+ * @param <S> Single value type, objects passed on workers
+ * @param <R> Reduced value type
+ */
+public class Reducer<S, R extends Writable> implements Writable {
+  /** Reduce operations */
+  private ReduceOperation<S, R> reduceOp;
+  /** Current (partially) reduced value*/
+  private R currentValue;
+
+  /**
+   * Constructor
+   */
+  public Reducer() {
+  }
+  /**
+   * Constructor
+   * @param reduceOp Reduce operations
+   */
+  public Reducer(ReduceOperation<S, R> reduceOp) {
+    this.reduceOp = reduceOp;
+    this.currentValue = reduceOp.createInitialValue();
+  }
+  /**
+   * Constructor
+   * @param reduceOp Reduce operations
+   * @param currentValue current reduced value
+   */
+  public Reducer(ReduceOperation<S, R> reduceOp, R currentValue) {
+    this.reduceOp = reduceOp;
+    this.currentValue = currentValue;
+  }
+
+  /**
+   * Reduce given value into current reduced value.
+   * @param valueToReduce Single value to reduce
+   */
+  public void reduceSingle(S valueToReduce) {
+    reduceOp.reduceSingle(currentValue, valueToReduce);
+  }
+  /**
+   * Reduce given partially reduced value into current reduced value.
+   * @param valueToReduce Partial value to reduce
+   */
+  public void reducePartial(R valueToReduce) {
+    reduceOp.reducePartial(currentValue, valueToReduce);
+  }
+  /**
+   * Return new initial reduced value.
+   * @return New initial reduced value
+   */
+  public R createInitialValue() {
+    return reduceOp.createInitialValue();
+  }
+
+  public ReduceOperation<S, R> getReduceOp() {
+    return reduceOp;
+  }
+
+  public R getCurrentValue() {
+    return currentValue;
+  }
+
+  public void setCurrentValue(R currentValue) {
+    this.currentValue = currentValue;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeWritableObject(reduceOp, out);
+    currentValue.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    reduceOp = WritableUtils.readWritableObject(in, null);
+    currentValue = reduceOp.createInitialValue();
+    currentValue.readFields(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/reducers/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/package-info.java b/giraph-core/src/main/java/org/apache/giraph/reducers/package-info.java
new file mode 100644
index 0000000..eeefdeb
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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 of Giraph reducers.
+ */
+package org.apache.giraph.reducers;

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index 5e046cc..3d654b4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -17,8 +17,12 @@
  */
 package org.apache.giraph.utils;
 
-import com.google.common.base.Splitter;
-import com.google.common.collect.Iterables;
+import static org.apache.giraph.conf.GiraphConstants.COMPUTATION_CLASS;
+import static org.apache.giraph.conf.GiraphConstants.TYPES_HOLDER_CLASS;
+
+import java.io.IOException;
+import java.util.List;
+
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -29,6 +33,7 @@ import org.apache.giraph.Algorithm;
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConfigurationSettable;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.GiraphTypes;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
@@ -57,11 +62,8 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.ZooKeeper;
 
-import java.io.IOException;
-import java.util.List;
-
-import static org.apache.giraph.conf.GiraphConstants.COMPUTATION_CLASS;
-import static org.apache.giraph.conf.GiraphConstants.TYPES_HOLDER_CLASS;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
 
 /**
  * Translate command line args into Configuration Key-Value pairs.
@@ -147,6 +149,10 @@ public final class ConfigurationUtils {
       ImmutableClassesGiraphConfiguration configuration) {
     if (configuration != null) {
       configuration.configureIfPossible(object);
+    } else if (object instanceof GiraphConfigurationSettable) {
+      throw new IllegalArgumentException(
+          "Trying to configure configurable object without value, " +
+          object.getClass());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 3c5cbad..923d369 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -739,4 +739,38 @@ public class WritableUtils {
     }
   }
 
+  /**
+   * Create a copy of Writable object, by serializing and deserializing it.
+   *
+   * @param reusableOut Reusable output stream to serialize into
+   * @param reusableIn Reusable input stream to deserialize out of
+   * @param original Original value of which to make a copy
+   * @param <T> Type of the object
+   * @return Copy of the original value
+   */
+  public static <T extends Writable> T createCopy(
+      UnsafeByteArrayOutputStream reusableOut,
+      UnsafeReusableByteArrayInput reusableIn, T original) {
+    T copy = (T) createWritable(original.getClass(), null);
+
+    try {
+      reusableOut.reset();
+      original.write(reusableOut);
+      reusableIn.initialize(
+          reusableOut.getByteArray(), 0, reusableOut.getPos());
+      copy.readFields(reusableIn);
+
+      if (reusableIn.available() != 0) {
+        throw new RuntimeException("Serialization of " +
+            original.getClass() + " encountered issues, " +
+            reusableIn.available() + " bytes left to be read");
+      }
+    } catch (IOException e) {
+      throw new IllegalStateException(
+          "IOException occurred while trying to create a copy " +
+          original.getClass(), e);
+    }
+    return copy;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index 120678f..f61e817 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -18,6 +18,27 @@
 
 package org.apache.giraph.worker;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+
+import net.iharder.Base64;
+
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
@@ -66,10 +87,10 @@ import org.apache.giraph.partition.PartitionStore;
 import org.apache.giraph.partition.WorkerGraphPartitioner;
 import org.apache.giraph.utils.CallableFactory;
 import org.apache.giraph.utils.JMapHistoDumper;
-import org.apache.giraph.utils.ReactiveJMapHistoDumper;
 import org.apache.giraph.utils.LoggerUtils;
 import org.apache.giraph.utils.MemoryUtils;
 import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.giraph.utils.ReactiveJMapHistoDumper;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.giraph.zk.BspEvent;
 import org.apache.giraph.zk.PredicateLock;
@@ -96,26 +117,6 @@ import org.json.JSONObject;
 
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import net.iharder.Base64;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
 
 /**
  * ZooKeeper-based implementation of {@link CentralizedServiceWorker}.
@@ -162,10 +163,10 @@ public class BspServiceWorker<I extends WritableComparable,
   private final WorkerContext workerContext;
 
   /** Handler for aggregators */
-  private final WorkerAggregatorHandler aggregatorHandler;
+  private final WorkerAggregatorHandler globalCommHandler;
 
   /** Superstep output */
-  private SuperstepOutput<I, V, E> superstepOutput;
+  private final SuperstepOutput<I, V, E> superstepOutput;
 
   /** array of observers to call back to */
   private final WorkerObserver[] observers;
@@ -212,10 +213,10 @@ public class BspServiceWorker<I extends WritableComparable,
     workerAggregatorRequestProcessor =
         new NettyWorkerAggregatorRequestProcessor(getContext(), conf, this);
 
-    aggregatorHandler = new WorkerAggregatorHandler(this, conf, context);
+    globalCommHandler = new WorkerAggregatorHandler(this, conf, context);
 
     workerContext = conf.createWorkerContext();
-    workerContext.setWorkerAggregatorUsage(aggregatorHandler);
+    workerContext.setWorkerGlobalCommUsage(globalCommHandler);
 
     superstepOutput = conf.createSuperstepOutput(context);
 
@@ -584,7 +585,7 @@ public class BspServiceWorker<I extends WritableComparable,
 
     // Initialize aggregator at worker side during setup.
     // Do this just before vertex and edge loading.
-    aggregatorHandler.prepareSuperstep(workerAggregatorRequestProcessor);
+    globalCommHandler.prepareSuperstep(workerAggregatorRequestProcessor);
 
     VertexEdgeCount vertexEdgeCount;
     long entriesLoaded;
@@ -895,7 +896,7 @@ public class BspServiceWorker<I extends WritableComparable,
       postSuperstepCallbacks();
     }
 
-    aggregatorHandler.finishSuperstep(workerAggregatorRequestProcessor);
+    globalCommHandler.finishSuperstep(workerAggregatorRequestProcessor);
 
     MessageStore<I, Writable> incomingMessageStore =
         getServerData().getIncomingMessageStore();
@@ -1920,15 +1921,16 @@ else[HADOOP_NON_SECURE]*/
     return workerServer.getServerData();
   }
 
+
   @Override
   public WorkerAggregatorHandler getAggregatorHandler() {
-    return aggregatorHandler;
+    return globalCommHandler;
   }
 
   @Override
   public void prepareSuperstep() {
     if (getSuperstep() != INPUT_SUPERSTEP) {
-      aggregatorHandler.prepareSuperstep(workerAggregatorRequestProcessor);
+      globalCommHandler.prepareSuperstep(workerAggregatorRequestProcessor);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java b/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
index 35ad94b..89f74b3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
@@ -18,6 +18,8 @@
 
 package org.apache.giraph.worker;
 
+import java.io.IOException;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.VertexEdgeCount;
@@ -37,8 +39,6 @@ import org.apache.log4j.Logger;
 import com.yammer.metrics.core.Counter;
 import com.yammer.metrics.core.Meter;
 
-import java.io.IOException;
-
 /**
  * Load as many edge input splits as possible.
  * Every thread will has its own instance of WorkerClientRequestProcessor
@@ -62,7 +62,7 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
       EdgeInputSplitsCallable.class);
 
   /** Aggregator handler */
-  private final WorkerThreadAggregatorUsage aggregatorUsage;
+  private final WorkerThreadGlobalCommUsage globalCommUsage;
   /** Bsp service worker (only use thread-safe methods) */
   private final BspServiceWorker<I, V, E> bspServiceWorker;
   /** Edge input format */
@@ -105,7 +105,7 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
     this.bspServiceWorker = bspServiceWorker;
     inputSplitMaxEdges = configuration.getInputSplitMaxEdges();
     // Initialize aggregator usage.
-    this.aggregatorUsage = bspServiceWorker.getAggregatorHandler()
+    this.globalCommUsage = bspServiceWorker.getAggregatorHandler()
       .newThreadAggregatorUsage();
     edgeInputFilter = configuration.getEdgeInputFilter();
     canEmbedInIds = bspServiceWorker
@@ -147,7 +147,7 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
 
     edgeReader.initialize(inputSplit, context);
     // Set aggregator usage to edge reader
-    edgeReader.setWorkerAggregatorUse(aggregatorUsage);
+    edgeReader.setWorkerGlobalCommUsage(globalCommUsage);
 
     long inputSplitEdgesLoaded = 0;
     long inputSplitEdgesFiltered = 0;

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/MappingInputSplitsCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/MappingInputSplitsCallable.java b/giraph-core/src/main/java/org/apache/giraph/worker/MappingInputSplitsCallable.java
index a2279a9..f6dca25 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/MappingInputSplitsCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/MappingInputSplitsCallable.java
@@ -18,21 +18,21 @@
 
 package org.apache.giraph.worker;
 
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.io.GiraphInputFormat;
 import org.apache.giraph.io.MappingInputFormat;
 import org.apache.giraph.io.MappingReader;
-import org.apache.giraph.mapping.MappingStore;
 import org.apache.giraph.mapping.MappingEntry;
+import org.apache.giraph.mapping.MappingStore;
 import org.apache.giraph.zk.ZooKeeperExt;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Mapper;
 
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
-
 /**
  * Load as many mapping input splits as possible.
  * Every thread will has its own instance of WorkerClientRequestProcessor
@@ -89,11 +89,11 @@ public class MappingInputSplitsCallable<I extends WritableComparable,
         mappingInputFormat.createMappingReader(inputSplit, context);
     mappingReader.setConf(configuration);
 
-    WorkerThreadAggregatorUsage aggregatorUsage = this.bspServiceWorker
+    WorkerThreadGlobalCommUsage globalCommUsage = this.bspServiceWorker
         .getAggregatorHandler().newThreadAggregatorUsage();
 
     mappingReader.initialize(inputSplit, context);
-    mappingReader.setWorkerAggregatorUse(aggregatorUsage);
+    mappingReader.setWorkerGlobalCommUsage(globalCommUsage);
 
     int entriesLoaded = 0;
     MappingStore<I, B> mappingStore =

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java b/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
index 4c85765..00a2781 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
@@ -18,6 +18,8 @@
 
 package org.apache.giraph.worker;
 
+import java.io.IOException;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.OutEdges;
@@ -42,8 +44,6 @@ import org.apache.log4j.Logger;
 import com.yammer.metrics.core.Counter;
 import com.yammer.metrics.core.Meter;
 
-import java.io.IOException;
-
 /**
  * Load as many vertex input splits as possible.
  * Every thread will has its own instance of WorkerClientRequestProcessor
@@ -79,7 +79,7 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
    * Whether the chosen {@link OutEdges} implementation allows for Edge
    * reuse.
    */
-  private boolean reuseEdgeObjects;
+  private final boolean reuseEdgeObjects;
   /** Used to translate Edges during vertex input phase based on localData */
   private final TranslateEdge<I, E> translateEdge;
 
@@ -152,13 +152,13 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
         vertexInputFormat.createVertexReader(inputSplit, context);
     vertexReader.setConf(configuration);
 
-    WorkerThreadAggregatorUsage aggregatorUsage =
+    WorkerThreadGlobalCommUsage globalCommUsage =
       this.bspServiceWorker
         .getAggregatorHandler().newThreadAggregatorUsage();
 
     vertexReader.initialize(inputSplit, context);
     // Set aggregator usage to vertex reader
-    vertexReader.setWorkerAggregatorUse(aggregatorUsage);
+    vertexReader.setWorkerGlobalCommUsage(globalCommUsage);
 
     long inputSplitVerticesLoaded = 0;
     long inputSplitVerticesFiltered = 0;

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
new file mode 100644
index 0000000..5238a07
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
@@ -0,0 +1,69 @@
+/*
+ * 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.worker;
+
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Class for delegating WorkerAggregatorUsage and
+ * WorkerGlobalCommUsage methods to corresponding interface.
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ */
+public abstract class WorkerAggregatorDelegator<I extends WritableComparable,
+  V extends Writable, E extends Writable>
+  extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
+  implements WorkerAggregatorUsage, WorkerGlobalCommUsage {
+
+  /** Worker aggregator usage */
+  private WorkerGlobalCommUsage workerGlobalCommUsage;
+
+  /**
+   * Set worker global communication usage
+   *
+   * @param workerGlobalCommUsage Worker global communication usage
+   */
+  public void setWorkerGlobalCommUsage(
+      WorkerGlobalCommUsage workerGlobalCommUsage) {
+    this.workerGlobalCommUsage = workerGlobalCommUsage;
+  }
+
+  @Override
+  public final void reduce(String name, Object value) {
+    workerGlobalCommUsage.reduce(name, value);
+  }
+
+  @Override
+  public final <B extends Writable> B getBroadcast(String name) {
+    return workerGlobalCommUsage.getBroadcast(name);
+  }
+
+  @Override
+  public final <A extends Writable> void aggregate(String name, A value) {
+    reduce(name, value);
+  }
+
+  @Override
+  public final <A extends Writable> A getAggregatedValue(String name) {
+    return this.<A>getBroadcast(name);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
index 45ca665..05a13a7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
@@ -15,22 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.giraph.worker;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.comm.aggregators.AggregatedValueOutputStream;
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
+import org.apache.giraph.comm.aggregators.GlobalCommValueOutputStream;
 import org.apache.giraph.comm.aggregators.OwnerAggregatorServerData;
 import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.utils.Factory;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.reducers.Reducer;
+import org.apache.giraph.utils.UnsafeByteArrayOutputStream;
+import org.apache.giraph.utils.UnsafeReusableByteArrayInput;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -38,35 +42,18 @@ import org.apache.log4j.Logger;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-/**
- * Handler for aggregators on worker. Provides the aggregated values and
- * performs aggregations from user vertex code (thread-safe). Also has
- * methods for all superstep coordination related to aggregators.
- *
- * At the beginning of any superstep any worker calls prepareSuperstep(),
- * which blocks until the final aggregates from the previous superstep have
- * been delivered to the worker.
- * Next, during the superstep worker can call aggregate() and
- * getAggregatedValue() (both methods are thread safe) the former
- * computes partial aggregates for this superstep from the worker,
- * the latter returns (read-only) final aggregates from the previous superstep.
- * Finally, at the end of the superstep, the worker calls finishSuperstep(),
- * which propagates non-owned partial aggregates to the owner workers,
- * and sends the final aggregate from the owner worker to the master.
- */
-public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
+/** Handler for reduce/broadcast on the workers */
+public class WorkerAggregatorHandler implements WorkerThreadGlobalCommUsage {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(WorkerAggregatorHandler.class);
-  /** Map of values from previous superstep */
-  private final Map<String, Writable> previousAggregatedValueMap =
+  /** Map of broadcasted values */
+  private final Map<String, Writable> broadcastedMap =
       Maps.newHashMap();
-  /** Map of aggregator factories for current superstep */
-  private final Map<String, Factory<Aggregator<Writable>>>
-  currentAggregatorFactoryMap = Maps.newHashMap();
-  /** Map of aggregators for current superstep */
-  private final Map<String, Aggregator<Writable>> currentAggregatorMap =
+  /** Map of reducers currently being reduced */
+  private final Map<String, Reducer<Object, Writable>> reducerMap =
       Maps.newHashMap();
+
   /** Service worker */
   private final CentralizedServiceWorker<?, ?, ?> serviceWorker;
   /** Progressable for reporting progress */
@@ -96,29 +83,48 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
   }
 
   @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    Aggregator<Writable> aggregator = currentAggregatorMap.get(name);
-    if (aggregator != null) {
+  public <B extends Writable> B getBroadcast(String name) {
+    B value = (B) broadcastedMap.get(name);
+    if (value == null) {
+      LOG.warn("getBroadcast: " +
+          AggregatorUtils.getUnregisteredAggregatorMessage(name,
+              broadcastedMap.size() != 0, conf));
+    }
+    return value;
+  }
+
+  @Override
+  public void reduce(String name, Object value) {
+    Reducer<Object, Writable> reducer = reducerMap.get(name);
+    if (reducer != null) {
       progressable.progress();
-      synchronized (aggregator) {
-        aggregator.aggregate(value);
+      synchronized (reducer) {
+        reducer.reduceSingle(value);
       }
     } else {
-      throw new IllegalStateException("aggregate: " +
+      throw new IllegalStateException("reduce: " +
           AggregatorUtils.getUnregisteredAggregatorMessage(name,
-              currentAggregatorMap.size() != 0, conf));
+              reducerMap.size() != 0, conf));
     }
   }
 
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    A value = (A) previousAggregatedValueMap.get(name);
-    if (value == null) {
-      LOG.warn("getAggregatedValue: " +
+  /**
+   * Combine partially reduced value into currently reduced value.
+   * @param name Name of the reducer
+   * @param valueToReduce Partial value to reduce
+   */
+  protected void reducePartial(String name, Writable valueToReduce) {
+    Reducer<Object, Writable> reducer = reducerMap.get(name);
+    if (reducer != null) {
+      progressable.progress();
+      synchronized (reducer) {
+        reducer.reducePartial(valueToReduce);
+      }
+    } else {
+      throw new IllegalStateException("reduce: " +
           AggregatorUtils.getUnregisteredAggregatorMessage(name,
-              previousAggregatedValueMap.size() != 0, conf));
+              reducerMap.size() != 0, conf));
     }
-    return value;
   }
 
   /**
@@ -128,53 +134,35 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
    */
   public void prepareSuperstep(
       WorkerAggregatorRequestProcessor requestProcessor) {
+    broadcastedMap.clear();
+    reducerMap.clear();
+
     if (LOG.isDebugEnabled()) {
       LOG.debug("prepareSuperstep: Start preparing aggregators");
     }
-    AllAggregatorServerData allAggregatorData =
+    AllAggregatorServerData allGlobalCommData =
         serviceWorker.getServerData().getAllAggregatorData();
     // Wait for my aggregators
     Iterable<byte[]> dataToDistribute =
-        allAggregatorData.getDataFromMasterWhenReady(
+        allGlobalCommData.getDataFromMasterWhenReady(
             serviceWorker.getMasterInfo());
     try {
       // Distribute my aggregators
-      requestProcessor.distributeAggregators(dataToDistribute);
+      requestProcessor.distributeReducedValues(dataToDistribute);
     } catch (IOException e) {
       throw new IllegalStateException("prepareSuperstep: " +
           "IOException occurred while trying to distribute aggregators", e);
     }
     // Wait for all other aggregators and store them
-    allAggregatorData.fillNextSuperstepMapsWhenReady(
-        getOtherWorkerIdsSet(), previousAggregatedValueMap,
-        currentAggregatorFactoryMap);
-    fillAndInitAggregatorsMap(currentAggregatorMap);
-    allAggregatorData.reset();
+    allGlobalCommData.fillNextSuperstepMapsWhenReady(
+        getOtherWorkerIdsSet(), broadcastedMap,
+        reducerMap);
     if (LOG.isDebugEnabled()) {
       LOG.debug("prepareSuperstep: Aggregators prepared");
     }
   }
 
   /**
-   * Fills aggregators map from currentAggregatorFactoryMap.
-   * All aggregators in this map will be set to initial value.
-   * @param aggregatorMap Map to fill.
-   */
-  private void fillAndInitAggregatorsMap(
-      Map<String, Aggregator<Writable>> aggregatorMap) {
-    for (Map.Entry<String, Factory<Aggregator<Writable>>> entry :
-        currentAggregatorFactoryMap.entrySet()) {
-      Aggregator<Writable> aggregator =
-          aggregatorMap.get(entry.getKey());
-      if (aggregator == null) {
-        aggregatorMap.put(entry.getKey(), entry.getValue().create());
-      } else {
-        aggregator.reset();
-      }
-    }
-  }
-
-  /**
    * Send aggregators to their owners and in the end to the master
    *
    * @param requestProcessor Request processor for aggregators
@@ -186,19 +174,19 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
           "workers will send their aggregated values " +
           "once they are done with superstep computation");
     }
-    OwnerAggregatorServerData ownerAggregatorData =
+    OwnerAggregatorServerData ownerGlobalCommData =
         serviceWorker.getServerData().getOwnerAggregatorData();
     // First send partial aggregated values to their owners and determine
     // which aggregators belong to this worker
-    for (Map.Entry<String, Aggregator<Writable>> entry :
-        currentAggregatorMap.entrySet()) {
+    for (Map.Entry<String, Reducer<Object, Writable>> entry :
+        reducerMap.entrySet()) {
       try {
-        boolean sent = requestProcessor.sendAggregatedValue(entry.getKey(),
-            entry.getValue().getAggregatedValue());
+        boolean sent = requestProcessor.sendReducedValue(entry.getKey(),
+            entry.getValue().getCurrentValue());
         if (!sent) {
           // If it's my aggregator, add it directly
-          ownerAggregatorData.aggregate(entry.getKey(),
-              entry.getValue().getAggregatedValue());
+          ownerGlobalCommData.reduce(entry.getKey(),
+              entry.getValue().getCurrentValue());
         }
       } catch (IOException e) {
         throw new IllegalStateException("finishSuperstep: " +
@@ -216,20 +204,21 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
     }
 
     // Wait to receive partial aggregated values from all other workers
-    Iterable<Map.Entry<String, Writable>> myAggregators =
-        ownerAggregatorData.getMyAggregatorValuesWhenReady(
+    Iterable<Map.Entry<String, Writable>> myReducedValues =
+        ownerGlobalCommData.getMyReducedValuesWhenReady(
             getOtherWorkerIdsSet());
 
     // Send final aggregated values to master
-    AggregatedValueOutputStream aggregatorOutput =
-        new AggregatedValueOutputStream();
-    for (Map.Entry<String, Writable> entry : myAggregators) {
+    GlobalCommValueOutputStream globalOutput =
+        new GlobalCommValueOutputStream(false);
+    for (Map.Entry<String, Writable> entry : myReducedValues) {
       try {
-        int currentSize = aggregatorOutput.addAggregator(entry.getKey(),
+        int currentSize = globalOutput.addValue(entry.getKey(),
+            GlobalCommType.REDUCED_VALUE,
             entry.getValue());
         if (currentSize > maxBytesPerAggregatorRequest) {
-          requestProcessor.sendAggregatedValuesToMaster(
-              aggregatorOutput.flush());
+          requestProcessor.sendReducedValuesToMaster(
+              globalOutput.flush());
         }
         progressable.progress();
       } catch (IOException e) {
@@ -239,7 +228,7 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
       }
     }
     try {
-      requestProcessor.sendAggregatedValuesToMaster(aggregatorOutput.flush());
+      requestProcessor.sendReducedValuesToMaster(globalOutput.flush());
     } catch (IOException e) {
       throw new IllegalStateException("finishSuperstep: " +
           "IOException occured while sending aggregators to master", e);
@@ -247,7 +236,7 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
     // Wait for master to receive aggregated values before proceeding
     serviceWorker.getWorkerClient().waitAllRequests();
 
-    ownerAggregatorData.reset();
+    ownerGlobalCommData.reset();
     if (LOG.isDebugEnabled()) {
       LOG.debug("finishSuperstep: Aggregators finished");
     }
@@ -259,9 +248,9 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
    *
    * @return New aggregator usage
    */
-  public WorkerThreadAggregatorUsage newThreadAggregatorUsage() {
+  public WorkerThreadGlobalCommUsage newThreadAggregatorUsage() {
     if (AggregatorUtils.useThreadLocalAggregators(conf)) {
-      return new ThreadLocalWorkerAggregatorUsage();
+      return new ThreadLocalWorkerGlobalCommUsage();
     } else {
       return this;
     }
@@ -290,56 +279,70 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
   }
 
   /**
-   * Not thread-safe implementation of {@link WorkerThreadAggregatorUsage}.
-   * We can use one instance of this object per thread to prevent
-   * synchronizing on each aggregate() call. In the end of superstep,
-   * values from each of these will be aggregated back to {@link
-   * WorkerAggregatorHandler}
-   */
-  public class ThreadLocalWorkerAggregatorUsage
-      implements WorkerThreadAggregatorUsage {
-    /** Thread-local aggregator map */
-    private final Map<String, Aggregator<Writable>> threadAggregatorMap;
+  * Not thread-safe implementation of {@link WorkerThreadGlobalCommUsage}.
+  * We can use one instance of this object per thread to prevent
+  * synchronizing on each aggregate() call. In the end of superstep,
+  * values from each of these will be aggregated back to {@link
+  * WorkerThreadGlobalCommUsage}
+  */
+  public class ThreadLocalWorkerGlobalCommUsage
+    implements WorkerThreadGlobalCommUsage {
+    /** Thread-local reducer map */
+    private final Map<String, Reducer<Object, Writable>> threadReducerMap;
 
     /**
-     * Constructor
-     *
-     * Creates new instances of all aggregators from
-     * {@link WorkerAggregatorHandler}
-     */
-    public ThreadLocalWorkerAggregatorUsage() {
-      threadAggregatorMap = Maps.newHashMapWithExpectedSize(
-          WorkerAggregatorHandler.this.currentAggregatorMap.size());
-      fillAndInitAggregatorsMap(threadAggregatorMap);
+    * Constructor
+    *
+    * Creates new instances of all reducers from
+    * {@link WorkerAggregatorHandler}
+    */
+    public ThreadLocalWorkerGlobalCommUsage() {
+      threadReducerMap = Maps.newHashMapWithExpectedSize(
+          WorkerAggregatorHandler.this.reducerMap.size());
+
+      UnsafeByteArrayOutputStream out = new UnsafeByteArrayOutputStream();
+      UnsafeReusableByteArrayInput in = new UnsafeReusableByteArrayInput();
+
+      for (Entry<String, Reducer<Object, Writable>> entry :
+          reducerMap.entrySet()) {
+        ReduceOperation<Object, Writable> globalReduceOp =
+            entry.getValue().getReduceOp();
+
+        ReduceOperation<Object, Writable> threadLocalCopy =
+            WritableUtils.createCopy(out, in, globalReduceOp);
+
+        threadReducerMap.put(entry.getKey(), new Reducer<>(threadLocalCopy));
+      }
     }
 
     @Override
-    public <A extends Writable> void aggregate(String name, A value) {
-      Aggregator<Writable> aggregator = threadAggregatorMap.get(name);
-      if (aggregator != null) {
+    public void reduce(String name, Object value) {
+      Reducer<Object, Writable> reducer = threadReducerMap.get(name);
+      if (reducer != null) {
         progressable.progress();
-        aggregator.aggregate(value);
+        reducer.reduceSingle(value);
       } else {
-        throw new IllegalStateException("aggregate: " +
+        throw new IllegalStateException("reduce: " +
             AggregatorUtils.getUnregisteredAggregatorMessage(name,
-                threadAggregatorMap.size() != 0, conf));
+                threadReducerMap.size() != 0, conf));
       }
     }
 
     @Override
-    public <A extends Writable> A getAggregatedValue(String name) {
-      return WorkerAggregatorHandler.this.<A>getAggregatedValue(name);
+    public <B extends Writable> B getBroadcast(String name) {
+      return WorkerAggregatorHandler.this.getBroadcast(name);
     }
 
     @Override
     public void finishThreadComputation() {
       // Aggregate the values this thread's vertices provided back to
       // WorkerAggregatorHandler
-      for (Map.Entry<String, Aggregator<Writable>> entry :
-          threadAggregatorMap.entrySet()) {
-        WorkerAggregatorHandler.this.aggregate(entry.getKey(),
-            entry.getValue().getAggregatedValue());
+      for (Entry<String, Reducer<Object, Writable>> entry :
+          threadReducerMap.entrySet()) {
+        WorkerAggregatorHandler.this.reducePartial(entry.getKey(),
+            entry.getValue().getCurrentValue());
       }
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
index 7a55d56..b977ea1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
@@ -18,30 +18,28 @@
 
 package org.apache.giraph.worker;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.requests.SendWorkerToWorkerMessageRequest;
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.GraphState;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
 /**
  * WorkerContext allows for the execution of user code
  * on a per-worker basis. There's one WorkerContext per worker.
  */
 @SuppressWarnings("rawtypes")
 public abstract class WorkerContext
-  extends DefaultImmutableClassesGiraphConfigurable
-  implements WorkerAggregatorUsage, Writable {
+  extends WorkerAggregatorDelegator<WritableComparable, Writable, Writable>
+  implements Writable {
   /** Global graph state */
   private GraphState graphState;
-  /** Worker aggregator usage */
-  private WorkerAggregatorUsage workerAggregatorUsage;
 
   /** Service worker */
   private CentralizedServiceWorker serviceWorker;
@@ -71,16 +69,6 @@ public abstract class WorkerContext
   }
 
   /**
-   * Set worker aggregator usage
-   *
-   * @param workerAggregatorUsage Worker aggregator usage
-   */
-  public void setWorkerAggregatorUsage(
-      WorkerAggregatorUsage workerAggregatorUsage) {
-    this.workerAggregatorUsage = workerAggregatorUsage;
-  }
-
-  /**
    * Initialize the WorkerContext.
    * This method is executed once on each Worker before the first
    * superstep starts.
@@ -196,16 +184,6 @@ public abstract class WorkerContext
     return graphState.getContext();
   }
 
-  @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    workerAggregatorUsage.aggregate(name, value);
-  }
-
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return workerAggregatorUsage.<A>getAggregatedValue(name);
-  }
-
   /**
    * Call this to log a line to command line of the job. Use in moderation -
    * it's a synchronous call to Job client

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
new file mode 100644
index 0000000..39566f5
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
@@ -0,0 +1,40 @@
+/*
+ * 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.worker;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Methods on worker can access broadcasted values and provide
+ * values to reduce through this interface
+ */
+public interface WorkerGlobalCommUsage {
+  /**
+   * Reduce given value.
+   * @param name Name of the reducer
+   * @param value Single value to reduce
+   */
+  void reduce(String name, Object value);
+  /**
+   * Get value broadcasted from master
+   * @param name Name of the broadcasted value
+   * @return Broadcasted value
+   * @param <B> Broadcast value type
+   */
+  <B extends Writable> B getBroadcast(String name);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadAggregatorUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadAggregatorUsage.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadAggregatorUsage.java
deleted file mode 100644
index 194127e..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadAggregatorUsage.java
+++ /dev/null
@@ -1,31 +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.worker;
-
-/**
- * {@link WorkerAggregatorUsage} which can be used in each of the
- * computation threads.
- */
-public interface WorkerThreadAggregatorUsage extends WorkerAggregatorUsage {
-  /**
-   * Call this after thread's computation is finished,
-   * i.e. when all vertices have provided their values to aggregators
-   */
-  void finishThreadComputation();
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadGlobalCommUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadGlobalCommUsage.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadGlobalCommUsage.java
new file mode 100644
index 0000000..8edbdc7
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerThreadGlobalCommUsage.java
@@ -0,0 +1,32 @@
+/*
+ * 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.worker;
+
+
+/**
+ * {@link WorkerAggregatorUsage} which can be used in each of the
+ * computation threads.
+ */
+public interface WorkerThreadGlobalCommUsage extends WorkerGlobalCommUsage {
+  /**
+   * Call this after thread's computation is finished,
+   * i.e. when all vertices have provided their values to aggregators
+   */
+  void finishThreadComputation();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
index 488e1ea..26459c0 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
@@ -18,6 +18,24 @@
 
 package org.apache.giraph;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.giraph.aggregators.TextAggregatorWriter;
 import org.apache.giraph.combiner.SimpleSumMessageCombiner;
 import org.apache.giraph.conf.GiraphConfiguration;
@@ -62,24 +80,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
 
-import java.io.BufferedReader;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.lang.reflect.InvocationTargetException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 /**
  * Unit test for many simple BSP applications.
  */
@@ -456,8 +456,8 @@ public class
           assertEquals(maxSuperstep + 2, maxValues.size());
           assertEquals(maxSuperstep + 2, vertexCounts.size());
 
-          assertEquals(maxPageRank, (double) maxValues.get(maxSuperstep), 0d);
-          assertEquals(minPageRank, (double) minValues.get(maxSuperstep), 0d);
+          assertEquals(maxPageRank, maxValues.get(maxSuperstep), 0d);
+          assertEquals(minPageRank, minValues.get(maxSuperstep), 0d);
           assertEquals(numVertices, (long) vertexCounts.get(maxSuperstep));
 
         } finally {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java b/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
index eb3f686..602ab32 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
@@ -18,34 +18,19 @@
 
 package org.apache.giraph.aggregators;
 
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
 import org.apache.giraph.BspCase;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.examples.AggregatorsTestComputation;
 import org.apache.giraph.examples.SimpleCheckpoint;
 import org.apache.giraph.job.GiraphJob;
-import org.apache.giraph.master.MasterAggregatorHandler;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.Progressable;
 import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 /** Tests if aggregators are handled on a proper way */
 public class TestAggregatorsHandling extends BspCase {
@@ -54,21 +39,6 @@ public class TestAggregatorsHandling extends BspCase {
     super(TestAggregatorsHandling.class.getName());
   }
 
-  private Map<String, AggregatorWrapper<Writable>> getAggregatorMap
-      (MasterAggregatorHandler aggregatorHandler) {
-    try {
-      Field aggregtorMapField = aggregatorHandler.getClass().getDeclaredField
-          ("aggregatorMap");
-      aggregtorMapField.setAccessible(true);
-      return (Map<String, AggregatorWrapper<Writable>>)
-          aggregtorMapField.get(aggregatorHandler);
-    } catch (IllegalAccessException e) {
-      throw new IllegalStateException(e);
-    } catch (NoSuchFieldException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
   /** Tests if aggregators are handled on a proper way during supersteps */
   @Test
   public void testAggregatorsHandling() throws IOException,
@@ -88,64 +58,6 @@ public class TestAggregatorsHandling extends BspCase {
     assertTrue(job.run(true));
   }
 
-  /** Test if aggregators serialization captures everything */
-  @Test
-  public void testMasterAggregatorsSerialization() throws
-      IllegalAccessException, InstantiationException, IOException {
-    ImmutableClassesGiraphConfiguration conf =
-        Mockito.mock(ImmutableClassesGiraphConfiguration.class);
-    Mockito.when(conf.getAggregatorWriterClass()).thenReturn(
-        TextAggregatorWriter.class);
-    Progressable progressable = Mockito.mock(Progressable.class);
-    MasterAggregatorHandler handler =
-        new MasterAggregatorHandler(conf, progressable);
-
-    String regularAggName = "regular";
-    LongWritable regularValue = new LongWritable(5);
-    handler.registerAggregator(regularAggName, LongSumAggregator.class);
-    handler.setAggregatedValue(regularAggName, regularValue);
-
-    String persistentAggName = "persistent";
-    DoubleWritable persistentValue = new DoubleWritable(10.5);
-    handler.registerPersistentAggregator(persistentAggName,
-        DoubleOverwriteAggregator.class);
-    handler.setAggregatedValue(persistentAggName, persistentValue);
-
-    for (AggregatorWrapper<Writable> aggregator :
-        getAggregatorMap(handler).values()) {
-      aggregator.setPreviousAggregatedValue(
-          aggregator.getCurrentAggregatedValue());
-    }
-
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    handler.write(new DataOutputStream(out));
-
-    MasterAggregatorHandler restartedHandler =
-        new MasterAggregatorHandler(conf, progressable);
-    restartedHandler.readFields(
-        new DataInputStream(new ByteArrayInputStream(out.toByteArray())));
-
-    assertEquals(2, getAggregatorMap(restartedHandler).size());
-
-    AggregatorWrapper<Writable> regularAgg =
-        getAggregatorMap(restartedHandler).get(regularAggName);
-    assertTrue(regularAgg.getAggregatorFactory().create().getClass().equals(
-        LongSumAggregator.class));
-    assertEquals(regularValue, regularAgg.getPreviousAggregatedValue());
-    assertEquals(regularValue,
-        restartedHandler.<LongWritable>getAggregatedValue(regularAggName));
-    assertFalse(regularAgg.isPersistent());
-
-    AggregatorWrapper<Writable> persistentAgg =
-        getAggregatorMap(restartedHandler).get(persistentAggName);
-    assertTrue(persistentAgg.getAggregatorFactory().create().getClass().equals
-        (DoubleOverwriteAggregator.class));
-    assertEquals(persistentValue, persistentAgg.getPreviousAggregatedValue());
-    assertEquals(persistentValue,
-        restartedHandler.<LongWritable>getAggregatedValue(persistentAggName));
-    assertTrue(persistentAgg.isPersistent());
-  }
-
   /**
    * Test if aggregators are are handled properly when restarting from a
    * checkpoint


[16/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-924: Fix checkpointing (edunov via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/02d9e6c2
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/02d9e6c2
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/02d9e6c2

Branch: refs/heads/release-1.1
Commit: 02d9e6c2533a3cd108f5e6feaf40f26e95deb64c
Parents: 0a90177
Author: Maja Kabiljo <ma...@fb.com>
Authored: Wed Jul 16 10:30:30 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Wed Jul 16 10:30:30 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../giraph/aggregators/AggregatorWrapper.java   |   6 +-
 .../java/org/apache/giraph/bsp/BspService.java  |  24 ++
 .../java/org/apache/giraph/comm/ServerData.java |  17 +
 .../org/apache/giraph/conf/GiraphConstants.java |  19 +
 .../apache/giraph/master/BspServiceMaster.java  | 179 ++++----
 .../giraph/partition/BasicPartitionOwner.java   |  10 -
 .../giraph/partition/HashMasterPartitioner.java |   6 +
 .../giraph/partition/HashWorkerPartitioner.java |   5 +-
 .../partition/MasterGraphPartitioner.java       |   7 +
 .../giraph/partition/PartitionBalancer.java     |   4 +-
 .../apache/giraph/partition/PartitionOwner.java |  16 -
 .../partition/SimpleMasterPartitioner.java      |   6 +
 .../partition/SimpleWorkerPartitioner.java      |   6 +-
 .../partition/WorkerGraphPartitioner.java       |   5 +-
 .../giraph/utils/InternalVertexRunner.java      | 260 ++++++++----
 .../utils/io/ExtendedDataInputOutput.java       |   2 +-
 .../apache/giraph/worker/BspServiceWorker.java  | 404 ++++++++++++-------
 .../org/apache/giraph/worker/WorkerContext.java |  16 +-
 .../SimpleRangePartitionFactoryTest.java        |   2 +-
 .../org/apache/giraph/TestCheckpointing.java    | 266 ++++++++++++
 21 files changed, 882 insertions(+), 380 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 7287490..4207339 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-924: Fix checkpointing (edunov via majakabiljo)
+
   GIRAPH-921: Create ByteValueVertex to store vertex values as bytes without object instance (akyrola via majakabiljo)
 
   GIRAPH-929: setIfUnset for EnumConfOption (pavanka)  

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
index 9613805..7150402 100644
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
@@ -44,13 +44,9 @@ public class AggregatorWrapper<A extends Writable> {
    * @param persistent      False iff aggregator should be reset at the end of
    *                        each super step
    * @param conf            Configuration
-   * @throws IllegalAccessException
-   * @throws InstantiationException
    */
   public AggregatorWrapper(Class<? extends Aggregator<A>> aggregatorClass,
-      boolean persistent, ImmutableClassesGiraphConfiguration conf) throws
-      IllegalAccessException,
-      InstantiationException {
+      boolean persistent, ImmutableClassesGiraphConfiguration conf) {
     this.persistent = persistent;
     currentAggregator = ReflectionUtils.newInstance(aggregatorClass, conf);
     changed = false;

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
----------------------------------------------------------------------
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 2e35373..02577b9 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
@@ -54,6 +54,7 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.apache.giraph.conf.GiraphConstants.CHECKPOINT_DIRECTORY;
+import static org.apache.giraph.conf.GiraphConstants.RESTART_JOB_ID;
 
 /**
  * Zookeeper-based implementation of {@link CentralizedService}.
@@ -198,6 +199,11 @@ public abstract class BspService<I extends WritableComparable,
    */
   public static final String CHECKPOINT_VALID_POSTFIX = ".valid";
   /**
+   * If at the end of a checkpoint file,
+   * indicates that we store WorkerContext and aggregator handler data.
+   */
+  public static final String CHECKPOINT_DATA_POSTFIX = ".data";
+  /**
    * If at the end of a checkpoint file, indicates the stitched checkpoint
    * file prefixes.  A checkpoint is not valid if this file does not exist.
    */
@@ -226,6 +232,8 @@ public abstract class BspService<I extends WritableComparable,
   protected final String cleanedUpPath;
   /** Path to the checkpoint's root (including job id) */
   protected final String checkpointBasePath;
+  /** Old checkpoint in case we want to restart some job */
+  protected final String savedCheckpointBasePath;
   /** Path to the master election path */
   protected final String masterElectionPath;
   /** Stores progress info of this worker */
@@ -350,6 +358,12 @@ public abstract class BspService<I extends WritableComparable,
         EDGE_INPUT_SPLITS_ALL_READY_NODE, EDGE_INPUT_SPLITS_ALL_DONE_NODE);
     applicationAttemptsPath = basePath + APPLICATION_ATTEMPTS_DIR;
     cleanedUpPath = basePath + CLEANED_UP_DIR;
+
+    String restartJobId = RESTART_JOB_ID.get(conf);
+    savedCheckpointBasePath =
+        CHECKPOINT_DIRECTORY.getWithDefault(getConfiguration(),
+            CHECKPOINT_DIRECTORY.getDefaultValue() + "/" +
+                (restartJobId == null ? getJobId() : restartJobId));
     checkpointBasePath =
         CHECKPOINT_DIRECTORY.getWithDefault(getConfiguration(),
             CHECKPOINT_DIRECTORY.getDefaultValue() + "/" + getJobId());
@@ -575,6 +589,16 @@ public abstract class BspService<I extends WritableComparable,
   }
 
   /**
+   * In case when we restart another job this will give us a path
+   * to saved checkpoint.
+   * @param superstep superstep to use
+   * @return Direcory path for restarted job based on the superstep
+   */
+  public final String getSavedCheckpointBasePath(long superstep) {
+    return savedCheckpointBasePath + "/" + superstep;
+  }
+
+  /**
    * Get the checkpoint from a finalized checkpoint path
    *
    * @param finalizedPath Path of the finalized checkpoint

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
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 85bfe04..036510e 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
@@ -161,6 +161,23 @@ public class ServerData<I extends WritableComparable,
     return (MessageStore<I, M>) currentMessageStore;
   }
 
+  /**
+   * Re-initialize message stores.
+   * Discards old values if any.
+   * @throws IOException
+   */
+  public void resetMessageStores() throws IOException {
+    if (currentMessageStore != null) {
+      currentMessageStore.clearAll();
+      currentMessageStore = null;
+    }
+    if (incomingMessageStore != null) {
+      incomingMessageStore.clearAll();
+      incomingMessageStore = null;
+    }
+    prepareSuperstep();
+  }
+
   /** Prepare for next super step */
   public void prepareSuperstep() {
     if (currentMessageStore != null) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 81c0e0b..3d16e9c 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
@@ -880,6 +880,13 @@ public interface GiraphConstants {
   String RESTART_SUPERSTEP = "giraph.restartSuperstep";
 
   /**
+   * If application is restarted manually we need to specify job ID
+   * to restart from.
+   */
+  StrConfOption RESTART_JOB_ID = new StrConfOption("giraph.restart.jobId",
+      null, "Which job ID should I try to restart?");
+
+  /**
    * Base ZNode for Giraph's state in the ZooKeeper cluster.  Must be a root
    * znode on the cluster beginning with "/"
    */
@@ -1124,5 +1131,17 @@ public interface GiraphConstants {
   IntConfOption HDFS_FILE_CREATION_RETRY_WAIT_MS =
       new IntConfOption("giraph.hdfs.file.creation.retry.wait.ms", 30_000,
           "Milliseconds to wait prior to retrying creation of an HDFS file");
+
+  /** Number of threads for writing and reading checkpoints */
+  IntConfOption NUM_CHECKPOINT_IO_THREADS =
+      new IntConfOption("giraph.checkpoint.io.threads", 8,
+          "Number of threads for writing and reading checkpoints");
+
+  /** Compression algorithm to be used for checkpointing */
+  StrConfOption CHECKPOINT_COMPRESSION_CODEC =
+      new StrConfOption("giraph.checkpoint.compression.codec",
+          "org.apache.hadoop.io.compress.DefaultCodec",
+          "Defines compression algorithm we will be using for " +
+              "storing checkpoint");
 }
 // CHECKSTYLE: resume InterfaceIsTypeCheck

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index 0275395..e129390 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -18,6 +18,8 @@
 
 package org.apache.giraph.master;
 
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.BspInputFormat;
@@ -42,6 +44,7 @@ import org.apache.giraph.io.GiraphInputFormat;
 import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.io.MappingInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
+import org.apache.giraph.partition.BasicPartitionOwner;
 import org.apache.giraph.partition.MasterGraphPartitioner;
 import org.apache.giraph.partition.PartitionOwner;
 import org.apache.giraph.partition.PartitionStats;
@@ -100,10 +103,8 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.Callable;
@@ -272,6 +273,7 @@ public class BspServiceMaster<I extends WritableComparable,
           Ids.OPEN_ACL_UNSAFE,
           CreateMode.PERSISTENT_SEQUENTIAL,
           true);
+      LOG.info("setJobState: " + jobState);
     } catch (KeeperException.NodeExistsException e) {
       throw new IllegalStateException(
           "setJobState: Imposible that " +
@@ -740,20 +742,18 @@ public class BspServiceMaster<I extends WritableComparable,
    * finalized checkpoint file and setting it.
    *
    * @param superstep Checkpoint set to examine.
-   * @param partitionOwners Partition owners to modify with checkpoint
-   *        prefixes
    * @throws IOException
    * @throws InterruptedException
    * @throws KeeperException
+   * @return Collection of generated partition owners.
    */
-  private void prepareCheckpointRestart(
-    long superstep,
-    Collection<PartitionOwner> partitionOwners)
+  private Collection<PartitionOwner> prepareCheckpointRestart(long superstep)
     throws IOException, KeeperException, InterruptedException {
+    List<PartitionOwner> partitionOwners = new ArrayList<>();
     FileSystem fs = getFs();
-    List<Path> validMetadataPathList = new ArrayList<Path>();
     String finalizedCheckpointPath =
-        getCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
+        getSavedCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
+    LOG.info("Loading checkpoint from " + finalizedCheckpointPath);
     DataInputStream finalizedStream =
         fs.open(new Path(finalizedCheckpointPath));
     GlobalStats globalStats = new GlobalStats();
@@ -763,51 +763,46 @@ public class BspServiceMaster<I extends WritableComparable,
     superstepClasses.readFields(finalizedStream);
     getConfiguration().updateSuperstepClasses(superstepClasses);
     int prefixFileCount = finalizedStream.readInt();
-    for (int i = 0; i < prefixFileCount; ++i) {
-      String metadataFilePath =
-          finalizedStream.readUTF() + CHECKPOINT_METADATA_POSTFIX;
-      validMetadataPathList.add(new Path(metadataFilePath));
-    }
 
-    aggregatorHandler.readFields(finalizedStream);
-    masterCompute.readFields(finalizedStream);
-    finalizedStream.close();
 
-    Map<Integer, PartitionOwner> idOwnerMap =
-        new HashMap<Integer, PartitionOwner>();
-    for (PartitionOwner partitionOwner : partitionOwners) {
-      if (idOwnerMap.put(partitionOwner.getPartitionId(),
-          partitionOwner) != null) {
-        throw new IllegalStateException(
-            "prepareCheckpointRestart: Duplicate partition " +
-                partitionOwner);
-      }
+    Int2ObjectMap<WorkerInfo> workersMap = new Int2ObjectOpenHashMap<>();
+    for (WorkerInfo worker : chosenWorkerInfoList) {
+      workersMap.put(worker.getTaskId(), worker);
     }
-    // Reading the metadata files.  Simply assign each partition owner
-    // the correct file prefix based on the partition id.
-    for (Path metadataPath : validMetadataPathList) {
-      String checkpointFilePrefix = metadataPath.toString();
-      checkpointFilePrefix =
-          checkpointFilePrefix.substring(
-              0,
-              checkpointFilePrefix.length() -
-              CHECKPOINT_METADATA_POSTFIX.length());
-      DataInputStream metadataStream = fs.open(metadataPath);
+    String checkpointFile =
+        finalizedStream.readUTF();
+    for (int i = 0; i < prefixFileCount; ++i) {
+      int mrTaskId = finalizedStream.readInt();
+
+      DataInputStream metadataStream = fs.open(new Path(checkpointFile +
+          "." + mrTaskId + CHECKPOINT_METADATA_POSTFIX));
       long partitions = metadataStream.readInt();
-      for (long i = 0; i < partitions; ++i) {
-        long dataPos = metadataStream.readLong();
+      WorkerInfo worker = workersMap.get(mrTaskId);
+      for (long p = 0; p < partitions; ++p) {
         int partitionId = metadataStream.readInt();
-        PartitionOwner partitionOwner = idOwnerMap.get(partitionId);
-        if (LOG.isInfoEnabled()) {
-          LOG.info("prepareSuperstepRestart: File " + metadataPath +
-              " with position " + dataPos +
-              ", partition id = " + partitionId +
-              " assigned to " + partitionOwner);
-        }
-        partitionOwner.setCheckpointFilesPrefix(checkpointFilePrefix);
+        PartitionOwner partitionOwner = new BasicPartitionOwner(partitionId,
+            worker);
+        partitionOwners.add(partitionOwner);
+        LOG.info("prepareCheckpointRestart partitionId=" + partitionId +
+            " assigned to " + partitionOwner);
       }
       metadataStream.close();
     }
+    //Ordering appears to be important as of right now we rely on this ordering
+    //in WorkerGraphPartitioner
+    Collections.sort(partitionOwners, new Comparator<PartitionOwner>() {
+      @Override
+      public int compare(PartitionOwner p1, PartitionOwner p2) {
+        return Integer.compare(p1.getPartitionId(), p2.getPartitionId());
+      }
+    });
+
+
+    aggregatorHandler.readFields(finalizedStream);
+    masterCompute.readFields(finalizedStream);
+    finalizedStream.close();
+
+    return partitionOwners;
   }
 
   @Override
@@ -1085,11 +1080,9 @@ public class BspServiceMaster<I extends WritableComparable,
         getZkExt().getData(superstepFinishedNode, false, null));
 
     finalizedOutputStream.writeInt(chosenWorkerInfoList.size());
+    finalizedOutputStream.writeUTF(getCheckpointBasePath(superstep));
     for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) {
-      String chosenWorkerInfoPrefix =
-          getCheckpointBasePath(superstep) + "." +
-              chosenWorkerInfo.getHostnameId();
-      finalizedOutputStream.writeUTF(chosenWorkerInfoPrefix);
+      finalizedOutputStream.writeInt(chosenWorkerInfo.getTaskId());
     }
     aggregatorHandler.write(finalizedOutputStream);
     masterCompute.write(finalizedOutputStream);
@@ -1104,18 +1097,10 @@ public class BspServiceMaster<I extends WritableComparable,
    * the workers will know how to do the exchange.  If this was a restarted
    * superstep, then make sure to provide information on where to find the
    * checkpoint file.
-   *
-   * @param allPartitionStatsList All partition stats
-   * @param chosenWorkerInfoList All the chosen worker infos
-   * @param masterGraphPartitioner Master graph partitioner
    */
-  private void assignPartitionOwners(
-      List<PartitionStats> allPartitionStatsList,
-      List<WorkerInfo> chosenWorkerInfoList,
-      MasterGraphPartitioner<I, V, E> masterGraphPartitioner) {
+  private void assignPartitionOwners() {
     Collection<PartitionOwner> partitionOwners;
-    if (getSuperstep() == INPUT_SUPERSTEP ||
-        getSuperstep() == getRestartedSuperstep()) {
+    if (getSuperstep() == INPUT_SUPERSTEP) {
       partitionOwners =
           masterGraphPartitioner.createInitialPartitionOwners(
               chosenWorkerInfoList, maxWorkers);
@@ -1123,23 +1108,10 @@ public class BspServiceMaster<I extends WritableComparable,
         throw new IllegalStateException(
             "assignAndExchangePartitions: No partition owners set");
       }
-    } else {
-      partitionOwners =
-          masterGraphPartitioner.generateChangedPartitionOwners(
-              allPartitionStatsList,
-              chosenWorkerInfoList,
-              maxWorkers,
-              getSuperstep());
-
-      PartitionUtils.analyzePartitionStats(partitionOwners,
-          allPartitionStatsList);
-    }
-    checkPartitions(masterGraphPartitioner.getCurrentPartitionOwners());
-
-    // If restarted, prepare the checkpoint restart
-    if (getRestartedSuperstep() == getSuperstep()) {
+    } else if (getRestartedSuperstep() == getSuperstep()) {
+      // If restarted, prepare the checkpoint restart
       try {
-        prepareCheckpointRestart(getSuperstep(), partitionOwners);
+        partitionOwners = prepareCheckpointRestart(getSuperstep());
       } catch (IOException e) {
         throw new IllegalStateException(
             "assignPartitionOwners: IOException on preparing", e);
@@ -1151,7 +1123,21 @@ public class BspServiceMaster<I extends WritableComparable,
             "assignPartitionOwners: InteruptedException on preparing",
             e);
       }
+      masterGraphPartitioner.setPartitionOwners(partitionOwners);
+    } else {
+      partitionOwners =
+          masterGraphPartitioner.generateChangedPartitionOwners(
+              allPartitionStatsList,
+              chosenWorkerInfoList,
+              maxWorkers,
+              getSuperstep());
+
+      PartitionUtils.analyzePartitionStats(partitionOwners,
+          allPartitionStatsList);
     }
+    checkPartitions(masterGraphPartitioner.getCurrentPartitionOwners());
+
+
 
     // There will be some exchange of partitions
     if (!partitionOwners.isEmpty()) {
@@ -1240,18 +1226,9 @@ public class BspServiceMaster<I extends WritableComparable,
     // 1. Remove all old input split data
     // 2. Increase the application attempt and set to the correct checkpoint
     // 3. Send command to all workers to restart their tasks
-    try {
-      getZkExt().deleteExt(vertexInputSplitsPaths.getPath(), -1,
-          true);
-      getZkExt().deleteExt(edgeInputSplitsPaths.getPath(), -1,
-          true);
-    } catch (InterruptedException e) {
-      throw new RuntimeException(
-          "restartFromCheckpoint: InterruptedException", e);
-    } catch (KeeperException e) {
-      throw new RuntimeException(
-          "restartFromCheckpoint: KeeperException", e);
-    }
+    zkDeleteNode(vertexInputSplitsPaths.getPath());
+    zkDeleteNode(edgeInputSplitsPaths.getPath());
+
     setApplicationAttempt(getApplicationAttempt() + 1);
     setCachedSuperstep(checkpoint);
     setRestartedSuperstep(checkpoint);
@@ -1261,6 +1238,26 @@ public class BspServiceMaster<I extends WritableComparable,
   }
 
   /**
+   * Safely removes node from zookeeper.
+   * Ignores if node is already removed. Can only throw runtime exception if
+   * anything wrong.
+   * @param path path to the node to be removed.
+   */
+  private void zkDeleteNode(String path) {
+    try {
+      getZkExt().deleteExt(path, -1, true);
+    } catch (KeeperException.NoNodeException e) {
+      LOG.info("zkDeleteNode: node has already been removed " + path);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(
+          "zkDeleteNode: InterruptedException", e);
+    } catch (KeeperException e) {
+      throw new RuntimeException(
+          "zkDeleteNode: KeeperException", e);
+    }
+  }
+
+  /**
    * Only get the finalized checkpoint files
    */
   public static class FinalizedCheckpointPathFilter implements PathFilter {
@@ -1277,7 +1274,7 @@ public class BspServiceMaster<I extends WritableComparable,
     if (lastCheckpointedSuperstep == -1) {
       try {
         FileStatus[] fileStatusArray =
-            getFs().listStatus(new Path(checkpointBasePath),
+            getFs().listStatus(new Path(savedCheckpointBasePath),
                 new FinalizedCheckpointPathFilter());
         if (fileStatusArray == null) {
           return -1;
@@ -1582,9 +1579,7 @@ public class BspServiceMaster<I extends WritableComparable,
 
     GiraphStats.getInstance().
         getCurrentWorkers().setValue(chosenWorkerInfoList.size());
-    assignPartitionOwners(allPartitionStatsList,
-        chosenWorkerInfoList,
-        masterGraphPartitioner);
+    assignPartitionOwners();
 
     // We need to finalize aggregators from previous superstep (send them to
     // worker owners) after new worker assignments

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartitionOwner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartitionOwner.java b/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartitionOwner.java
index 545d1af..b6cf813 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartitionOwner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartitionOwner.java
@@ -103,16 +103,6 @@ public class BasicPartitionOwner implements PartitionOwner,
   }
 
   @Override
-  public String getCheckpointFilesPrefix() {
-    return checkpointFilesPrefix;
-  }
-
-  @Override
-  public void setCheckpointFilesPrefix(String checkpointFilesPrefix) {
-    this.checkpointFilesPrefix = checkpointFilesPrefix;
-  }
-
-  @Override
   public void writeWithWorkerIds(DataOutput output) throws IOException {
     output.writeInt(partitionId);
     output.writeInt(workerInfo.getTaskId());

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
index 240687e..caede8c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
@@ -18,6 +18,7 @@
 
 package org.apache.giraph.partition;
 
+import com.google.common.collect.Lists;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
@@ -75,6 +76,11 @@ public class HashMasterPartitioner<I extends WritableComparable,
   }
 
   @Override
+  public void setPartitionOwners(Collection<PartitionOwner> partitionOwners) {
+    this.partitionOwnerList = Lists.newArrayList(partitionOwners);
+  }
+
+  @Override
   public Collection<PartitionOwner> getCurrentPartitionOwners() {
     return partitionOwnerList;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
index d833895..12aa417 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
@@ -65,10 +65,9 @@ public class HashWorkerPartitioner<I extends WritableComparable,
   @Override
   public PartitionExchange updatePartitionOwners(
       WorkerInfo myWorkerInfo,
-      Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore<I, V, E> partitionStore) {
+      Collection<? extends PartitionOwner> masterSetPartitionOwners) {
     return PartitionBalancer.updatePartitionOwners(partitionOwnerList,
-        myWorkerInfo, masterSetPartitionOwners, partitionStore);
+        myWorkerInfo, masterSetPartitionOwners);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
index 50c750a..d2363fb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
@@ -47,6 +47,13 @@ public interface MasterGraphPartitioner<I extends WritableComparable,
       Collection<WorkerInfo> availableWorkerInfos, int maxWorkers);
 
   /**
+   * Sets partition owners for the graph.
+   * Used then loading from checkpoint.
+   * @param partitionOwners assigned partition owners.
+   */
+  void setPartitionOwners(Collection<PartitionOwner> partitionOwners);
+
+  /**
    * After the worker stats have been merged to a single list, the master can
    * use this information to send commands to the workers for any
    * {@link Partition} changes. This protocol is specific to the

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/PartitionBalancer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionBalancer.java b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionBalancer.java
index 3454d62..0d8f3cf 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionBalancer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionBalancer.java
@@ -309,14 +309,12 @@ public class PartitionBalancer {
    * @param myWorkerInfo Worker info
    * @param masterSetPartitionOwners Master set partition owners, received
    *        prior to beginning the superstep
-   * @param partitionStore Partition store for the given worker
    * @return Information for the partition exchange.
    */
   public static PartitionExchange updatePartitionOwners(
       List<PartitionOwner> partitionOwnerList,
       WorkerInfo myWorkerInfo,
-      Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore partitionStore) {
+      Collection<? extends PartitionOwner> masterSetPartitionOwners) {
     partitionOwnerList.clear();
     partitionOwnerList.addAll(masterSetPartitionOwners);
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/PartitionOwner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionOwner.java b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionOwner.java
index 0ac74da..f303a09 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionOwner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionOwner.java
@@ -69,22 +69,6 @@ public interface PartitionOwner extends Writable {
   void setPreviousWorkerInfo(WorkerInfo workerInfo);
 
   /**
-   * If this is a restarted checkpoint, the worker will use this information
-   * to determine where the checkpointed partition was stored on HDFS.
-   *
-   * @return Prefix of the checkpoint HDFS files for this partition, null if
-   *         this is not a restarted superstep.
-   */
-  String getCheckpointFilesPrefix();
-
-  /**
-   * Set the checkpoint files prefix.  Master uses this.
-   *
-   * @param checkpointFilesPrefix HDFS checkpoint file prefix
-   */
-  void setCheckpointFilesPrefix(String checkpointFilesPrefix);
-
-  /**
    * Write to the output, but don't serialize the whole WorkerInfo,
    * instead use just the task id
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/SimpleMasterPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleMasterPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleMasterPartitioner.java
index f128f34..7d4c1cb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleMasterPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleMasterPartitioner.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import com.google.common.collect.Lists;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
@@ -74,6 +75,11 @@ public abstract class SimpleMasterPartitioner<I extends WritableComparable,
   }
 
   @Override
+  public void setPartitionOwners(Collection<PartitionOwner> partitionOwners) {
+    partitionOwnerList = Lists.newArrayList(partitionOwners);
+  }
+
+  @Override
   public Collection<PartitionOwner> generateChangedPartitionOwners(
       Collection<PartitionStats> allPartitionStatsList,
       Collection<WorkerInfo> availableWorkers,

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/SimpleWorkerPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleWorkerPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleWorkerPartitioner.java
index 3c0de44..0ee8d92 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleWorkerPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleWorkerPartitioner.java
@@ -71,11 +71,9 @@ public abstract class SimpleWorkerPartitioner<I extends WritableComparable,
 
   @Override
   public PartitionExchange updatePartitionOwners(WorkerInfo myWorkerInfo,
-      Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore<I, V, E> partitionStore) {
+      Collection<? extends PartitionOwner> masterSetPartitionOwners) {
     PartitionExchange exchange = PartitionBalancer.updatePartitionOwners(
-        partitionOwnerList, myWorkerInfo, masterSetPartitionOwners,
-        partitionStore);
+        partitionOwnerList, myWorkerInfo, masterSetPartitionOwners);
     extractAvailableWorkers();
     return exchange;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
index 004ea81..211fedb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
@@ -75,14 +75,11 @@ public interface WorkerGraphPartitioner<I extends WritableComparable,
    * @param myWorkerInfo Worker info.
    * @param masterSetPartitionOwners Master set partition owners, received
    *        prior to beginning the superstep
-   * @param partitionStore Partition store for this worker
-   *        (can be used to fill the return map of partitions to send)
    * @return Information for the partition exchange.
    */
   PartitionExchange updatePartitionOwners(
       WorkerInfo myWorkerInfo,
-      Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore<I, V, E> partitionStore);
+      Collection<? extends PartitionOwner> masterSetPartitionOwners);
 
   /**
    * Get a collection of the {@link PartitionOwner} objects.

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
index 2c4606f..bb2865c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
@@ -142,81 +142,107 @@ public class InternalVertexRunner {
       GiraphConfiguration conf,
       String[] vertexInputData,
       String[] edgeInputData) throws Exception {
-    File tmpDir = null;
+    // Prepare input file, output folder and temporary folders
+    File tmpDir = FileUtils.createTestDir(conf.getComputationName());
     try {
-      // Prepare input file, output folder and temporary folders
-      tmpDir = FileUtils.createTestDir(conf.getComputationName());
-
-      File vertexInputFile = null;
-      File edgeInputFile = null;
-      if (conf.hasVertexInputFormat()) {
-        vertexInputFile = FileUtils.createTempFile(tmpDir, "vertices.txt");
-      }
-      if (conf.hasEdgeInputFormat()) {
-        edgeInputFile = FileUtils.createTempFile(tmpDir, "edges.txt");
-      }
+      return run(conf, vertexInputData, edgeInputData, null, tmpDir);
+    } finally {
+      FileUtils.delete(tmpDir);
+    }
+  }
 
-      File outputDir = FileUtils.createTempDir(tmpDir, "output");
-      File zkDir = FileUtils.createTempDir(tmpDir, "_bspZooKeeper");
-      File zkMgrDir = FileUtils.createTempDir(tmpDir, "_defaultZkManagerDir");
-      File checkpointsDir = FileUtils.createTempDir(tmpDir, "_checkpoints");
+  /**
+   * Attempts to run the vertex internally in the current JVM, reading from and
+   * writing to a temporary folder on local disk. Will start its own zookeeper
+   * instance.
+   *
+   *
+   * @param conf GiraphClasses specifying which types to use
+   * @param vertexInputData linewise vertex input data
+   * @param edgeInputData linewise edge input data
+   * @param checkpointsDir if set, will use this folder
+   *                          for storing checkpoints.
+   * @param tmpDir file path for storing temporary files.
+   * @return linewise output data, or null if job fails
+   * @throws Exception if anything goes wrong
+   */
+  public static Iterable<String> run(
+      GiraphConfiguration conf,
+      String[] vertexInputData,
+      String[] edgeInputData,
+      String checkpointsDir,
+      File tmpDir) throws Exception {
+    File vertexInputFile = null;
+    File edgeInputFile = null;
+    if (conf.hasVertexInputFormat()) {
+      vertexInputFile = FileUtils.createTempFile(tmpDir, "vertices.txt");
+    }
+    if (conf.hasEdgeInputFormat()) {
+      edgeInputFile = FileUtils.createTempFile(tmpDir, "edges.txt");
+    }
 
-      // Write input data to disk
-      if (conf.hasVertexInputFormat()) {
-        FileUtils.writeLines(vertexInputFile, vertexInputData);
-      }
-      if (conf.hasEdgeInputFormat()) {
-        FileUtils.writeLines(edgeInputFile, edgeInputData);
-      }
+    File outputDir = FileUtils.createTempDir(tmpDir, "output");
+    File zkDir = FileUtils.createTempDir(tmpDir, "_bspZooKeeper");
+    File zkMgrDir = FileUtils.createTempDir(tmpDir, "_defaultZkManagerDir");
+    // Write input data to disk
+    if (conf.hasVertexInputFormat()) {
+      FileUtils.writeLines(vertexInputFile, vertexInputData);
+    }
+    if (conf.hasEdgeInputFormat()) {
+      FileUtils.writeLines(edgeInputFile, edgeInputData);
+    }
 
-      int localZookeeperPort = findAvailablePort();
+    int localZookeeperPort = findAvailablePort();
 
-      conf.setWorkerConfiguration(1, 1, 100.0f);
-      GiraphConstants.SPLIT_MASTER_WORKER.set(conf, false);
-      GiraphConstants.LOCAL_TEST_MODE.set(conf, true);
-      conf.setZookeeperList("localhost:" +
+    conf.setWorkerConfiguration(1, 1, 100.0f);
+    GiraphConstants.SPLIT_MASTER_WORKER.set(conf, false);
+    GiraphConstants.LOCAL_TEST_MODE.set(conf, true);
+    conf.setZookeeperList("localhost:" +
           String.valueOf(localZookeeperPort));
 
-      conf.set(GiraphConstants.ZOOKEEPER_DIR, zkDir.toString());
-      GiraphConstants.ZOOKEEPER_MANAGER_DIRECTORY.set(conf,
-          zkMgrDir.toString());
-      GiraphConstants.CHECKPOINT_DIRECTORY.set(conf, checkpointsDir.toString());
+    conf.set(GiraphConstants.ZOOKEEPER_DIR, zkDir.toString());
+    GiraphConstants.ZOOKEEPER_MANAGER_DIRECTORY.set(conf,
+        zkMgrDir.toString());
 
-      // Create and configure the job to run the vertex
-      GiraphJob job = new GiraphJob(conf, conf.getComputationName());
+    if (checkpointsDir == null) {
+      checkpointsDir = FileUtils.createTempDir(
+          tmpDir, "_checkpoints").toString();
+    }
+    GiraphConstants.CHECKPOINT_DIRECTORY.set(conf, checkpointsDir);
 
-      Job internalJob = job.getInternalJob();
-      if (conf.hasVertexInputFormat()) {
-        GiraphFileInputFormat.setVertexInputPath(internalJob.getConfiguration(),
-            new Path(vertexInputFile.toString()));
-      }
-      if (conf.hasEdgeInputFormat()) {
-        GiraphFileInputFormat.setEdgeInputPath(internalJob.getConfiguration(),
-            new Path(edgeInputFile.toString()));
-      }
-      FileOutputFormat.setOutputPath(job.getInternalJob(),
-          new Path(outputDir.toString()));
+    // Create and configure the job to run the vertex
+    GiraphJob job = new GiraphJob(conf, conf.getComputationName());
 
-      // Configure a local zookeeper instance
-      Properties zkProperties = configLocalZooKeeper(zkDir, localZookeeperPort);
+    Job internalJob = job.getInternalJob();
+    if (conf.hasVertexInputFormat()) {
+      GiraphFileInputFormat.setVertexInputPath(internalJob.getConfiguration(),
+          new Path(vertexInputFile.toString()));
+    }
+    if (conf.hasEdgeInputFormat()) {
+      GiraphFileInputFormat.setEdgeInputPath(internalJob.getConfiguration(),
+          new Path(edgeInputFile.toString()));
+    }
+    FileOutputFormat.setOutputPath(job.getInternalJob(),
+        new Path(outputDir.toString()));
 
-      QuorumPeerConfig qpConfig = new QuorumPeerConfig();
-      qpConfig.parseProperties(zkProperties);
+    // Configure a local zookeeper instance
+    Properties zkProperties = configLocalZooKeeper(zkDir, localZookeeperPort);
 
-      boolean success = runZooKeeperAndJob(qpConfig, job);
-      if (!success) {
-        return null;
-      }
+    QuorumPeerConfig qpConfig = new QuorumPeerConfig();
+    qpConfig.parseProperties(zkProperties);
 
-      File outFile = new File(outputDir, "part-m-00000");
-      if (conf.hasVertexOutputFormat() && outFile.canRead()) {
-        return Files.readLines(outFile, Charsets.UTF_8);
-      } else {
-        return ImmutableList.of();
-      }
-    } finally {
-      FileUtils.delete(tmpDir);
+    boolean success = runZooKeeperAndJob(qpConfig, job);
+    if (!success) {
+      return null;
     }
+
+    File outFile = new File(outputDir, "part-m-00000");
+    if (conf.hasVertexOutputFormat() && outFile.canRead()) {
+      return Files.readLines(outFile, Charsets.UTF_8);
+    } else {
+      return ImmutableList.of();
+    }
+
   }
 
   /**
@@ -236,42 +262,97 @@ public class InternalVertexRunner {
       E extends Writable> void run(
       GiraphConfiguration conf,
       TestGraph<I, V, E> graph) throws Exception {
-    File tmpDir = null;
+    // Prepare temporary folders
+    File tmpDir = FileUtils.createTestDir(conf.getComputationName());
     try {
-      // Prepare temporary folders
-      tmpDir = FileUtils.createTestDir(conf.getComputationName());
+      run(conf, graph, tmpDir, null);
+    } finally {
+      FileUtils.delete(tmpDir);
+    }
+  }
 
-      File zkDir = FileUtils.createTempDir(tmpDir, "_bspZooKeeper");
-      File zkMgrDir = FileUtils.createTempDir(tmpDir, "_defaultZkManagerDir");
-      File checkpointsDir = FileUtils.createTempDir(tmpDir, "_checkpoints");
+  /**
+   * Attempts to run the vertex internally in the current JVM,
+   * reading from an in-memory graph. Will start its own zookeeper
+   * instance.
+   *
+   * @param <I> Vertex ID
+   * @param <V> Vertex Value
+   * @param <E> Edge Value
+   * @param conf GiraphClasses specifying which types to use
+   * @param graph input graph
+   * @param tmpDir file path for storing temporary files.
+   * @param checkpointsDir if set, will use this folder
+   *                          for storing checkpoints.
+   * @throws Exception if anything goes wrong
+   */
+  public static <I extends WritableComparable,
+      V extends Writable,
+      E extends Writable> void run(
+      GiraphConfiguration conf,
+      TestGraph<I, V, E> graph,
+      File tmpDir,
+      String checkpointsDir) throws Exception {
+    File zkDir = FileUtils.createTempDir(tmpDir, "_bspZooKeeper");
+    File zkMgrDir = FileUtils.createTempDir(tmpDir, "_defaultZkManagerDir");
+
+    if (checkpointsDir == null) {
+      checkpointsDir = FileUtils.
+          createTempDir(tmpDir, "_checkpoints").toString();
+    }
 
-      conf.setVertexInputFormatClass(InMemoryVertexInputFormat.class);
+    conf.setVertexInputFormatClass(InMemoryVertexInputFormat.class);
 
-      // Create and configure the job to run the vertex
-      GiraphJob job = new GiraphJob(conf, conf.getComputationName());
+    // Create and configure the job to run the vertex
+    GiraphJob job = new GiraphJob(conf, conf.getComputationName());
 
-      InMemoryVertexInputFormat.setGraph(graph);
+    InMemoryVertexInputFormat.setGraph(graph);
 
-      int localZookeeperPort = findAvailablePort();
+    int localZookeeperPort = findAvailablePort();
 
-      conf.setWorkerConfiguration(1, 1, 100.0f);
-      GiraphConstants.SPLIT_MASTER_WORKER.set(conf, false);
-      GiraphConstants.LOCAL_TEST_MODE.set(conf, true);
-      GiraphConstants.ZOOKEEPER_LIST.set(conf, "localhost:" +
+    conf.setWorkerConfiguration(1, 1, 100.0f);
+    GiraphConstants.SPLIT_MASTER_WORKER.set(conf, false);
+    GiraphConstants.LOCAL_TEST_MODE.set(conf, true);
+    GiraphConstants.ZOOKEEPER_LIST.set(conf, "localhost:" +
           String.valueOf(localZookeeperPort));
 
-      conf.set(GiraphConstants.ZOOKEEPER_DIR, zkDir.toString());
-      GiraphConstants.ZOOKEEPER_MANAGER_DIRECTORY.set(conf,
-          zkMgrDir.toString());
-      GiraphConstants.CHECKPOINT_DIRECTORY.set(conf, checkpointsDir.toString());
+    conf.set(GiraphConstants.ZOOKEEPER_DIR, zkDir.toString());
+    GiraphConstants.ZOOKEEPER_MANAGER_DIRECTORY.set(conf,
+        zkMgrDir.toString());
+    GiraphConstants.CHECKPOINT_DIRECTORY.set(conf, checkpointsDir);
+
+    // Configure a local zookeeper instance
+    Properties zkProperties = configLocalZooKeeper(zkDir, localZookeeperPort);
+
+    QuorumPeerConfig qpConfig = new QuorumPeerConfig();
+    qpConfig.parseProperties(zkProperties);
 
-      // Configure a local zookeeper instance
-      Properties zkProperties = configLocalZooKeeper(zkDir, localZookeeperPort);
+    runZooKeeperAndJob(qpConfig, job);
 
-      QuorumPeerConfig qpConfig = new QuorumPeerConfig();
-      qpConfig.parseProperties(zkProperties);
+  }
 
-      runZooKeeperAndJob(qpConfig, job);
+  /**
+   * Attempts to run the vertex internally in the current JVM, reading and
+   * writing to an in-memory graph. Will start its own zookeeper
+   * instance.
+   *
+   * @param <I> Vertex ID
+   * @param <V> Vertex Value
+   * @param <E> Edge Value
+   * @param conf GiraphClasses specifying which types to use
+   * @param graph input graph
+   * @return Output graph
+   * @throws Exception if anything goes wrong
+   */
+  public static <I extends WritableComparable,
+      V extends Writable,
+      E extends Writable> TestGraph<I, V, E> runWithInMemoryOutput(
+      GiraphConfiguration conf,
+      TestGraph<I, V, E> graph) throws Exception {
+    // Prepare temporary folders
+    File tmpDir = FileUtils.createTestDir(conf.getComputationName());
+    try {
+      return runWithInMemoryOutput(conf, graph, tmpDir, null);
     } finally {
       FileUtils.delete(tmpDir);
     }
@@ -287,6 +368,9 @@ public class InternalVertexRunner {
    * @param <E> Edge Value
    * @param conf GiraphClasses specifying which types to use
    * @param graph input graph
+   * @param tmpDir file path for storing temporary files.
+   * @param checkpointsDir if set, will use this folder
+   *                       for storing checkpoints.
    * @return Output graph
    * @throws Exception if anything goes wrong
    */
@@ -294,10 +378,12 @@ public class InternalVertexRunner {
       V extends Writable,
       E extends Writable> TestGraph<I, V, E> runWithInMemoryOutput(
       GiraphConfiguration conf,
-      TestGraph<I, V, E> graph) throws Exception {
+      TestGraph<I, V, E> graph,
+      File tmpDir,
+      String checkpointsDir) throws Exception {
     conf.setVertexOutputFormatClass(InMemoryVertexOutputFormat.class);
     InMemoryVertexOutputFormat.initializeOutputGraph(conf);
-    InternalVertexRunner.run(conf, graph);
+    InternalVertexRunner.run(conf, graph, tmpDir, checkpointsDir);
     return InMemoryVertexOutputFormat.getOutputGraph();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/utils/io/ExtendedDataInputOutput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/io/ExtendedDataInputOutput.java b/giraph-core/src/main/java/org/apache/giraph/utils/io/ExtendedDataInputOutput.java
index af45426..a5b1567 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/io/ExtendedDataInputOutput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/io/ExtendedDataInputOutput.java
@@ -65,6 +65,6 @@ public class ExtendedDataInputOutput extends DataInputOutput {
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    WritableUtils.readExtendedDataOutput(in, conf);
+    dataOutput = WritableUtils.readExtendedDataOutput(in, conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index de7af28..0d90a59 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -70,10 +70,13 @@ import org.apache.giraph.utils.ProgressableUtils;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.giraph.zk.BspEvent;
 import org.apache.giraph.zk.PredicateLock;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.log4j.Level;
@@ -92,9 +95,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import net.iharder.Base64;
 
-import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
-import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.charset.Charset;
@@ -110,6 +111,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -566,7 +568,7 @@ public class BspServiceWorker<I extends WritableComparable,
     Collection<? extends PartitionOwner> masterSetPartitionOwners =
         startSuperstep();
     workerGraphPartitioner.updatePartitionOwners(
-        getWorkerInfo(), masterSetPartitionOwners, getPartitionStore());
+        getWorkerInfo(), masterSetPartitionOwners);
 
     /*if[HADOOP_NON_SECURE]
       workerClient.setup();
@@ -1370,73 +1372,42 @@ public class BspServiceWorker<I extends WritableComparable,
     // Algorithm:
     // For each partition, dump vertices and messages
     Path metadataFilePath =
-        new Path(getCheckpointBasePath(getSuperstep()) + "." +
-            getHostnamePartitionId() +
-            CHECKPOINT_METADATA_POSTFIX);
-    Path verticesFilePath =
-        new Path(getCheckpointBasePath(getSuperstep()) + "." +
-            getHostnamePartitionId() +
-            CHECKPOINT_VERTICES_POSTFIX);
+        createCheckpointFilePathSafe(CHECKPOINT_METADATA_POSTFIX);
     Path validFilePath =
-        new Path(getCheckpointBasePath(getSuperstep()) + "." +
-            getHostnamePartitionId() +
-            CHECKPOINT_VALID_POSTFIX);
+        createCheckpointFilePathSafe(CHECKPOINT_VALID_POSTFIX);
+    Path checkpointFilePath =
+        createCheckpointFilePathSafe(CHECKPOINT_DATA_POSTFIX);
 
-    // Remove these files if they already exist (shouldn't though, unless
-    // of previous failure of this worker)
-    if (getFs().delete(validFilePath, false)) {
-      LOG.warn("storeCheckpoint: Removed valid file " +
-          validFilePath);
-    }
-    if (getFs().delete(metadataFilePath, false)) {
-      LOG.warn("storeCheckpoint: Removed metadata file " +
-          metadataFilePath);
-    }
-    if (getFs().delete(verticesFilePath, false)) {
-      LOG.warn("storeCheckpoint: Removed file " + verticesFilePath);
-    }
 
-    FSDataOutputStream verticesOutputStream =
-        getFs().create(verticesFilePath);
-    ByteArrayOutputStream metadataByteStream = new ByteArrayOutputStream();
-    DataOutput metadataOutput = new DataOutputStream(metadataByteStream);
-    for (Integer partitionId : getPartitionStore().getPartitionIds()) {
-      Partition<I, V, E> partition =
-          getPartitionStore().getOrCreatePartition(partitionId);
-      long startPos = verticesOutputStream.getPos();
-      partition.write(verticesOutputStream);
-      // write messages
-      getServerData().getCurrentMessageStore().writePartition(
-          verticesOutputStream, partition.getId());
-      // Write the metadata for this partition
-      // Format:
-      // <index count>
-      //   <index 0 start pos><partition id>
-      //   <index 1 start pos><partition id>
-      metadataOutput.writeLong(startPos);
-      metadataOutput.writeInt(partition.getId());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("storeCheckpoint: Vertex file starting " +
-            "offset = " + startPos + ", length = " +
-            (verticesOutputStream.getPos() - startPos) +
-            ", partition = " + partition.toString());
-      }
-      getPartitionStore().putPartition(partition);
-      getContext().progress();
-    }
     // Metadata is buffered and written at the end since it's small and
     // needs to know how many partitions this worker owns
     FSDataOutputStream metadataOutputStream =
         getFs().create(metadataFilePath);
     metadataOutputStream.writeInt(getPartitionStore().getNumPartitions());
-    metadataOutputStream.write(metadataByteStream.toByteArray());
+
+    for (Integer partitionId : getPartitionStore().getPartitionIds()) {
+      metadataOutputStream.writeInt(partitionId);
+    }
     metadataOutputStream.close();
-    verticesOutputStream.close();
-    if (LOG.isInfoEnabled()) {
-      LOG.info("storeCheckpoint: Finished metadata (" +
-          metadataFilePath + ") and vertices (" + verticesFilePath + ").");
+
+    storeCheckpointVertices();
+
+    FSDataOutputStream checkpointOutputStream =
+        getFs().create(checkpointFilePath);
+    workerContext.write(checkpointOutputStream);
+    getContext().progress();
+
+    for (Integer partitionId : getPartitionStore().getPartitionIds()) {
+      // write messages
+      checkpointOutputStream.writeInt(partitionId);
+      getServerData().getCurrentMessageStore().writePartition(
+          checkpointOutputStream, partitionId);
+      getContext().progress();
+
     }
 
+    checkpointOutputStream.close();
+
     getFs().createNewFile(validFilePath);
 
     // Notify master that checkpoint is stored
@@ -1462,116 +1433,247 @@ public class BspServiceWorker<I extends WritableComparable,
     }
   }
 
+  /**
+   * Create checkpoint file safely. If file already exists remove it first.
+   * @param name file extension
+   * @return full file path to newly created file
+   * @throws IOException
+   */
+  private Path createCheckpointFilePathSafe(String name) throws IOException {
+    Path validFilePath = new Path(getCheckpointBasePath(getSuperstep()) + "." +
+        getTaskPartition() + name);
+    // Remove these files if they already exist (shouldn't though, unless
+    // of previous failure of this worker)
+    if (getFs().delete(validFilePath, false)) {
+      LOG.warn("storeCheckpoint: Removed " + name + " file " +
+          validFilePath);
+    }
+    return validFilePath;
+  }
+
+  /**
+   * Returns path to saved checkpoint.
+   * Doesn't check if file actually exists.
+   * @param superstep saved superstep.
+   * @param name extension name
+   * @return fill file path to checkpoint file
+   */
+  private Path getSavedCheckpoint(long superstep, String name) {
+    return new Path(getSavedCheckpointBasePath(superstep) + "." +
+        getTaskPartition() + name);
+  }
+
+  /**
+   * Save partitions. To speed up this operation
+   * runs in multiple threads.
+   */
+  private void storeCheckpointVertices() {
+    final int numPartitions = getPartitionStore().getNumPartitions();
+    int numThreads = Math.min(
+        GiraphConstants.NUM_CHECKPOINT_IO_THREADS.get(getConfiguration()),
+        numPartitions);
+
+    final Queue<Integer> partitionIdQueue =
+        (numPartitions == 0) ? new LinkedList<Integer>() :
+            new ArrayBlockingQueue<Integer>(numPartitions);
+    Iterables.addAll(partitionIdQueue, getPartitionStore().getPartitionIds());
+
+    final CompressionCodec codec =
+        new CompressionCodecFactory(getConfiguration())
+            .getCodecByClassName(
+                GiraphConstants.CHECKPOINT_COMPRESSION_CODEC
+                    .get(getConfiguration()));
+
+    long t0 = System.currentTimeMillis();
+
+    CallableFactory<Void> callableFactory = new CallableFactory<Void>() {
+      @Override
+      public Callable<Void> newCallable(int callableId) {
+        return new Callable<Void>() {
+
+          @Override
+          public Void call() throws Exception {
+            while (!partitionIdQueue.isEmpty()) {
+              Integer partitionId = partitionIdQueue.poll();
+              if (partitionId == null) {
+                break;
+              }
+              Path path =
+                  createCheckpointFilePathSafe("_" + partitionId +
+                      CHECKPOINT_VERTICES_POSTFIX);
+
+              FSDataOutputStream uncompressedStream =
+                  getFs().create(path);
+
+
+              DataOutputStream stream = codec == null ? uncompressedStream :
+                  new DataOutputStream(
+                      codec.createOutputStream(uncompressedStream));
+
+              Partition<I, V, E> partition =
+                  getPartitionStore().getOrCreatePartition(partitionId);
+
+              partition.write(stream);
+
+              getPartitionStore().putPartition(partition);
+
+              stream.close();
+              uncompressedStream.close();
+            }
+            return null;
+          }
+
+
+        };
+      }
+    };
+
+    ProgressableUtils.getResultsWithNCallables(callableFactory, numThreads,
+        "checkpoint-vertices-%d", getContext());
+
+    LOG.info("Save checkpoint in " + (System.currentTimeMillis() - t0) +
+        " ms, using " + numThreads + " threads");
+  }
+
+  /**
+   * Load saved partitions in multiple threads.
+   * @param superstep superstep to load
+   * @param partitions list of partitions to load
+   */
+  private void loadCheckpointVertices(final long superstep,
+                                      List<Integer> partitions) {
+    int numThreads = Math.min(
+        GiraphConstants.NUM_CHECKPOINT_IO_THREADS.get(getConfiguration()),
+        partitions.size());
+
+    final Queue<Integer> partitionIdQueue =
+        new ConcurrentLinkedQueue<>(partitions);
+
+    final CompressionCodec codec =
+        new CompressionCodecFactory(getConfiguration())
+            .getCodecByClassName(
+                GiraphConstants.CHECKPOINT_COMPRESSION_CODEC
+                    .get(getConfiguration()));
+
+    long t0 = System.currentTimeMillis();
+
+    CallableFactory<Void> callableFactory = new CallableFactory<Void>() {
+      @Override
+      public Callable<Void> newCallable(int callableId) {
+        return new Callable<Void>() {
+
+          @Override
+          public Void call() throws Exception {
+            while (!partitionIdQueue.isEmpty()) {
+              Integer partitionId = partitionIdQueue.poll();
+              if (partitionId == null) {
+                break;
+              }
+              Path path =
+                  getSavedCheckpoint(superstep, "_" + partitionId +
+                      CHECKPOINT_VERTICES_POSTFIX);
+
+              FSDataInputStream compressedStream =
+                  getFs().open(path);
+
+              DataInputStream stream = codec == null ? compressedStream :
+                  new DataInputStream(
+                      codec.createInputStream(compressedStream));
+
+              Partition<I, V, E> partition =
+                  getConfiguration().createPartition(partitionId, getContext());
+
+              partition.readFields(stream);
+
+              getPartitionStore().addPartition(partition);
+
+              stream.close();
+            }
+            return null;
+          }
+
+        };
+      }
+    };
+
+    ProgressableUtils.getResultsWithNCallables(callableFactory, numThreads,
+        "load-vertices-%d", getContext());
+
+    LOG.info("Loaded checkpoint in " + (System.currentTimeMillis() - t0) +
+        " ms, using " + numThreads + " threads");
+  }
+
   @Override
   public VertexEdgeCount loadCheckpoint(long superstep) {
-    try {
-      // clear old message stores
-      getServerData().getIncomingMessageStore().clearAll();
-      getServerData().getCurrentMessageStore().clearAll();
-    } catch (IOException e) {
-      throw new RuntimeException(
-          "loadCheckpoint: Failed to clear message stores ", e);
-    }
+    Path metadataFilePath =
+        getSavedCheckpoint(superstep, CHECKPOINT_METADATA_POSTFIX);
 
+    Path checkpointFilePath =
+        getSavedCheckpoint(superstep, CHECKPOINT_DATA_POSTFIX);
     // Algorithm:
     // Examine all the partition owners and load the ones
     // that match my hostname and id from the master designated checkpoint
     // prefixes.
-    long startPos = 0;
-    int loadedPartitions = 0;
-    for (PartitionOwner partitionOwner :
-      workerGraphPartitioner.getPartitionOwners()) {
-      if (partitionOwner.getWorkerInfo().equals(getWorkerInfo())) {
-        String metadataFile =
-            partitionOwner.getCheckpointFilesPrefix() +
-            CHECKPOINT_METADATA_POSTFIX;
-        String partitionsFile =
-            partitionOwner.getCheckpointFilesPrefix() +
-            CHECKPOINT_VERTICES_POSTFIX;
-        try {
-          int partitionId = -1;
-          DataInputStream metadataStream =
-              getFs().open(new Path(metadataFile));
-          int partitions = metadataStream.readInt();
-          for (int i = 0; i < partitions; ++i) {
-            startPos = metadataStream.readLong();
-            partitionId = metadataStream.readInt();
-            if (partitionId == partitionOwner.getPartitionId()) {
-              break;
-            }
-          }
-          if (partitionId != partitionOwner.getPartitionId()) {
-            throw new IllegalStateException(
-                "loadCheckpoint: " + partitionOwner +
-                " not found!");
-          }
-          metadataStream.close();
-          Partition<I, V, E> partition =
-              getConfiguration().createPartition(partitionId, getContext());
-          DataInputStream partitionsStream =
-              getFs().open(new Path(partitionsFile));
-          if (partitionsStream.skip(startPos) != startPos) {
-            throw new IllegalStateException(
-                "loadCheckpoint: Failed to skip " + startPos +
-                " on " + partitionsFile);
-          }
-          partition.readFields(partitionsStream);
-          getServerData().getIncomingMessageStore().readFieldsForPartition(
-              partitionsStream, partitionId);
-          partitionsStream.close();
-          if (LOG.isInfoEnabled()) {
-            LOG.info("loadCheckpoint: Loaded partition " +
-                partition);
-          }
-          if (getPartitionStore().hasPartition(partitionId)) {
-            throw new IllegalStateException(
-                "loadCheckpoint: Already has partition owner " +
-                    partitionOwner);
-          }
-          getPartitionStore().addPartition(partition);
-          getContext().progress();
-          ++loadedPartitions;
-        } catch (IOException e) {
-          throw new RuntimeException(
-              "loadCheckpoint: Failed to get partition owner " +
-                  partitionOwner, e);
-        }
+    try {
+      DataInputStream metadataStream =
+          getFs().open(metadataFilePath);
+
+      int partitions = metadataStream.readInt();
+      List<Integer> partitionIds = new ArrayList<>(partitions);
+      for (int i = 0; i < partitions; i++) {
+        int partitionId = metadataStream.readInt();
+        partitionIds.add(partitionId);
       }
-    }
-    if (LOG.isInfoEnabled()) {
-      LOG.info("loadCheckpoint: Loaded " + loadedPartitions +
-          " partitions of out " +
-          workerGraphPartitioner.getPartitionOwners().size() +
-          " total.");
-    }
 
-    // Load global stats and superstep classes
-    GlobalStats globalStats = new GlobalStats();
-    SuperstepClasses superstepClasses = new SuperstepClasses();
-    String finalizedCheckpointPath =
-        getCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
-    try {
+      loadCheckpointVertices(superstep, partitionIds);
+
+      getContext().progress();
+
+      metadataStream.close();
+
+      DataInputStream checkpointStream =
+          getFs().open(checkpointFilePath);
+      workerContext.readFields(checkpointStream);
+
+      // Load global stats and superstep classes
+      GlobalStats globalStats = new GlobalStats();
+      SuperstepClasses superstepClasses = new SuperstepClasses();
+      String finalizedCheckpointPath =
+          getSavedCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
       DataInputStream finalizedStream =
           getFs().open(new Path(finalizedCheckpointPath));
       globalStats.readFields(finalizedStream);
       superstepClasses.readFields(finalizedStream);
       getConfiguration().updateSuperstepClasses(superstepClasses);
-    } catch (IOException e) {
-      throw new IllegalStateException(
-          "loadCheckpoint: Failed to load global stats and superstep classes",
-          e);
-    }
+      getServerData().resetMessageStores();
 
-    getServerData().prepareSuperstep();
-    // Communication service needs to setup the connections prior to
-    // processing vertices
+      for (int i = 0; i < partitions; i++) {
+        int partitionId = checkpointStream.readInt();
+        getServerData().getCurrentMessageStore().readFieldsForPartition(
+            checkpointStream, partitionId);
+      }
+      checkpointStream.close();
+
+      if (LOG.isInfoEnabled()) {
+        LOG.info("loadCheckpoint: Loaded " +
+            workerGraphPartitioner.getPartitionOwners().size() +
+            " total.");
+      }
+
+      // Communication service needs to setup the connections prior to
+      // processing vertices
 /*if[HADOOP_NON_SECURE]
     workerClient.setup();
 else[HADOOP_NON_SECURE]*/
-    workerClient.setup(getConfiguration().authenticate());
+      workerClient.setup(getConfiguration().authenticate());
 /*end[HADOOP_NON_SECURE]*/
-    return new VertexEdgeCount(globalStats.getVertexCount(),
-        globalStats.getEdgeCount());
+      return new VertexEdgeCount(globalStats.getVertexCount(),
+          globalStats.getEdgeCount());
+
+    } catch (IOException e) {
+      throw new RuntimeException(
+          "loadCheckpoint: Failed for superstep=" + superstep, e);
+    }
   }
 
   /**
@@ -1651,7 +1753,7 @@ else[HADOOP_NON_SECURE]*/
     // 5. Add the partitions to myself.
     PartitionExchange partitionExchange =
         workerGraphPartitioner.updatePartitionOwners(
-            getWorkerInfo(), masterSetPartitionOwners, getPartitionStore());
+            getWorkerInfo(), masterSetPartitionOwners);
     workerClient.openConnections();
 
     Map<WorkerInfo, List<Integer>> sendWorkerPartitionMap =

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
index 29835c5..aca9944 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
@@ -25,6 +25,9 @@ import org.apache.giraph.graph.GraphState;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.List;
 
 /**
@@ -33,9 +36,8 @@ import java.util.List;
  */
 @SuppressWarnings("rawtypes")
 public abstract class WorkerContext
-    extends DefaultImmutableClassesGiraphConfigurable
-    implements WorkerAggregatorUsage {
-
+  extends DefaultImmutableClassesGiraphConfigurable
+  implements WorkerAggregatorUsage, Writable {
   /** Global graph state */
   private GraphState graphState;
   /** Worker aggregator usage */
@@ -203,4 +205,12 @@ public abstract class WorkerContext
   public <A extends Writable> A getAggregatedValue(String name) {
     return workerAggregatorUsage.<A>getAggregatedValue(name);
   }
+
+  @Override
+  public void write(DataOutput dataOutput) throws IOException {
+  }
+
+  @Override
+  public void readFields(DataInput dataInput) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-core/src/test/java/org/apache/giraph/partition/SimpleRangePartitionFactoryTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/SimpleRangePartitionFactoryTest.java b/giraph-core/src/test/java/org/apache/giraph/partition/SimpleRangePartitionFactoryTest.java
index 96bd5d7..57bebbd 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/SimpleRangePartitionFactoryTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/SimpleRangePartitionFactoryTest.java
@@ -66,7 +66,7 @@ public class SimpleRangePartitionFactoryTest {
 
     WorkerGraphPartitioner<LongWritable, Writable, Writable> workerPartitioner =
         factory.createWorkerGraphPartitioner();
-    workerPartitioner.updatePartitionOwners(null, owners, null);
+    workerPartitioner.updatePartitionOwners(null, owners);
     LongWritable longWritable = new LongWritable();
 
     int[] partitions = new int[keySpaceSize];

http://git-wip-us.apache.org/repos/asf/giraph/blob/02d9e6c2/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java b/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
new file mode 100644
index 0000000..387b937
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
@@ -0,0 +1,266 @@
+/*
+ * 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;
+
+import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.examples.SimpleSuperstepComputation;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.job.GiraphJob;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.worker.DefaultWorkerContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests that worker context and master computation
+ * are properly saved and loaded back at checkpoint.
+ */
+public class TestCheckpointing extends BspCase {
+
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(TestCheckpointing.class);
+  /** ID to be used with test job */
+  public static final String TEST_JOB_ID = "test_job";
+  /**
+   * Compute will double check that we don't run supersteps
+   * lesser than specified by this key. That way we ensure that
+   * computation actually restarted and not recalculated from the
+   * beginning.
+   */
+  public static final String KEY_MIN_SUPERSTEP = "minimum.superstep";
+
+  /**
+   * Create the test case
+   */
+  public TestCheckpointing() {
+    super(TestCheckpointing.class.getName());
+  }
+
+
+  @Test
+  public void testBspCheckpoint()
+      throws IOException, InterruptedException, ClassNotFoundException {
+    Path checkpointsDir = getTempPath("checkpointing");
+    Path outputPath = getTempPath(getCallingMethodName());
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setComputationClass(
+        CheckpointComputation.class);
+    conf.setWorkerContextClass(
+        CheckpointVertexWorkerContext.class);
+    conf.setMasterComputeClass(
+        CheckpointVertexMasterCompute.class);
+    conf.setVertexInputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat.class);
+    conf.setVertexOutputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat.class);
+    conf.set("mapred.job.id", TEST_JOB_ID);
+    conf.set(KEY_MIN_SUPERSTEP, "0");
+    GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
+
+    GiraphConfiguration configuration = job.getConfiguration();
+    GiraphConstants.CHECKPOINT_DIRECTORY.set(configuration, checkpointsDir.toString());
+    GiraphConstants.CLEANUP_CHECKPOINTS_AFTER_SUCCESS.set(configuration, false);
+    configuration.setCheckpointFrequency(2);
+
+    assertTrue(job.run(true));
+
+    long idSum = 0;
+    if (!runningInDistributedMode()) {
+      FileStatus fileStatus = getSinglePartFileStatus(job.getConfiguration(),
+          outputPath);
+      idSum = CheckpointVertexWorkerContext
+          .getFinalSum();
+      LOG.info("testBspCheckpoint: idSum = " + idSum +
+          " fileLen = " + fileStatus.getLen());
+    }
+
+    // Restart the test from superstep 2
+    LOG.info("testBspCheckpoint: Restarting from superstep 2" +
+        " with checkpoint path = " + checkpointsDir);
+    outputPath = getTempPath("checkpointing_restarted");
+
+    GiraphConstants.RESTART_JOB_ID.set(conf, TEST_JOB_ID);
+    conf.set("mapred.job.id", "restarted_test_job");
+    conf.set(GiraphConstants.RESTART_SUPERSTEP, "2");
+    conf.set(KEY_MIN_SUPERSTEP, "2");
+
+    GiraphJob restartedJob = prepareJob(getCallingMethodName() + "Restarted",
+        conf, outputPath);
+
+    GiraphConstants.CHECKPOINT_DIRECTORY.set(restartedJob.getConfiguration(),
+        checkpointsDir.toString());
+
+    assertTrue(restartedJob.run(true));
+    if (!runningInDistributedMode()) {
+      long idSumRestarted =
+          CheckpointVertexWorkerContext
+              .getFinalSum();
+      LOG.info("testBspCheckpoint: idSumRestarted = " +
+          idSumRestarted);
+      assertEquals(idSum, idSumRestarted);
+    }
+  }
+
+
+  /**
+   * Actual computation.
+   */
+  public static class CheckpointComputation extends
+      BasicComputation<LongWritable, IntWritable, FloatWritable,
+          FloatWritable> {
+    @Override
+    public void compute(
+        Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+        Iterable<FloatWritable> messages) throws IOException {
+      CheckpointVertexWorkerContext workerContext = getWorkerContext();
+      assertEquals(getSuperstep() + 1, workerContext.testValue);
+
+      if (getSuperstep() < getConf().getInt(KEY_MIN_SUPERSTEP, Integer.MAX_VALUE)){
+        fail("Should not be running compute on superstep " + getSuperstep());
+      }
+
+      if (getSuperstep() > 4) {
+        vertex.voteToHalt();
+        return;
+      }
+
+      aggregate(LongSumAggregator.class.getName(),
+          new LongWritable(vertex.getId().get()));
+
+      float msgValue = 0.0f;
+      for (FloatWritable message : messages) {
+        float curMsgValue = message.get();
+        msgValue += curMsgValue;
+      }
+
+      int vertexValue = vertex.getValue().get();
+      vertex.setValue(new IntWritable(vertexValue + (int) msgValue));
+      for (Edge<LongWritable, FloatWritable> edge : vertex.getEdges()) {
+        FloatWritable newEdgeValue = new FloatWritable(edge.getValue().get() +
+            (float) vertexValue);
+        Edge<LongWritable, FloatWritable> newEdge =
+            EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
+        vertex.addEdge(newEdge);
+        sendMessage(edge.getTargetVertexId(), newEdgeValue);
+      }
+    }
+  }
+
+  /**
+   * Worker context associated.
+   */
+  public static class CheckpointVertexWorkerContext
+      extends DefaultWorkerContext {
+    /** User can access this after the application finishes if local */
+    private static long FINAL_SUM;
+
+    private int testValue;
+
+    public static long getFinalSum() {
+      return FINAL_SUM;
+    }
+
+    @Override
+    public void postApplication() {
+      setFinalSum(this.<LongWritable>getAggregatedValue(
+          LongSumAggregator.class.getName()).get());
+      LOG.info("FINAL_SUM=" + FINAL_SUM);
+    }
+
+    /**
+     * Set the final sum
+     *
+     * @param value sum
+     */
+    private static void setFinalSum(long value) {
+      FINAL_SUM = value;
+    }
+
+    @Override
+    public void preSuperstep() {
+      assertEquals(getSuperstep(), testValue++);
+    }
+
+    @Override
+    public void readFields(DataInput dataInput) throws IOException {
+      super.readFields(dataInput);
+      testValue = dataInput.readInt();
+    }
+
+    @Override
+    public void write(DataOutput dataOutput) throws IOException {
+      super.write(dataOutput);
+      dataOutput.writeInt(testValue);
+    }
+  }
+
+  /**
+   * Master compute
+   */
+  public static class CheckpointVertexMasterCompute extends
+      DefaultMasterCompute {
+
+    private int testValue = 0;
+
+    @Override
+    public void compute() {
+      long superstep = getSuperstep();
+      assertEquals(superstep, testValue++);
+    }
+
+    @Override
+    public void initialize() throws InstantiationException,
+        IllegalAccessException {
+      registerAggregator(LongSumAggregator.class.getName(),
+          LongSumAggregator.class);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      testValue = in.readInt();
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      out.writeInt(testValue);
+    }
+  }
+
+
+
+}


[39/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Reduce/broadcast API

Summary:
Adding reduce and broadcast API

Reduce and broadcast API should be a simplest comprehensive API for what it and aggregators do,
and has somewhat simpler implementation, since there is no strange interaction with values from
previous and next superstep.

It is very flexible and allows building more complex abstraction on top of it.
Aggregators API is built on top of it in AggregatorToGlobalCommTranslation,
when reverse is not true, which shows it is more flexible then previous API.

Once primitive types diff goes in, it will be trivial to create generic reducers
(instead of having huge number of specialized aggregators)

Current aggregator API has multiple issues:
- no matter whether aggregated values are needed on workers, they are distributed to them
- there is no way to register aggregator for a single superstep
- in order for master to send data to workers, it needs to go through an aggregator - even though it can only be Writable
- value to be aggregated and result of aggregation need to be of the same type
- logic of how to do aggregation is combined with how Aggregator is kept (i.e. aggregate(valueToAggregate), instead of aggregate(currentAggregatedValue, valueToAggregate)), and so every aggregator needs to extend BasicAggregator, but that still limits what can be done.

Related to https://phabricator.fb.com/D1303953

Test Plan:
All unit tests, will run some jobs on production.
Will add unit test to use reduce/broadcast directly, instead of through aggregators

Reviewers: majakabiljo, avery.ching, pavanka, sergey.edunov, maja.kabiljo

Differential Revision: https://reviews.facebook.net/D21423


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/f43f4500
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/f43f4500
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/f43f4500

Branch: refs/heads/release-1.1
Commit: f43f450093876ba8ae164681789cad0e6ea4b68e
Parents: 61db689
Author: Igor Kabiljo <ik...@fb.com>
Authored: Thu Oct 9 14:55:20 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Thu Oct 9 14:55:20 2014 -0700

----------------------------------------------------------------------
 .../giraph/aggregators/AggregatorWrapper.java   | 150 --------
 .../aggregators/ClassAggregatorFactory.java     |  17 +-
 .../giraph/benchmark/ReducersBenchmark.java     | 263 +++++++++++++
 .../giraph/bsp/CentralizedServiceMaster.java    |  20 +-
 .../org/apache/giraph/comm/GlobalCommType.java  |  32 ++
 .../org/apache/giraph/comm/MasterClient.java    |  15 +-
 .../AggregatedValueOutputStream.java            |  45 ---
 .../aggregators/AggregatorOutputStream.java     |  51 ---
 .../comm/aggregators/AggregatorUtils.java       |   6 -
 .../aggregators/AllAggregatorServerData.java    | 117 +++---
 .../GlobalCommValueOutputStream.java            |  71 ++++
 .../aggregators/OwnerAggregatorServerData.java  |  71 ++--
 .../aggregators/SendAggregatedValueCache.java   |  87 -----
 .../comm/aggregators/SendAggregatorCache.java   |  92 -----
 .../comm/aggregators/SendGlobalCommCache.java   | 102 +++++
 .../WorkerAggregatorRequestProcessor.java       |  34 +-
 .../giraph/comm/netty/NettyMasterClient.java    |  32 +-
 .../giraph/comm/netty/NettyMasterServer.java    |  10 +-
 .../NettyWorkerAggregatorRequestProcessor.java  |  44 +--
 .../handler/MasterRequestServerHandler.java     |   4 +-
 .../giraph/comm/requests/RequestType.java       |   2 +-
 .../SendAggregatorsToMasterRequest.java         |  61 ---
 .../requests/SendAggregatorsToOwnerRequest.java |  45 ++-
 .../SendAggregatorsToWorkerRequest.java         |  28 +-
 .../requests/SendReducedToMasterRequest.java    |  61 +++
 .../requests/SendWorkerAggregatorsRequest.java  |  35 +-
 .../giraph/graph/AbstractComputation.java       |  31 +-
 .../org/apache/giraph/graph/Computation.java    |  13 +-
 .../apache/giraph/graph/ComputeCallable.java    |   4 +-
 .../apache/giraph/graph/GraphTaskManager.java   |  40 +-
 .../java/org/apache/giraph/io/EdgeReader.java   |  36 +-
 .../org/apache/giraph/io/MappingReader.java     |  40 +-
 .../java/org/apache/giraph/io/VertexReader.java |  38 +-
 .../giraph/io/internal/WrappedEdgeReader.java   |  13 +-
 .../io/internal/WrappedMappingReader.java       |  14 +-
 .../giraph/io/internal/WrappedVertexReader.java |  11 +-
 .../master/AggregatorReduceOperation.java       |  92 +++++
 .../AggregatorToGlobalCommTranslation.java      | 240 ++++++++++++
 .../apache/giraph/master/BspServiceMaster.java  | 144 ++++---
 .../giraph/master/MasterAggregatorHandler.java  | 371 ++++++++-----------
 .../org/apache/giraph/master/MasterCompute.java |  40 +-
 .../giraph/master/MasterGlobalCommUsage.java    |  68 ++++
 .../giraph/reducers/OnSameReduceOperation.java  |  34 ++
 .../apache/giraph/reducers/ReduceOperation.java |  57 +++
 .../org/apache/giraph/reducers/Reducer.java     | 110 ++++++
 .../apache/giraph/reducers/package-info.java    |  21 ++
 .../apache/giraph/utils/ConfigurationUtils.java |  20 +-
 .../org/apache/giraph/utils/WritableUtils.java  |  34 ++
 .../apache/giraph/worker/BspServiceWorker.java  |  60 +--
 .../giraph/worker/EdgeInputSplitsCallable.java  |  10 +-
 .../worker/MappingInputSplitsCallable.java      |  12 +-
 .../worker/VertexInputSplitsCallable.java       |  10 +-
 .../worker/WorkerAggregatorDelegator.java       |  69 ++++
 .../giraph/worker/WorkerAggregatorHandler.java  | 241 ++++++------
 .../org/apache/giraph/worker/WorkerContext.java |  38 +-
 .../giraph/worker/WorkerGlobalCommUsage.java    |  40 ++
 .../worker/WorkerThreadAggregatorUsage.java     |  31 --
 .../worker/WorkerThreadGlobalCommUsage.java     |  32 ++
 .../java/org/apache/giraph/TestBspBasic.java    |  40 +-
 .../aggregators/TestAggregatorsHandling.java    |  96 +----
 60 files changed, 2104 insertions(+), 1541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
deleted file mode 100644
index fa18a64..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
+++ /dev/null
@@ -1,150 +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.aggregators;
-
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.utils.WritableFactory;
-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 factory */
-  private final WritableFactory<? extends Aggregator<A>> aggregatorFactory;
-  /** 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 aggregatorFactory Aggregator Factory
-   * @param persistent        False iff aggregator should be reset at the end
-   *                          of each super step
-   * @param conf              Configuration
-   */
-  public AggregatorWrapper(
-      WritableFactory<? extends Aggregator<A>> aggregatorFactory,
-      boolean persistent, ImmutableClassesGiraphConfiguration conf) {
-    this.persistent = persistent;
-    this.aggregatorFactory = aggregatorFactory;
-    currentAggregator = aggregatorFactory.create();
-    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 WritableFactory<? extends Aggregator<A>> getAggregatorFactory() {
-    return aggregatorFactory;
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
index 944656e..a022480 100644
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
@@ -21,8 +21,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.utils.WritableFactory;
 import org.apache.giraph.utils.WritableUtils;
@@ -36,7 +34,6 @@ import com.google.common.base.Preconditions;
  * @param <T> Aggregated value type
  */
 public class ClassAggregatorFactory<T extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable
     implements WritableFactory<Aggregator<T>> {
   /** Aggregator class */
   private Class<? extends Aggregator<T>> aggregatorClass;
@@ -51,26 +48,14 @@ public class ClassAggregatorFactory<T extends Writable>
    */
   public ClassAggregatorFactory(
       Class<? extends Aggregator<T>> aggregatorClass) {
-    this(aggregatorClass, null);
-
-  }
-
-  /**
-   * Constructor
-   * @param aggregatorClass Aggregator class
-   * @param conf Configuration
-   */
-  public ClassAggregatorFactory(Class<? extends Aggregator<T>> aggregatorClass,
-      ImmutableClassesGiraphConfiguration conf) {
     Preconditions.checkNotNull(aggregatorClass,
         "aggregatorClass cannot be null in ClassAggregatorFactory");
     this.aggregatorClass = aggregatorClass;
-    setConf(conf);
   }
 
   @Override
   public Aggregator<T> create() {
-    return ReflectionUtils.newInstance(aggregatorClass, getConf());
+    return ReflectionUtils.newInstance(aggregatorClass, null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
new file mode 100644
index 0000000..ce5c96e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
@@ -0,0 +1,263 @@
+/*
+ * 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.benchmark;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.PseudoRandomInputFormatConstants;
+import org.apache.giraph.io.formats.PseudoRandomVertexInputFormat;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.reducers.OnSameReduceOperation;
+import org.apache.giraph.worker.DefaultWorkerContext;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Benchmark for reducers. Also checks the correctness.
+ */
+public class ReducersBenchmark extends GiraphBenchmark {
+  /** Number of reducers setting */
+  private static final String REDUCERS_NUM = "reducersbenchmark.num";
+
+  /** Option for number of reducers */
+  private static final BenchmarkOption REDUCERS =
+      new BenchmarkOption("r", "reducers",
+          true, "Reducers", "Need to set number of reducers (-r)");
+
+  /** LongSumReducer */
+  public static class TestLongSumReducer
+      extends OnSameReduceOperation<LongWritable> {
+    /** Singleton */
+    public static final TestLongSumReducer INSTANCE = new TestLongSumReducer();
+
+    @Override
+    public LongWritable createInitialValue() {
+      return new LongWritable();
+    }
+
+    @Override
+    public void reduceSingle(
+        LongWritable curValue, LongWritable valueToReduce) {
+      curValue.set(curValue.get() + valueToReduce.get());
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+    }
+  }
+
+  /**
+   * Vertex class for ReducersBenchmark
+   */
+  public static class ReducersBenchmarkComputation extends
+      BasicComputation<LongWritable, DoubleWritable, DoubleWritable,
+          DoubleWritable> {
+    @Override
+    public void compute(
+        Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex,
+        Iterable<DoubleWritable> messages) throws IOException {
+      int n = getNumReducers(getConf());
+      long superstep = getSuperstep();
+      int w = getWorkerContextReduced(getConf(), superstep);
+      for (int i = 0; i < n; i++) {
+        reduce("w" + i, new LongWritable((superstep + 1) * i));
+        reduce("p" + i, new LongWritable(i));
+
+        if (superstep > 0) {
+          assertEquals(superstep * (getTotalNumVertices() * i) + w,
+              ((LongWritable) getBroadcast("w" + i)).get());
+          assertEquals(-(superstep * i),
+              ((LongWritable) getBroadcast("m" + i)).get());
+          assertEquals(superstep * getTotalNumVertices() * i,
+              ((LongWritable) getBroadcast("p" + i)).get());
+        }
+      }
+      if (superstep > 2) {
+        vertex.voteToHalt();
+      }
+    }
+  }
+
+  /**
+   * MasterCompute class for ReducersBenchmark
+   */
+  public static class ReducersBenchmarkMasterCompute extends
+      DefaultMasterCompute {
+    @Override
+    public void compute() {
+      int n = getNumReducers(getConf());
+      long superstep = getSuperstep();
+      int w = getWorkerContextReduced(getConf(), superstep);
+      for (int i = 0; i < n; i++) {
+        String wi = "w" + i;
+        String mi = "m" + i;
+        String pi = "p" + i;
+
+        registerReduce(wi, TestLongSumReducer.INSTANCE);
+        registerReduce(mi, new TestLongSumReducer());
+
+        if (superstep > 0) {
+          broadcast(wi, getReduced(wi));
+          broadcast(mi, new LongWritable(-superstep * i));
+          broadcast(pi, getReduced(pi));
+
+          registerReduce(pi, new TestLongSumReducer(),
+              (LongWritable) getReduced(pi));
+
+          assertEquals(superstep * (getTotalNumVertices() * i) + w,
+              ((LongWritable) getReduced(wi)).get());
+          assertEquals(superstep * getTotalNumVertices() * i,
+              ((LongWritable) getReduced(pi)).get());
+        } else {
+          registerReduce(pi, new TestLongSumReducer());
+        }
+      }
+    }
+  }
+
+  /**
+   * WorkerContext class for ReducersBenchmark
+   */
+  public static class ReducersBenchmarkWorkerContext
+      extends DefaultWorkerContext {
+    @Override
+    public void preSuperstep() {
+      addToWorkerReducers(1);
+      checkReducers();
+    }
+
+    @Override
+    public void postSuperstep() {
+      addToWorkerReducers(2);
+      checkReducers();
+    }
+
+    /**
+     * Check if reducer values are correct for current superstep
+     */
+    private void checkReducers() {
+      int n = getNumReducers(getContext().getConfiguration());
+      long superstep = getSuperstep();
+      int w = getWorkerContextReduced(
+          getContext().getConfiguration(), superstep);
+      for (int i = 0; i < n; i++) {
+        if (superstep > 0) {
+          assertEquals(superstep * (getTotalNumVertices() * i) + w,
+              ((LongWritable) getBroadcast("w" + i)).get());
+          assertEquals(-(superstep * i),
+              ((LongWritable) getBroadcast("m" + i)).get());
+          assertEquals(superstep * getTotalNumVertices() * i,
+              ((LongWritable) getBroadcast("p" + i)).get());
+        }
+      }
+    }
+
+    /**
+     * Add some value to worker reducers.
+     *
+     * @param valueToAdd Which value to add
+     */
+    private void addToWorkerReducers(int valueToAdd) {
+      int n = getNumReducers(getContext().getConfiguration());
+      for (int i = 0; i < n; i++) {
+        reduce("w" + i, new LongWritable(valueToAdd));
+      }
+    }
+  }
+
+  /**
+   * Get the number of reducers from configuration
+   *
+   * @param conf Configuration
+   * @return Number of reducers
+   */
+  private static int getNumReducers(Configuration conf) {
+    return conf.getInt(REDUCERS_NUM, 0);
+  }
+
+  /**
+   * Get the value which should be reduced by worker context
+   *
+   * @param conf Configuration
+   * @param superstep Superstep
+   * @return The value which should be reduced by worker context
+   */
+  private static int getWorkerContextReduced(Configuration conf,
+      long superstep) {
+    return (superstep <= 0) ? 0 : conf.getInt("workers", 0) * 3;
+  }
+
+  /**
+   * Check if values are equal, throw an exception if they aren't
+   *
+   * @param expected Expected value
+   * @param actual Actual value
+   */
+  private static void assertEquals(long expected, long actual) {
+    if (expected != actual) {
+      throw new RuntimeException("expected: " + expected +
+          ", actual: " + actual);
+    }
+  }
+
+  @Override
+  public Set<BenchmarkOption> getBenchmarkOptions() {
+    return Sets.newHashSet(BenchmarkOption.VERTICES, REDUCERS);
+  }
+
+  @Override
+  protected void prepareConfiguration(GiraphConfiguration conf,
+      CommandLine cmd) {
+    conf.setComputationClass(ReducersBenchmarkComputation.class);
+    conf.setMasterComputeClass(ReducersBenchmarkMasterCompute.class);
+    conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
+    conf.setWorkerContextClass(ReducersBenchmarkWorkerContext.class);
+    conf.setLong(PseudoRandomInputFormatConstants.AGGREGATE_VERTICES,
+        BenchmarkOption.VERTICES.getOptionLongValue(cmd));
+    conf.setLong(PseudoRandomInputFormatConstants.EDGES_PER_VERTEX, 1);
+    conf.setInt(REDUCERS_NUM, REDUCERS.getOptionIntValue(cmd));
+    conf.setInt("workers", conf.getInt(GiraphConstants.MAX_WORKERS, -1));
+  }
+
+  /**
+   * Execute the benchmark.
+   *
+   * @param args Typically the command line arguments.
+   * @throws Exception Any exception from the computation.
+   */
+  public static void main(final String[] args) throws Exception {
+    System.exit(ToolRunner.run(new ReducersBenchmark(), args));
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
index 9b4f9d6..1e8d519 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
@@ -18,6 +18,10 @@
 
 package org.apache.giraph.bsp;
 
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.giraph.master.AggregatorToGlobalCommTranslation;
 import org.apache.giraph.master.MasterAggregatorHandler;
 import org.apache.giraph.master.MasterCompute;
 import org.apache.giraph.master.MasterInfo;
@@ -26,9 +30,6 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.zookeeper.KeeperException;
 
-import java.io.IOException;
-import java.util.List;
-
 /**
  * At most, there will be one active master at a time, but many threads can
  * be trying to be the active master.
@@ -139,11 +140,18 @@ public interface CentralizedServiceMaster<I extends WritableComparable,
     long desiredSuperstep);
 
   /**
-   * Get master aggregator handler
+   * Get handler for global communication
+   *
+   * @return Global communication handler
+   */
+  MasterAggregatorHandler getGlobalCommHandler();
+
+  /**
+   * Handler for aggregators to reduce/broadcast translation
    *
-   * @return Master aggregator handler
+   * @return aggregator translation handler
    */
-  MasterAggregatorHandler getAggregatorHandler();
+  AggregatorToGlobalCommTranslation getAggregatorTranslationHandler();
 
   /**
    * Get MasterCompute object

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/GlobalCommType.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/GlobalCommType.java b/giraph-core/src/main/java/org/apache/giraph/comm/GlobalCommType.java
new file mode 100644
index 0000000..539b3bd
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/GlobalCommType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.comm;
+
+/**
+ * Type tag distinguishing different global communication messages.
+ */
+public enum GlobalCommType {
+  /** ReduceOperation object */
+  REDUCE_OPERATIONS,
+  /** Reduced value object */
+  REDUCED_VALUE,
+  /** Broadcasted value */
+  BROADCAST,
+  /** Special count used internally for counting requests */
+  SPECIAL_COUNT;
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
index b7718a7..aea93fd 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
@@ -20,8 +20,6 @@ package org.apache.giraph.comm;
 
 import java.io.IOException;
 
-import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -36,19 +34,18 @@ public interface MasterClient {
   /**
    * Sends aggregator to its owner
    *
-   * @param aggregatorName Name of the aggregator
-   * @param aggregatorFactory Aggregator factory
-   * @param aggregatedValue Value of the aggregator
+   * @param name Name of the object
+   * @param type Global communication type
+   * @param value Object value
    * @throws IOException
    */
-  void sendAggregator(String aggregatorName,
-      WritableFactory<? extends Aggregator> aggregatorFactory,
-      Writable aggregatedValue) throws IOException;
+  void sendToOwner(String name, GlobalCommType type, Writable value)
+    throws IOException;
 
   /**
    * Flush aggregated values cache.
    */
-  void finishSendingAggregatedValues() throws IOException;
+  void finishSendingValues() throws IOException;
 
   /**
    * Flush all outgoing messages.  This will synchronously ensure that all

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatedValueOutputStream.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatedValueOutputStream.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatedValueOutputStream.java
deleted file mode 100644
index 0010dba..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatedValueOutputStream.java
+++ /dev/null
@@ -1,45 +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.comm.aggregators;
-
-import org.apache.hadoop.io.Writable;
-
-import java.io.IOException;
-
-/**
- * Implementation of {@link CountingOutputStream} which allows writing of
- * aggregator values in the form of pair (name, value)
- */
-public class AggregatedValueOutputStream extends CountingOutputStream {
-  /**
-   * Write aggregator to the stream and increment internal counter
-   *
-   * @param aggregatorName Name of the aggregator
-   * @param aggregatedValue Value of aggregator
-   * @return Number of bytes occupied by the stream
-   * @throws IOException
-   */
-  public int addAggregator(String aggregatorName,
-      Writable aggregatedValue) throws IOException {
-    incrementCounter();
-    dataOutput.writeUTF(aggregatorName);
-    aggregatedValue.write(dataOutput);
-    return getSize();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
deleted file mode 100644
index 79bc08a..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
+++ /dev/null
@@ -1,51 +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.comm.aggregators;
-
-import java.io.IOException;
-
-import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.utils.WritableFactory;
-import org.apache.giraph.utils.WritableUtils;
-import org.apache.hadoop.io.Writable;
-
-/**
- * Implementation of {@link CountingOutputStream} which allows writing of
- * aggregators in the form of triple (name, classname, value)
- */
-public class AggregatorOutputStream extends CountingOutputStream {
-  /**
-   * Write aggregator to the stream and increment internal counter
-   *
-   * @param aggregatorName Name of the aggregator
-   * @param aggregatorFactory Aggregator factory
-   * @param aggregatedValue Value of aggregator
-   * @return Number of bytes occupied by the stream
-   * @throws IOException
-   */
-  public int addAggregator(String aggregatorName,
-      WritableFactory<? extends Aggregator> aggregatorFactory,
-      Writable aggregatedValue) throws IOException {
-    incrementCounter();
-    dataOutput.writeUTF(aggregatorName);
-    WritableUtils.writeWritableObject(aggregatorFactory, dataOutput);
-    aggregatedValue.write(dataOutput);
-    return getSize();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
index a94ab38..ecb3a6b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
@@ -27,12 +27,6 @@ import org.apache.giraph.worker.WorkerInfo;
  * Class for aggregator constants and utility methods
  */
 public class AggregatorUtils {
-  /**
-   * Special aggregator name which will be used to send the total number of
-   * aggregators requests which should arrive
-   */
-  public static final String SPECIAL_COUNT_AGGREGATOR =
-      "__aggregatorRequestCount";
 
   /** How big a single aggregator request can be (in bytes) */
   public static final String MAX_BYTES_PER_AGGREGATOR_REQUEST =

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
index effc9bf..b1c0781 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
@@ -21,19 +21,21 @@ package org.apache.giraph.comm.aggregators;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.master.MasterInfo;
-import org.apache.giraph.utils.Factory;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.reducers.Reducer;
 import org.apache.giraph.utils.TaskIdsPermitsBarrier;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -51,12 +53,12 @@ public class AllAggregatorServerData {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(AllAggregatorServerData.class);
-  /** Map of aggregator factories */
-  private final ConcurrentMap<String, WritableFactory<Aggregator<Writable>>>
-  aggregatorFactoriesMap = Maps.newConcurrentMap();
-  /** Map of values of aggregators from previous superstep */
+  /** Map of broadcasted values from master */
   private final ConcurrentMap<String, Writable>
-  aggregatedValuesMap = Maps.newConcurrentMap();
+  broadcastedMap = Maps.newConcurrentMap();
+  /** Map of registered reducers for current superstep */
+  private final ConcurrentMap<String, ReduceOperation<Object, Writable>>
+  reduceOpMap = Maps.newConcurrentMap();
   /**
    * Counts the requests with final aggregators from master.
    * It uses values from special aggregators
@@ -97,54 +99,36 @@ public class AllAggregatorServerData {
   }
 
   /**
-   * Register the class of the aggregator, received by master or worker.
-   *
-   * @param name              Aggregator name
-   * @param aggregatorFactory Aggregator factory
-   */
-  public void registerAggregatorClass(String name,
-      WritableFactory<Aggregator<Writable>> aggregatorFactory) {
-    aggregatorFactoriesMap.put(name, aggregatorFactory);
-    progressable.progress();
-  }
-
-  /**
-   * Set the value of aggregator from previous superstep,
-   * received by master or worker.
-   *
-   * @param name Name of the aggregator
-   * @param value Value of the aggregator
-   */
-  public void setAggregatorValue(String name, Writable value) {
-    aggregatedValuesMap.put(name, value);
-    progressable.progress();
-  }
-
-  /**
-   * Create initial aggregated value for an aggregator. Used so requests
-   * would be able to deserialize data.
-   * registerAggregatorClass needs to be called first to ensure that we have
-   * the class of the aggregator.
-   *
-   * @param name Name of the aggregator
-   * @return Empty aggregated value for this aggregator
+   * Received value through global communication from master.
+   * @param name Name
+   * @param type Global communication type
+   * @param value Object value
    */
-  public Writable createAggregatorInitialValue(String name) {
-    WritableFactory<Aggregator<Writable>> aggregatorFactory =
-        aggregatorFactoriesMap.get(name);
-    synchronized (aggregatorFactory) {
-      return aggregatorFactory.create().createInitialValue();
+  public void receiveValueFromMaster(
+      String name, GlobalCommType type, Writable value) {
+    switch (type) {
+    case BROADCAST:
+      broadcastedMap.put(name, value);
+      break;
+
+    case REDUCE_OPERATIONS:
+      reduceOpMap.put(name, (ReduceOperation<Object, Writable>) value);
+      break;
+
+    default:
+      throw new IllegalArgumentException("Unkown request type " + type);
     }
+    progressable.progress();
   }
 
   /**
    * Notify this object that an aggregator request from master has been
    * received.
    *
-   * @param aggregatorData Byte request with data received from master
+   * @param data Byte request with data received from master
    */
-  public void receivedRequestFromMaster(byte[] aggregatorData) {
-    masterData.add(aggregatorData);
+  public void receivedRequestFromMaster(byte[] data) {
+    masterData.add(data);
     masterBarrier.releaseOnePermit();
   }
 
@@ -200,35 +184,32 @@ public class AllAggregatorServerData {
    * arrived, and fill the maps for next superstep when ready.
    *
    * @param workerIds All workers in the job apart from the current one
-   * @param previousAggregatedValuesMap Map of values from previous
-   *                                    superstep to fill out
-   * @param currentAggregatorFactoryMap Map of aggregators factories for
-   *                                    current superstep to fill out.
+   * @param broadcastedMapToFill Broadcast map to fill out
+   * @param reducerMapToFill Registered reducer map to fill out.
    */
   public void fillNextSuperstepMapsWhenReady(
       Set<Integer> workerIds,
-      Map<String, Writable> previousAggregatedValuesMap,
-      Map<String, Factory<Aggregator<Writable>>> currentAggregatorFactoryMap) {
+      Map<String, Writable> broadcastedMapToFill,
+      Map<String, Reducer<Object, Writable>> reducerMapToFill) {
     workersBarrier.waitForRequiredPermits(workerIds);
     if (LOG.isDebugEnabled()) {
-      LOG.debug("fillNextSuperstepMapsWhenReady: Aggregators ready");
+      LOG.debug("fillNextSuperstepMapsWhenReady: Global data ready");
     }
-    previousAggregatedValuesMap.clear();
-    previousAggregatedValuesMap.putAll(aggregatedValuesMap);
-    for (Map.Entry<String, WritableFactory<Aggregator<Writable>>> entry :
-        aggregatorFactoriesMap.entrySet()) {
-      Factory<Aggregator<Writable>> aggregatorFactory =
-          currentAggregatorFactoryMap.get(entry.getKey());
-      if (aggregatorFactory == null) {
-        currentAggregatorFactoryMap.put(entry.getKey(), entry.getValue());
-      }
+
+    Preconditions.checkArgument(broadcastedMapToFill.isEmpty(),
+        "broadcastedMap needs to be empty for filling");
+    Preconditions.checkArgument(reducerMapToFill.isEmpty(),
+        "reducerMap needs to be empty for filling");
+
+    broadcastedMapToFill.putAll(broadcastedMap);
+
+    for (Entry<String, ReduceOperation<Object, Writable>> entry :
+        reduceOpMap.entrySet()) {
+      reducerMapToFill.put(entry.getKey(), new Reducer<>(entry.getValue()));
     }
-  }
 
-  /**
-   * Prepare for next superstep
-   */
-  public void reset() {
+    broadcastedMap.clear();
+    reduceOpMap.clear();
     masterData.clear();
     if (LOG.isDebugEnabled()) {
       LOG.debug("reset: Ready for next superstep");

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/GlobalCommValueOutputStream.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/GlobalCommValueOutputStream.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/GlobalCommValueOutputStream.java
new file mode 100644
index 0000000..0add1e9
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/GlobalCommValueOutputStream.java
@@ -0,0 +1,71 @@
+/*
+ * 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.comm.aggregators;
+
+import java.io.IOException;
+
+import org.apache.giraph.comm.GlobalCommType;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Implementation of {@link CountingOutputStream} which allows writing of
+ * reduced values in the form of pair (name, type, value)
+ *
+ * There are two modes:
+ * - when class of the value is written into the stream.
+ * - when it isn't, and reader needs to know Class of the value in order
+ *   to read it.
+ */
+public class GlobalCommValueOutputStream extends CountingOutputStream {
+  /** whether to write Class object for values into the stream */
+  private final boolean writeClass;
+
+  /**
+   * Constructor
+   *
+   * @param writeClass boolean whether to write Class object for values
+   */
+  public GlobalCommValueOutputStream(boolean writeClass) {
+    this.writeClass = writeClass;
+  }
+
+  /**
+   * Write global communication object to the stream
+   * and increment internal counter
+   *
+   * @param name Name
+   * @param type Global communication type
+   * @param value Object value
+   * @return Number of bytes occupied by the stream
+   * @throws IOException
+   */
+  public int addValue(String name, GlobalCommType type,
+      Writable value) throws IOException {
+    incrementCounter();
+    dataOutput.writeUTF(name);
+    dataOutput.writeByte(type.ordinal());
+    if (writeClass) {
+      WritableUtils.writeWritableObject(value, dataOutput);
+    } else {
+      value.write(dataOutput);
+    }
+    return getSize();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
index 2f3d5e5..9e92efc 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
@@ -23,9 +23,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.reducers.Reducer;
 import org.apache.giraph.utils.TaskIdsPermitsBarrier;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -62,13 +62,12 @@ public class OwnerAggregatorServerData {
   private static final Logger LOG =
       Logger.getLogger(OwnerAggregatorServerData.class);
   /** Map of aggregators which current worker owns */
-  private final ConcurrentMap<String, Aggregator<Writable>>
-  myAggregatorMap = Maps.newConcurrentMap();
+  private final ConcurrentMap<String, Reducer<Object, Writable>>
+  myReducerMap = Maps.newConcurrentMap();
   /**
    * Counts the requests with partial aggregated values from other workers.
-   * It uses values from special aggregators
-   * (named AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)
-   * to know how many requests it has to receive.
+   * It uses GlobalCommType.SPECIAL_COUNT to know how many requests it
+   * has to receive.
    */
   private final TaskIdsPermitsBarrier workersBarrier;
   /** Progressable used to report progress */
@@ -85,49 +84,50 @@ public class OwnerAggregatorServerData {
   }
 
   /**
-   * Register an aggregator which current worker owns. Thread-safe.
+   * Register a reducer which current worker owns. Thread-safe.
    *
    * @param name Name of aggregator
-   * @param aggregatorFactory Aggregator factory
+   * @param reduceOp Reduce operation
    */
-  public void registerAggregator(String name,
-      WritableFactory<Aggregator<Writable>> aggregatorFactory) {
-    if (LOG.isDebugEnabled() && myAggregatorMap.isEmpty()) {
+  public void registerReducer(String name,
+      ReduceOperation<Object, Writable> reduceOp) {
+    if (LOG.isDebugEnabled() && myReducerMap.isEmpty()) {
       LOG.debug("registerAggregator: The first registration after a reset()");
     }
-    myAggregatorMap.putIfAbsent(name, aggregatorFactory.create());
+    myReducerMap.putIfAbsent(name, new Reducer<>(reduceOp));
     progressable.progress();
   }
 
   /**
-   * Aggregate partial value of one of current worker's aggregators.
+   * Reduce partial value of one of current worker's reducers.
    *
-   * Thread-safe. Call only after aggregators have been registered.
+   * Thread-safe. Call only after reducers have been registered.
    *
-   * @param name Name of the aggregator
-   * @param value Value to aggregate to it
+   * @param name Name of the reducer
+   * @param value Value to reduce to it
    */
-  public void aggregate(String name, Writable value) {
-    Aggregator<Writable> aggregator = myAggregatorMap.get(name);
-    synchronized (aggregator) {
-      aggregator.aggregate(value);
+  public void reduce(String name, Writable value) {
+    Reducer<Object, Writable> reducer = myReducerMap.get(name);
+    synchronized (reducer) {
+      reducer.reducePartial(value);
     }
     progressable.progress();
   }
 
+
   /**
-   * Create initial aggregated value for an aggregator. Used so requests
+   * Create initial value for a reducer. Used so requests
    * would be able to deserialize data.
    *
-   * Thread-safe. Call only after aggregators have been registered.
+   * Thread-safe. Call only after reducer has been registered.
    *
-   * @param name Name of the aggregator
-   * @return Empty aggregated value for this aggregator
+   * @param name Name of the reducer
+   * @return Empty value
    */
-  public Writable createAggregatorInitialValue(String name) {
-    Aggregator<Writable> aggregator = myAggregatorMap.get(name);
-    synchronized (aggregator) {
-      return aggregator.createInitialValue();
+  public Writable createInitialValue(String name) {
+    Reducer<Object, Writable> reducer = myReducerMap.get(name);
+    synchronized (reducer) {
+      return reducer.createInitialValue();
     }
   }
 
@@ -159,20 +159,20 @@ public class OwnerAggregatorServerData {
    * @return Iterable through final aggregated values which this worker owns
    */
   public Iterable<Map.Entry<String, Writable>>
-  getMyAggregatorValuesWhenReady(Set<Integer> workerIds) {
+  getMyReducedValuesWhenReady(Set<Integer> workerIds) {
     workersBarrier.waitForRequiredPermits(workerIds);
     if (LOG.isDebugEnabled()) {
       LOG.debug("getMyAggregatorValuesWhenReady: Values ready");
     }
-    return Iterables.transform(myAggregatorMap.entrySet(),
-        new Function<Map.Entry<String, Aggregator<Writable>>,
+    return Iterables.transform(myReducerMap.entrySet(),
+        new Function<Map.Entry<String, Reducer<Object, Writable>>,
             Map.Entry<String, Writable>>() {
           @Override
           public Map.Entry<String, Writable> apply(
-              Map.Entry<String, Aggregator<Writable>> aggregator) {
+              Map.Entry<String, Reducer<Object, Writable>> aggregator) {
             return new AbstractMap.SimpleEntry<String, Writable>(
                 aggregator.getKey(),
-                aggregator.getValue().getAggregatedValue());
+                aggregator.getValue().getCurrentValue());
           }
         });
   }
@@ -181,9 +181,10 @@ public class OwnerAggregatorServerData {
    * Prepare for next superstep
    */
   public void reset() {
-    myAggregatorMap.clear();
+    myReducerMap.clear();
     if (LOG.isDebugEnabled()) {
       LOG.debug("reset: Ready for next superstep");
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatedValueCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatedValueCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatedValueCache.java
deleted file mode 100644
index 468ee5c..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatedValueCache.java
+++ /dev/null
@@ -1,87 +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.comm.aggregators;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-
-import com.google.common.collect.Maps;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Takes and serializes aggregated values and keeps them grouped by owner
- * partition id, to be sent in bulk.
- */
-public class SendAggregatedValueCache extends CountingCache {
-  /** Map from worker partition id to aggregator output stream */
-  private final Map<Integer, AggregatedValueOutputStream> aggregatorMap =
-      Maps.newHashMap();
-
-  /**
-   * Add aggregated value to the cache
-   *
-   * @param taskId Task id of worker which owns the aggregator
-   * @param aggregatorName Name of the aggregator
-   * @param aggregatedValue Value of the aggregator
-   * @return Number of bytes in serialized data for this worker
-   * @throws IOException
-   */
-  public int addAggregator(Integer taskId, String aggregatorName,
-      Writable aggregatedValue) throws IOException {
-    AggregatedValueOutputStream out = aggregatorMap.get(taskId);
-    if (out == null) {
-      out = new AggregatedValueOutputStream();
-      aggregatorMap.put(taskId, out);
-    }
-    return out.addAggregator(aggregatorName, aggregatedValue);
-  }
-
-  /**
-   * Remove and get aggregators for certain worker
-   *
-   * @param taskId Partition id of worker owner
-   * @return Serialized aggregator data for this worker
-   */
-  public byte[] removeAggregators(Integer taskId) {
-    incrementCounter(taskId);
-    AggregatedValueOutputStream out = aggregatorMap.remove(taskId);
-    if (out == null) {
-      return new byte[4];
-    } else {
-      return out.flush();
-    }
-  }
-
-  /**
-   * Creates fake aggregator which will hold the total number of aggregator
-   * requests for worker with selected task id. This should be called after all
-   * aggregators for the worker have been added to the cache.
-   *
-   * @param taskId Destination worker's task id
-   * @throws IOException
-   */
-  public void addCountAggregator(Integer taskId) throws IOException {
-    // current number of requests, plus one for the last flush
-    long totalCount = getCount(taskId) + 1;
-    addAggregator(taskId, AggregatorUtils.SPECIAL_COUNT_AGGREGATOR,
-        new LongWritable(totalCount));
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
deleted file mode 100644
index 8f880b4..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
+++ /dev/null
@@ -1,92 +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.comm.aggregators;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.utils.WritableFactory;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-
-import com.google.common.collect.Maps;
-
-/**
- * Takes and serializes aggregators and keeps them grouped by owner
- * partition id, to be sent in bulk.
- */
-public class SendAggregatorCache extends CountingCache {
-  /** Map from worker partition id to aggregator output stream */
-  private final Map<Integer, AggregatorOutputStream> aggregatorMap =
-      Maps.newHashMap();
-
-  /**
-   * Add aggregator to the cache
-   *
-   * @param taskId Task id of worker which owns the aggregator
-   * @param aggregatorName Name of the aggregator
-   * @param aggregatorFactory Aggregator factory
-   * @param aggregatedValue Value of the aggregator
-   * @return Number of bytes in serialized data for this worker
-   * @throws IOException
-   */
-  public int addAggregator(Integer taskId, String aggregatorName,
-      WritableFactory<? extends Aggregator> aggregatorFactory,
-      Writable aggregatedValue) throws IOException {
-    AggregatorOutputStream out = aggregatorMap.get(taskId);
-    if (out == null) {
-      out = new AggregatorOutputStream();
-      aggregatorMap.put(taskId, out);
-    }
-    return out.addAggregator(aggregatorName, aggregatorFactory,
-        aggregatedValue);
-  }
-
-  /**
-   * Remove and get aggregators for certain worker
-   *
-   * @param taskId Task id of worker owner
-   * @return Serialized aggregator data for this worker
-   */
-  public byte[] removeAggregators(Integer taskId) {
-    incrementCounter(taskId);
-    AggregatorOutputStream out = aggregatorMap.remove(taskId);
-    if (out == null) {
-      return new byte[4];
-    } else {
-      return out.flush();
-    }
-  }
-
-  /**
-   * Creates fake aggregator which will hold the total number of aggregator
-   * requests for worker with selected task id. This should be called after all
-   * aggregators for the worker have been added to the cache.
-   *
-   * @param taskId Destination worker's task id
-   * @throws IOException
-   */
-  public void addCountAggregator(Integer taskId) throws IOException {
-    // current number of requests, plus one for the last flush
-    long totalCount = getCount(taskId) + 1;
-    addAggregator(taskId, AggregatorUtils.SPECIAL_COUNT_AGGREGATOR,
-        null, new LongWritable(totalCount));
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendGlobalCommCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendGlobalCommCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendGlobalCommCache.java
new file mode 100644
index 0000000..5e10c2f
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendGlobalCommCache.java
@@ -0,0 +1,102 @@
+/*
+ * 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.comm.aggregators;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.giraph.comm.GlobalCommType;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Takes and serializes global communication values and keeps them grouped by
+ * owner partition id, to be sent in bulk.
+ * Includes broadcast messages, reducer registrations and special count.
+ */
+public class SendGlobalCommCache extends CountingCache {
+  /** Map from worker partition id to global communication output stream */
+  private final Map<Integer, GlobalCommValueOutputStream> globalCommMap =
+      Maps.newHashMap();
+
+  /** whether to write Class object for values into the stream */
+  private final boolean writeClass;
+
+  /**
+   * Constructor
+   *
+   * @param writeClass boolean whether to write Class object for values
+   */
+  public SendGlobalCommCache(boolean writeClass) {
+    this.writeClass = writeClass;
+  }
+
+  /**
+   * Add global communication value to the cache
+   *
+   * @param taskId Task id of worker which owns the value
+   * @param name Name
+   * @param type Global communication type
+   * @param value Value
+   * @return Number of bytes in serialized data for this worker
+   * @throws IOException
+   */
+  public int addValue(Integer taskId, String name,
+      GlobalCommType type, Writable value) throws IOException {
+    GlobalCommValueOutputStream out = globalCommMap.get(taskId);
+    if (out == null) {
+      out = new GlobalCommValueOutputStream(writeClass);
+      globalCommMap.put(taskId, out);
+    }
+    return out.addValue(name, type, value);
+  }
+
+  /**
+   * Remove and get values for certain worker
+   *
+   * @param taskId Partition id of worker owner
+   * @return Serialized global communication data for this worker
+   */
+  public byte[] removeSerialized(Integer taskId) {
+    incrementCounter(taskId);
+    GlobalCommValueOutputStream out = globalCommMap.remove(taskId);
+    if (out == null) {
+      return new byte[4];
+    } else {
+      return out.flush();
+    }
+  }
+
+  /**
+   * Creates special value which will hold the total number of global
+   * communication requests for worker with selected task id. This should be
+   * called after all values for the worker have been added to the cache.
+   *
+   * @param taskId Destination worker's task id
+   * @throws IOException
+   */
+  public void addSpecialCount(Integer taskId) throws IOException {
+    // current number of requests, plus one for the last flush
+    long totalCount = getCount(taskId) + 1;
+    addValue(taskId, GlobalCommType.SPECIAL_COUNT.name(),
+        GlobalCommType.SPECIAL_COUNT, new LongWritable(totalCount));
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/WorkerAggregatorRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/WorkerAggregatorRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/WorkerAggregatorRequestProcessor.java
index 360a39b..42009a2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/WorkerAggregatorRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/WorkerAggregatorRequestProcessor.java
@@ -18,25 +18,25 @@
 
 package org.apache.giraph.comm.aggregators;
 
-import org.apache.hadoop.io.Writable;
-
 import java.io.IOException;
 
+import org.apache.hadoop.io.Writable;
+
 /**
  * Aggregates worker aggregator requests and sends them off
  */
 public interface WorkerAggregatorRequestProcessor {
   /**
-   * Sends worker aggregated value to the owner of aggregator
+   * Sends worker reduced value to the owner of reducer
    *
-   * @param aggregatorName Name of the aggregator
-   * @param aggregatedValue Value of the aggregator
+   * @param name Name of the reducer
+   * @param reducedValue Reduced partial value
    * @throws java.io.IOException
-   * @return True if aggregated value will be sent, false if this worker is
-   * the owner of the aggregator
+   * @return True if reduced value will be sent, false if this worker is
+   * the owner of the reducer
    */
-  boolean sendAggregatedValue(String aggregatorName,
-      Writable aggregatedValue) throws IOException;
+  boolean sendReducedValue(String name,
+      Writable reducedValue) throws IOException;
 
   /**
    * Flush aggregated values cache.
@@ -46,19 +46,19 @@ public interface WorkerAggregatorRequestProcessor {
   void flush() throws IOException;
 
   /**
-   * Sends aggregated values to the master. This worker is the owner of these
-   * aggregators.
+   * Sends reduced values to the master. This worker is the owner of these
+   * reducers.
    *
-   * @param aggregatorData Serialized aggregator data
+   * @param data Serialized reduced values data
    * @throws IOException
    */
-  void sendAggregatedValuesToMaster(byte[] aggregatorData) throws IOException;
+  void sendReducedValuesToMaster(byte[] data) throws IOException;
 
   /**
-   * Sends aggregators to all other workers
+   * Sends reduced values to all other workers
    *
-   * @param aggregatorDataList Serialized aggregator data split into chunks
+   * @param reducedDataList Serialized reduced values data split into chunks
    */
-  void distributeAggregators(
-      Iterable<byte[]> aggregatorDataList) throws IOException;
+  void distributeReducedValues(
+      Iterable<byte[]> reducedDataList) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/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 51277c9..e110782 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
@@ -20,14 +20,13 @@ package org.apache.giraph.comm.netty;
 
 import java.io.IOException;
 
-import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
+import org.apache.giraph.comm.GlobalCommType;
 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.aggregators.SendGlobalCommCache;
 import org.apache.giraph.comm.requests.SendAggregatorsToOwnerRequest;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -42,8 +41,8 @@ public class NettyMasterClient implements MasterClient {
   /** Worker information for current superstep */
   private final CentralizedServiceMaster<?, ?, ?> service;
   /** Cached map of partition ids to serialized aggregator data */
-  private final SendAggregatorCache sendAggregatorCache =
-      new SendAggregatorCache();
+  private final SendGlobalCommCache sendGlobalCommCache =
+      new SendGlobalCommCache(true);
   /** How big a single aggregator request can be */
   private final int maxBytesPerAggregatorRequest;
   /** Progressable used to report progress */
@@ -78,26 +77,25 @@ public class NettyMasterClient implements MasterClient {
   }
 
   @Override
-  public void sendAggregator(String aggregatorName,
-      WritableFactory<? extends Aggregator> aggregatorFactory,
-      Writable aggregatedValue) throws IOException {
+  public void sendToOwner(String name, GlobalCommType sendType, Writable object)
+    throws IOException {
     WorkerInfo owner =
-        AggregatorUtils.getOwner(aggregatorName, service.getWorkerInfoList());
-    int currentSize = sendAggregatorCache.addAggregator(owner.getTaskId(),
-        aggregatorName, aggregatorFactory, aggregatedValue);
+        AggregatorUtils.getOwner(name, service.getWorkerInfoList());
+    int currentSize = sendGlobalCommCache.addValue(owner.getTaskId(),
+        name, sendType, object);
     if (currentSize >= maxBytesPerAggregatorRequest) {
       flushAggregatorsToWorker(owner);
     }
   }
 
   @Override
-  public void finishSendingAggregatedValues() throws IOException {
+  public void finishSendingValues() throws IOException {
     for (WorkerInfo worker : service.getWorkerInfoList()) {
-      sendAggregatorCache.addCountAggregator(worker.getTaskId());
+      sendGlobalCommCache.addSpecialCount(worker.getTaskId());
       flushAggregatorsToWorker(worker);
       progressable.progress();
     }
-    sendAggregatorCache.reset();
+    sendGlobalCommCache.reset();
   }
 
   /**
@@ -106,10 +104,10 @@ public class NettyMasterClient implements MasterClient {
    * @param worker Worker which we want to send aggregators to
    */
   private void flushAggregatorsToWorker(WorkerInfo worker) {
-    byte[] aggregatorData =
-        sendAggregatorCache.removeAggregators(worker.getTaskId());
+    byte[] data =
+        sendGlobalCommCache.removeSerialized(worker.getTaskId());
     nettyClient.sendWritableRequest(
-        worker.getTaskId(), new SendAggregatorsToOwnerRequest(aggregatorData,
+        worker.getTaskId(), new SendAggregatorsToOwnerRequest(data,
           service.getMasterInfo().getTaskId()));
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
index 1c05910..60566f9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
@@ -18,14 +18,14 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import java.net.InetSocketAddress;
+
 import org.apache.giraph.bsp.CentralizedServiceMaster;
-import org.apache.giraph.comm.netty.handler.MasterRequestServerHandler;
 import org.apache.giraph.comm.MasterServer;
+import org.apache.giraph.comm.netty.handler.MasterRequestServerHandler;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.hadoop.util.Progressable;
 
-import java.net.InetSocketAddress;
-
 /**
  * Netty implementation of {@link MasterServer}
  */
@@ -46,7 +46,7 @@ public class NettyMasterServer implements MasterServer {
       Progressable progressable,
       Thread.UncaughtExceptionHandler exceptionHandler) {
     nettyServer = new NettyServer(conf,
-        new MasterRequestServerHandler.Factory(service.getAggregatorHandler()),
+        new MasterRequestServerHandler.Factory(service.getGlobalCommHandler()),
         service.getMasterInfo(), progressable, exceptionHandler);
     nettyServer.start();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/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 8b5f293..3096c6e 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
@@ -18,21 +18,22 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import java.io.IOException;
+
 import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
-import org.apache.giraph.comm.aggregators.SendAggregatedValueCache;
-import org.apache.giraph.comm.requests.SendAggregatorsToMasterRequest;
+import org.apache.giraph.comm.aggregators.SendGlobalCommCache;
+import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
 import org.apache.giraph.comm.requests.SendAggregatorsToWorkerRequest;
+import org.apache.giraph.comm.requests.SendReducedToMasterRequest;
 import org.apache.giraph.comm.requests.SendWorkerAggregatorsRequest;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 
-import java.io.IOException;
-
 /**
  * Netty implementation of {@link WorkerAggregatorRequestProcessor}
  */
@@ -45,8 +46,8 @@ public class NettyWorkerAggregatorRequestProcessor
   /** Service worker */
   private final CentralizedServiceWorker<?, ?, ?> serviceWorker;
   /** Cached map of partition ids to serialized aggregator data */
-  private final SendAggregatedValueCache sendAggregatedValueCache =
-      new SendAggregatedValueCache();
+  private final SendGlobalCommCache sendReducedValuesCache =
+      new SendGlobalCommCache(false);
   /** How big a single aggregator request can be */
   private final int maxBytesPerAggregatorRequest;
 
@@ -71,16 +72,16 @@ public class NettyWorkerAggregatorRequestProcessor
   }
 
   @Override
-  public boolean sendAggregatedValue(String aggregatorName,
-      Writable aggregatedValue) throws IOException {
+  public boolean sendReducedValue(String name,
+      Writable reducedValue) throws IOException {
     WorkerInfo owner =
-        AggregatorUtils.getOwner(aggregatorName,
+        AggregatorUtils.getOwner(name,
             serviceWorker.getWorkerInfoList());
     if (isThisWorker(owner)) {
       return false;
     } else {
-      int currentSize = sendAggregatedValueCache.addAggregator(
-          owner.getTaskId(), aggregatorName, aggregatedValue);
+      int currentSize = sendReducedValuesCache.addValue(owner.getTaskId(),
+          name, GlobalCommType.REDUCED_VALUE, reducedValue);
       if (currentSize >= maxBytesPerAggregatorRequest) {
         flushAggregatorsToWorker(owner);
       }
@@ -92,12 +93,12 @@ public class NettyWorkerAggregatorRequestProcessor
   public void flush() throws IOException {
     for (WorkerInfo workerInfo : serviceWorker.getWorkerInfoList()) {
       if (!isThisWorker(workerInfo)) {
-        sendAggregatedValueCache.addCountAggregator(workerInfo.getTaskId());
+        sendReducedValuesCache.addSpecialCount(workerInfo.getTaskId());
         flushAggregatorsToWorker(workerInfo);
         progressable.progress();
       }
     }
-    sendAggregatedValueCache.reset();
+    sendReducedValuesCache.reset();
   }
 
   /**
@@ -106,22 +107,21 @@ public class NettyWorkerAggregatorRequestProcessor
    * @param worker Worker which we want to send aggregators to
    */
   private void flushAggregatorsToWorker(WorkerInfo worker) {
-    byte[] aggregatorData =
-        sendAggregatedValueCache.removeAggregators(worker.getTaskId());
+    byte[] data =
+        sendReducedValuesCache.removeSerialized(worker.getTaskId());
     workerClient.sendWritableRequest(worker.getTaskId(),
-        new SendWorkerAggregatorsRequest(aggregatorData,
+        new SendWorkerAggregatorsRequest(data,
             serviceWorker.getWorkerInfo().getTaskId()));
   }
 
   @Override
-  public void sendAggregatedValuesToMaster(
-      byte[] aggregatorData) throws IOException {
+  public void sendReducedValuesToMaster(byte[] data) throws IOException {
     workerClient.sendWritableRequest(serviceWorker.getMasterInfo().getTaskId(),
-        new SendAggregatorsToMasterRequest(aggregatorData));
+        new SendReducedToMasterRequest(data));
   }
 
   @Override
-  public void distributeAggregators(
+  public void distributeReducedValues(
       Iterable<byte[]> aggregatorDataList) throws IOException {
     for (byte[] aggregatorData : aggregatorDataList) {
       for (WorkerInfo worker : serviceWorker.getWorkerInfoList()) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/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 e043314..02c72f7 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
@@ -18,10 +18,10 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.requests.MasterRequest;
-import org.apache.giraph.master.MasterAggregatorHandler;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
+import org.apache.giraph.master.MasterAggregatorHandler;
 
 /** Handler for requests on master */
 public class MasterRequestServerHandler extends

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
index c7561ee..26eaa8c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/RequestType.java
@@ -54,7 +54,7 @@ public enum RequestType {
   /** Send aggregated values from one worker's vertices */
   SEND_WORKER_AGGREGATORS_REQUEST(SendWorkerAggregatorsRequest.class),
   /** Send aggregated values from worker owner to master */
-  SEND_AGGREGATORS_TO_MASTER_REQUEST(SendAggregatorsToMasterRequest.class),
+  SEND_AGGREGATORS_TO_MASTER_REQUEST(SendReducedToMasterRequest.class),
   /** Send aggregators from master to worker owners */
   SEND_AGGREGATORS_TO_OWNER_REQUEST(SendAggregatorsToOwnerRequest.class),
   /** Send aggregators from worker owner to other workers */

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/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
deleted file mode 100644
index 2a05192..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendAggregatorsToMasterRequest.java
+++ /dev/null
@@ -1,61 +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.comm.requests;
-
-import org.apache.giraph.master.MasterAggregatorHandler;
-
-import java.io.IOException;
-
-/**
- * Request to send final aggregated values from worker which owns
- * aggregators to the master
- */
-public class SendAggregatorsToMasterRequest extends ByteArrayRequest
-    implements MasterRequest {
-
-  /**
-   * Constructor
-   *
-   * @param data Serialized aggregator data
-   */
-  public SendAggregatorsToMasterRequest(byte[] data) {
-    super(data);
-  }
-
-  /**
-   * Constructor used for reflection only
-   */
-  public SendAggregatorsToMasterRequest() {
-  }
-
-  @Override
-  public void doRequest(MasterAggregatorHandler aggregatorHandler) {
-    try {
-      aggregatorHandler.acceptAggregatedValues(getDataInput());
-    } catch (IOException e) {
-      throw new IllegalStateException("doRequest: " +
-          "IOException occurred while processing request", e);
-    }
-  }
-
-  @Override
-  public RequestType getType() {
-    return RequestType.SEND_AGGREGATORS_TO_MASTER_REQUEST;
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/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 10d8d02..2d5cc51 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,11 +21,12 @@ package org.apache.giraph.comm.requests;
 import java.io.DataInput;
 import java.io.IOException;
 
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.ServerData;
-import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
-import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.utils.UnsafeByteArrayOutputStream;
+import org.apache.giraph.utils.UnsafeReusableByteArrayInput;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
@@ -55,28 +56,32 @@ public class SendAggregatorsToOwnerRequest
 
   @Override
   public void doRequest(ServerData serverData) {
+    UnsafeByteArrayOutputStream reusedOut = new UnsafeByteArrayOutputStream();
+    UnsafeReusableByteArrayInput reusedIn = new UnsafeReusableByteArrayInput();
+
     DataInput input = getDataInput();
     AllAggregatorServerData aggregatorData = serverData.getAllAggregatorData();
     try {
-      int numAggregators = input.readInt();
-      for (int i = 0; i < numAggregators; i++) {
-        String aggregatorName = input.readUTF();
-        WritableFactory<Aggregator<Writable>> aggregatorFactory =
-            WritableUtils.readWritableObject(input, conf);
-        if (aggregatorName.equals(AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)) {
-          LongWritable count = new LongWritable(0);
-          count.readFields(input);
-          aggregatorData.receivedRequestCountFromMaster(count.get(),
+      int num = input.readInt();
+      for (int i = 0; i < num; i++) {
+        String name = input.readUTF();
+        GlobalCommType type = GlobalCommType.values()[input.readByte()];
+        Writable value = WritableUtils.readWritableObject(input, conf);
+        if (type == GlobalCommType.SPECIAL_COUNT) {
+          aggregatorData.receivedRequestCountFromMaster(
+              ((LongWritable) value).get(),
               getSenderTaskId());
         } else {
-          aggregatorData.registerAggregatorClass(aggregatorName,
-              aggregatorFactory);
-          Writable aggregatorValue =
-              aggregatorData.createAggregatorInitialValue(aggregatorName);
-          aggregatorValue.readFields(input);
-          aggregatorData.setAggregatorValue(aggregatorName, aggregatorValue);
-          serverData.getOwnerAggregatorData().registerAggregator(
-              aggregatorName, aggregatorFactory);
+          aggregatorData.receiveValueFromMaster(name, type, value);
+
+          if (type == GlobalCommType.REDUCE_OPERATIONS) {
+            ReduceOperation<Object, Writable> reduceOpCopy =
+                (ReduceOperation<Object, Writable>)
+                WritableUtils.createCopy(reusedOut, reusedIn, value);
+
+            serverData.getOwnerAggregatorData().registerReducer(
+                name, reduceOpCopy);
+          }
         }
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/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 d469e96..361bdc9 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,11 +21,9 @@ package org.apache.giraph.comm.requests;
 import java.io.DataInput;
 import java.io.IOException;
 
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.ServerData;
-import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
@@ -58,23 +56,17 @@ public class SendAggregatorsToWorkerRequest extends
     DataInput input = getDataInput();
     AllAggregatorServerData aggregatorData = serverData.getAllAggregatorData();
     try {
-      int numAggregators = input.readInt();
-      for (int i = 0; i < numAggregators; i++) {
-        String aggregatorName = input.readUTF();
-        WritableFactory<Aggregator<Writable>> aggregatorFactory =
-            WritableUtils.readWritableObject(input, conf);
-        if (aggregatorName.equals(AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)) {
-          LongWritable count = new LongWritable(0);
-          count.readFields(input);
-          aggregatorData.receivedRequestCountFromWorker(count.get(),
+      int num = input.readInt();
+      for (int i = 0; i < num; i++) {
+        String name = input.readUTF();
+        GlobalCommType type = GlobalCommType.values()[input.readByte()];
+        Writable value = WritableUtils.readWritableObject(input, conf);
+        if (type == GlobalCommType.SPECIAL_COUNT) {
+          aggregatorData.receivedRequestCountFromWorker(
+              ((LongWritable) value).get(),
               getSenderTaskId());
         } else {
-          aggregatorData.registerAggregatorClass(aggregatorName,
-              aggregatorFactory);
-          Writable aggregatorValue =
-              aggregatorData.createAggregatorInitialValue(aggregatorName);
-          aggregatorValue.readFields(input);
-          aggregatorData.setAggregatorValue(aggregatorName, aggregatorValue);
+          aggregatorData.receiveValueFromMaster(name, type, value);
         }
       }
     } catch (IOException e) {


[35/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Fix tests


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/da3c7b2d
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/da3c7b2d
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/da3c7b2d

Branch: refs/heads/release-1.1
Commit: da3c7b2d648fa270b997e514933dfc081ae7810b
Parents: 414ef89
Author: Maja Kabiljo <ma...@fb.com>
Authored: Thu Oct 2 17:37:23 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Thu Oct 2 17:37:23 2014 -0700

----------------------------------------------------------------------
 .../combiner/DoubleSumMessageCombiner.java      |  4 +--
 .../combiner/FloatSumMessageCombiner.java       |  4 +--
 .../combiner/MinimumDoubleMessageCombiner.java  |  4 +--
 .../combiner/MinimumIntMessageCombiner.java     |  4 +--
 .../combiner/SimpleSumMessageCombiner.java      |  4 +--
 .../primitives/IntFloatMessageStore.java        | 35 ++++++++++----------
 .../primitives/LongDoubleMessageStore.java      | 32 +++++++++---------
 .../giraph/examples/MinimumIntCombinerTest.java |  7 ++--
 8 files changed, 47 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
index c7c97a4..04f6bdf 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/DoubleSumMessageCombiner.java
@@ -28,8 +28,8 @@ public class DoubleSumMessageCombiner
     extends
     MessageCombiner<WritableComparable, DoubleWritable> {
   @Override
-  public void combine(WritableComparable vertexIndex, DoubleWritable originalMessage,
-      DoubleWritable messageToCombine) {
+  public void combine(WritableComparable vertexIndex,
+      DoubleWritable originalMessage, DoubleWritable messageToCombine) {
     originalMessage.set(originalMessage.get() + messageToCombine.get());
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
index 0bbea4e..3015e2b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/FloatSumMessageCombiner.java
@@ -28,8 +28,8 @@ public class FloatSumMessageCombiner
     extends
     MessageCombiner<WritableComparable, FloatWritable> {
   @Override
-  public void combine(WritableComparable vertexIndex, FloatWritable originalMessage,
-      FloatWritable messageToCombine) {
+  public void combine(WritableComparable vertexIndex,
+      FloatWritable originalMessage, FloatWritable messageToCombine) {
     originalMessage.set(originalMessage.get() + messageToCombine.get());
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
index ebd9f0a..db43008 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumDoubleMessageCombiner.java
@@ -28,8 +28,8 @@ public class MinimumDoubleMessageCombiner
     extends
     MessageCombiner<WritableComparable, DoubleWritable> {
   @Override
-  public void combine(WritableComparable vertexIndex, DoubleWritable originalMessage,
-      DoubleWritable messageToCombine) {
+  public void combine(WritableComparable vertexIndex,
+      DoubleWritable originalMessage, DoubleWritable messageToCombine) {
     if (originalMessage.get() > messageToCombine.get()) {
       originalMessage.set(messageToCombine.get());
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
index 0aec90a..df80b8f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/MinimumIntMessageCombiner.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.io.WritableComparable;
 public class MinimumIntMessageCombiner
     extends MessageCombiner<WritableComparable, IntWritable> {
   @Override
-  public void combine(WritableComparable vertexIndex, IntWritable originalMessage,
-      IntWritable messageToCombine) {
+  public void combine(WritableComparable vertexIndex,
+      IntWritable originalMessage, IntWritable messageToCombine) {
     if (originalMessage.get() > messageToCombine.get()) {
       originalMessage.set(messageToCombine.get());
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java b/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
index cd00fbb..e3ae597 100644
--- a/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/combiner/SimpleSumMessageCombiner.java
@@ -28,8 +28,8 @@ public class SimpleSumMessageCombiner
     extends MessageCombiner<WritableComparable, IntWritable> {
 
   @Override
-  public void combine(WritableComparable vertexIndex, IntWritable originalMessage,
-      IntWritable messageToCombine) {
+  public void combine(WritableComparable vertexIndex,
+      IntWritable originalMessage, IntWritable messageToCombine) {
     originalMessage.set(originalMessage.get() + messageToCombine.get());
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
index 97086e1..8095ad6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IntFloatMessageStore.java
@@ -18,33 +18,32 @@
 
 package org.apache.giraph.comm.messages.primitives;
 
+import it.unimi.dsi.fastutil.ints.Int2FloatMap;
+import it.unimi.dsi.fastutil.ints.Int2FloatOpenHashMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.ints.IntIterator;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.comm.messages.MessageStore;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionStore;
-
+import org.apache.giraph.utils.EmptyIterable;
 import org.apache.giraph.utils.VertexIdMessageIterator;
 import org.apache.giraph.utils.VertexIdMessages;
-import org.apache.giraph.utils.EmptyIterable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 
 import com.google.common.collect.Lists;
 
-import it.unimi.dsi.fastutil.ints.Int2FloatMap;
-import it.unimi.dsi.fastutil.ints.Int2FloatOpenHashMap;
-import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
-import it.unimi.dsi.fastutil.ints.IntIterator;
-import it.unimi.dsi.fastutil.objects.ObjectIterator;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
 /**
  * Special message store to be used when ids are IntWritable and messages
  * are FloatWritable and messageCombiner is used.
@@ -56,7 +55,8 @@ public class IntFloatMessageStore
   /** Map from partition id to map from vertex id to message */
   private final Int2ObjectOpenHashMap<Int2FloatOpenHashMap> map;
   /** Message messageCombiner */
-  private final MessageCombiner<IntWritable, FloatWritable> messageCombiner;
+  private final
+  MessageCombiner<? super IntWritable, FloatWritable> messageCombiner;
   /** Service worker */
   private final CentralizedServiceWorker<IntWritable, ?, ?> service;
 
@@ -68,10 +68,9 @@ public class IntFloatMessageStore
    */
   public IntFloatMessageStore(
       CentralizedServiceWorker<IntWritable, Writable, Writable> service,
-      MessageCombiner<IntWritable, FloatWritable> messageCombiner) {
+      MessageCombiner<? super IntWritable, FloatWritable> messageCombiner) {
     this.service = service;
-    this.messageCombiner =
-        messageCombiner;
+    this.messageCombiner = messageCombiner;
 
     map = new Int2ObjectOpenHashMap<Int2FloatOpenHashMap>();
     for (int partitionId : service.getPartitionStore().getPartitionIds()) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
index b0452c1..dac98c9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/LongDoubleMessageStore.java
@@ -18,20 +18,6 @@
 
 package org.apache.giraph.comm.messages.primitives;
 
-import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.combiner.MessageCombiner;
-import org.apache.giraph.comm.messages.MessageStore;
-import org.apache.giraph.partition.Partition;
-import org.apache.giraph.utils.VertexIdMessageIterator;
-import org.apache.giraph.utils.VertexIdMessages;
-import org.apache.giraph.utils.EmptyIterable;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-import org.apache.hadoop.io.Writable;
-
-import com.google.common.collect.Lists;
-
 import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import it.unimi.dsi.fastutil.longs.Long2DoubleMap;
 import it.unimi.dsi.fastutil.longs.Long2DoubleOpenHashMap;
@@ -44,6 +30,19 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.utils.EmptyIterable;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.collect.Lists;
+
 /**
  * Special message store to be used when ids are LongWritable and messages
  * are DoubleWritable and messageCombiner is used.
@@ -55,7 +54,8 @@ public class LongDoubleMessageStore
   /** Map from partition id to map from vertex id to message */
   private final Int2ObjectOpenHashMap<Long2DoubleOpenHashMap> map;
   /** Message messageCombiner */
-  private final MessageCombiner<LongWritable, DoubleWritable> messageCombiner;
+  private final
+  MessageCombiner<? super LongWritable, DoubleWritable> messageCombiner;
   /** Service worker */
   private final CentralizedServiceWorker<LongWritable, ?, ?> service;
 
@@ -67,7 +67,7 @@ public class LongDoubleMessageStore
    */
   public LongDoubleMessageStore(
       CentralizedServiceWorker<LongWritable, Writable, Writable> service,
-      MessageCombiner<LongWritable, DoubleWritable> messageCombiner) {
+      MessageCombiner<? super LongWritable, DoubleWritable> messageCombiner) {
     this.service = service;
     this.messageCombiner =
         messageCombiner;

http://git-wip-us.apache.org/repos/asf/giraph/blob/da3c7b2d/giraph-examples/src/test/java/org/apache/giraph/examples/MinimumIntCombinerTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/MinimumIntCombinerTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/MinimumIntCombinerTest.java
index aa6cd8a..50a7130 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/MinimumIntCombinerTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/MinimumIntCombinerTest.java
@@ -18,18 +18,19 @@
 
 package org.apache.giraph.examples;
 
+import static org.junit.Assert.assertEquals;
+
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.combiner.MinimumIntMessageCombiner;
 import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.WritableComparable;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-
 public class MinimumIntCombinerTest {
 
   @Test
   public void testCombiner() throws Exception {
-    MessageCombiner<IntWritable, IntWritable>
+    MessageCombiner<WritableComparable, IntWritable>
         messageCombiner =
         new MinimumIntMessageCombiner();
 


[02/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-918: GIRAPH-908 has a small bug reg counting entries (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/ffdddff3
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/ffdddff3
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/ffdddff3

Branch: refs/heads/release-1.1
Commit: ffdddff32e7ac6e2cbb5ae8471c1192c69ac6a94
Parents: 372c35e
Author: Pavan Kumar <pa...@fb.com>
Authored: Tue Jun 10 16:12:23 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Tue Jun 10 16:12:23 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                                |  2 ++
 .../java/org/apache/giraph/worker/BspServiceWorker.java  | 11 ++++++-----
 2 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/ffdddff3/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index ec1f798..f35d4ba 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-918: GIRAPH-908 has a small bug reg counting entries (pavanka)
+
   GIRAPH-842: option to dump histogram of memory usage when heap is low on memory (pavanka)
 
   GIRAPH-904: Giraph can hang when hostnames include uppercase letters (netj via pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/ffdddff3/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index a89da24..8dcf19a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -320,7 +320,7 @@ public class BspServiceWorker<I extends WritableComparable,
    *
    * @return Count of mapping entries loaded
    */
-  private Integer loadMapping() throws KeeperException,
+  private long loadMapping() throws KeeperException,
     InterruptedException {
     List<String> inputSplitPathList =
         getZkExt().getChildrenExt(mappingInputSplitsPaths.getPath(),
@@ -341,7 +341,7 @@ public class BspServiceWorker<I extends WritableComparable,
             this,
             getZkExt());
 
-    int entriesLoaded = 0;
+    long entriesLoaded = 0;
     // Determine how many threads to use based on the number of input splits
     int maxInputSplitThreads = inputSplitPathList.size();
     int numThreads = Math.min(getConfiguration().getNumInputSplitsThreads(),
@@ -577,7 +577,7 @@ public class BspServiceWorker<I extends WritableComparable,
     aggregatorHandler.prepareSuperstep(workerAggregatorRequestProcessor);
 
     VertexEdgeCount vertexEdgeCount;
-    int entriesLoaded = 0;
+    long entriesLoaded;
 
     if (getConfiguration().hasMappingInputFormat()) {
       // Ensure the mapping InputSplits are ready for processing
@@ -673,13 +673,14 @@ public class BspServiceWorker<I extends WritableComparable,
       }
     }
 
+    // remove mapping store if possible
+    localData.removeMappingStoreIfPossible();
+
     if (getConfiguration().hasEdgeInputFormat()) {
       // Move edges from temporary storage to their source vertices.
       getServerData().getEdgeStore().moveEdgesToVertices();
     }
 
-    localData.removeMappingStoreIfPossible();
-
     // Generate the partition stats for the input superstep and process
     // if necessary
     List<PartitionStats> partitionStatsList =


[19/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-934: Allow having state in aggregators (ikabiljo via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/ce97134d
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/ce97134d
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/ce97134d

Branch: refs/heads/release-1.1
Commit: ce97134d253c4e9fca48b7cede2048e60f36ff79
Parents: 9303522
Author: Maja Kabiljo <ma...@fb.com>
Authored: Fri Aug 1 08:20:10 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Fri Aug 1 08:20:10 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../giraph/aggregators/AggregatorWrapper.java   |  22 ++--
 .../aggregators/ArrayAggregatorFactory.java     | 128 +++++++++++++++++++
 .../giraph/aggregators/BasicAggregator.java     |   9 ++
 .../aggregators/ClassAggregatorFactory.java     |  87 +++++++++++++
 .../org/apache/giraph/comm/MasterClient.java    |   9 +-
 .../java/org/apache/giraph/comm/ServerData.java |  14 +-
 .../aggregators/AggregatorOutputStream.java     |  12 +-
 .../comm/aggregators/AggregatorUtils.java       |  39 +-----
 .../aggregators/AllAggregatorServerData.java    |  69 ++++------
 .../aggregators/OwnerAggregatorServerData.java  |  26 ++--
 .../comm/aggregators/SendAggregatorCache.java   |  15 ++-
 .../giraph/comm/netty/NettyMasterClient.java    |  17 +--
 .../requests/SendAggregatorsToOwnerRequest.java |  19 +--
 .../SendAggregatorsToWorkerRequest.java         |  17 +--
 .../giraph/comm/requests/WritableRequest.java   |   2 +-
 .../giraph/master/MasterAggregatorHandler.java  |  66 ++++++----
 .../giraph/master/MasterAggregatorUsage.java    |  16 +++
 .../org/apache/giraph/master/MasterCompute.java |   8 ++
 .../org/apache/giraph/utils/ArrayWritable.java  | 100 +++++++++++++++
 .../apache/giraph/utils/WritableFactory.java    |  28 ++++
 .../org/apache/giraph/utils/WritableUtils.java  |  84 ++++++++++--
 .../giraph/worker/WorkerAggregatorHandler.java  |  52 +++++---
 .../giraph/aggregators/TestArrayAggregator.java |  50 ++++++++
 24 files changed, 686 insertions(+), 205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 54ed3a3..dbb134a 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-934: Allow having state in aggregators (ikabiljo via majakabiljo)
+
   GIRAPH-932: Adding .arcconfig to GIRAPH for Arcanist support (aching)
 
   GIRAPH-927: Decouple netty server threads from message processing (edunov via pavanka)  

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
index 7150402..fa18a64 100644
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/aggregators/AggregatorWrapper.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.aggregators;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -34,21 +34,25 @@ public class AggregatorWrapper<A extends Writable> {
   private final boolean persistent;
   /** Value aggregated in previous super step */
   private A previousAggregatedValue;
+  /** Aggregator factory */
+  private final WritableFactory<? extends Aggregator<A>> aggregatorFactory;
   /** 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
-   * @param conf            Configuration
+   * @param aggregatorFactory Aggregator Factory
+   * @param persistent        False iff aggregator should be reset at the end
+   *                          of each super step
+   * @param conf              Configuration
    */
-  public AggregatorWrapper(Class<? extends Aggregator<A>> aggregatorClass,
+  public AggregatorWrapper(
+      WritableFactory<? extends Aggregator<A>> aggregatorFactory,
       boolean persistent, ImmutableClassesGiraphConfiguration conf) {
     this.persistent = persistent;
-    currentAggregator = ReflectionUtils.newInstance(aggregatorClass, conf);
+    this.aggregatorFactory = aggregatorFactory;
+    currentAggregator = aggregatorFactory.create();
     changed = false;
     previousAggregatedValue = currentAggregator.createInitialValue();
   }
@@ -140,7 +144,7 @@ public class AggregatorWrapper<A extends Writable> {
    *
    * @return Aggregator class
    */
-  public Class<? extends Aggregator> getAggregatorClass() {
-    return currentAggregator.getClass();
+  public WritableFactory<? extends Aggregator<A>> getAggregatorFactory() {
+    return aggregatorFactory;
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java
new file mode 100644
index 0000000..c977c57
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java
@@ -0,0 +1,128 @@
+/*
+ * 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.aggregators;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Array;
+
+import org.apache.giraph.utils.ArrayWritable;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Generic array aggregator factory, used to aggregate elements
+ * of ArrayWritable via passed element aggregator.
+ *
+ * @param <A> Type of individual element
+ */
+public class ArrayAggregatorFactory<A extends Writable>
+    implements WritableFactory<Aggregator<ArrayWritable<A>>> {
+  /** number of elements in array */
+  private int n;
+  /** element aggregator class */
+  private WritableFactory<? extends Aggregator<A>> elementAggregatorFactory;
+
+  /**
+   * Constructor
+   * @param n Number of elements in array
+   * @param elementAggregatorClass Type of element aggregator
+   */
+  public ArrayAggregatorFactory(
+      int n, Class<? extends Aggregator<A>> elementAggregatorClass) {
+    this(n, new ClassAggregatorFactory<>(elementAggregatorClass));
+  }
+
+  /**
+   * Constructor
+   * @param n Number of elements in array
+   * @param elementAggregatorFactory Element aggregator factory
+   */
+  public ArrayAggregatorFactory(int n,
+      WritableFactory<? extends Aggregator<A>> elementAggregatorFactory) {
+    this.n = n;
+    this.elementAggregatorFactory = elementAggregatorFactory;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    n = in.readInt();
+    elementAggregatorFactory = WritableUtils.readWritableObject(in, null);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(n);
+    WritableUtils.writeWritableObject(elementAggregatorFactory, out);
+  }
+
+  @Override
+  public Aggregator<ArrayWritable<A>> create() {
+    return new ArrayAggregator<>(
+        n, elementAggregatorFactory.create());
+  }
+
+  /**
+   * Stateful aggregator that aggregates ArrayWritable by
+   * aggregating individual elements
+   *
+   * @param <A> Type of individual element
+   */
+  public static class ArrayAggregator<A extends Writable>
+      extends BasicAggregator<ArrayWritable<A>> {
+    /** number of elements in array */
+    private final int n;
+    /** element aggregator */
+    private final Aggregator<A> elementAggregator;
+
+    /**
+     * Constructor
+     * @param n Number of elements in array
+     * @param elementAggregator Element aggregator
+     */
+    public ArrayAggregator(int n, Aggregator<A> elementAggregator) {
+      super(null);
+      this.n = n;
+      this.elementAggregator = elementAggregator;
+      reset();
+    }
+
+    @Override
+    public void aggregate(ArrayWritable<A> other) {
+      A[] array = getAggregatedValue().get();
+      for (int i = 0; i < n; i++) {
+        elementAggregator.setAggregatedValue(array[i]);
+        elementAggregator.aggregate(other.get()[i]);
+        array[i] = elementAggregator.getAggregatedValue();
+      }
+    }
+
+    @Override
+    public ArrayWritable<A> createInitialValue() {
+      Class<A> elementClass =
+          (Class) elementAggregator.createInitialValue().getClass();
+      A[] array = (A[]) Array.newInstance(elementClass, n);
+      for (int i = 0; i < n; i++) {
+        array[i] = elementAggregator.createInitialValue();
+      }
+      return new ArrayWritable<>(elementClass, array);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/aggregators/BasicAggregator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/BasicAggregator.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/BasicAggregator.java
index 07a4100..c351846 100644
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/BasicAggregator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/aggregators/BasicAggregator.java
@@ -40,6 +40,15 @@ public abstract class BasicAggregator<A extends Writable> implements
     value = createInitialValue();
   }
 
+  /**
+   * Constructor
+   * @param initialValue initial value
+   */
+  public BasicAggregator(A initialValue) {
+    value = initialValue;
+  }
+
+
   @Override
   public A getAggregatedValue() {
     return value;

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
new file mode 100644
index 0000000..944656e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
@@ -0,0 +1,87 @@
+/*
+ * 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.aggregators;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Aggregator factory based on aggregatorClass.
+ *
+ * @param <T> Aggregated value type
+ */
+public class ClassAggregatorFactory<T extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable
+    implements WritableFactory<Aggregator<T>> {
+  /** Aggregator class */
+  private Class<? extends Aggregator<T>> aggregatorClass;
+
+  /** Constructor */
+  public ClassAggregatorFactory() {
+  }
+
+  /**
+   * Constructor
+   * @param aggregatorClass Aggregator class
+   */
+  public ClassAggregatorFactory(
+      Class<? extends Aggregator<T>> aggregatorClass) {
+    this(aggregatorClass, null);
+
+  }
+
+  /**
+   * Constructor
+   * @param aggregatorClass Aggregator class
+   * @param conf Configuration
+   */
+  public ClassAggregatorFactory(Class<? extends Aggregator<T>> aggregatorClass,
+      ImmutableClassesGiraphConfiguration conf) {
+    Preconditions.checkNotNull(aggregatorClass,
+        "aggregatorClass cannot be null in ClassAggregatorFactory");
+    this.aggregatorClass = aggregatorClass;
+    setConf(conf);
+  }
+
+  @Override
+  public Aggregator<T> create() {
+    return ReflectionUtils.newInstance(aggregatorClass, getConf());
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    aggregatorClass = WritableUtils.readClass(in);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Preconditions.checkNotNull(aggregatorClass,
+        "aggregatorClass cannot be null in ClassAggregatorFactory");
+    WritableUtils.writeClass(aggregatorClass, out);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
index 793d059..b7718a7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/MasterClient.java
@@ -18,11 +18,12 @@
 
 package org.apache.giraph.comm;
 
+import java.io.IOException;
+
 import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 
-import java.io.IOException;
-
 /**
  * Interface for master to send messages to workers
  */
@@ -36,12 +37,12 @@ public interface MasterClient {
    * Sends aggregator to its owner
    *
    * @param aggregatorName Name of the aggregator
-   * @param aggregatorClass Class of the aggregator
+   * @param aggregatorFactory Aggregator factory
    * @param aggregatedValue Value of the aggregator
    * @throws IOException
    */
   void sendAggregator(String aggregatorName,
-      Class<? extends Aggregator> aggregatorClass,
+      WritableFactory<? extends Aggregator> aggregatorFactory,
       Writable aggregatedValue) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
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 29488fc..a92cd1c 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
@@ -18,6 +18,12 @@
 
 package org.apache.giraph.comm;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
 import org.apache.giraph.comm.aggregators.OwnerAggregatorServerData;
@@ -36,12 +42,6 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-
 /**
  * Anything that the server stores
  *
@@ -123,7 +123,7 @@ public class ServerData<I extends WritableComparable,
     EdgeStoreFactory<I, V, E> edgeStoreFactory = conf.createEdgeStoreFactory();
     edgeStoreFactory.initialize(service, conf, context);
     edgeStore = edgeStoreFactory.newStore();
-    ownerAggregatorData = new OwnerAggregatorServerData(context, conf);
+    ownerAggregatorData = new OwnerAggregatorServerData(context);
     allAggregatorData = new AllAggregatorServerData(context, conf);
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
index 627b4cc..79bc08a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorOutputStream.java
@@ -18,11 +18,13 @@
 
 package org.apache.giraph.comm.aggregators;
 
+import java.io.IOException;
+
 import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 
-import java.io.IOException;
-
 /**
  * Implementation of {@link CountingOutputStream} which allows writing of
  * aggregators in the form of triple (name, classname, value)
@@ -32,17 +34,17 @@ public class AggregatorOutputStream extends CountingOutputStream {
    * Write aggregator to the stream and increment internal counter
    *
    * @param aggregatorName Name of the aggregator
-   * @param aggregatorClass Class of aggregator
+   * @param aggregatorFactory Aggregator factory
    * @param aggregatedValue Value of aggregator
    * @return Number of bytes occupied by the stream
    * @throws IOException
    */
   public int addAggregator(String aggregatorName,
-      Class<? extends Aggregator> aggregatorClass,
+      WritableFactory<? extends Aggregator> aggregatorFactory,
       Writable aggregatedValue) throws IOException {
     incrementCounter();
     dataOutput.writeUTF(aggregatorName);
-    dataOutput.writeUTF(aggregatorClass.getName());
+    WritableUtils.writeWritableObject(aggregatorFactory, dataOutput);
     aggregatedValue.write(dataOutput);
     return getSize();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
index ceb30a8..a94ab38 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
@@ -18,13 +18,10 @@
 
 package org.apache.giraph.comm.aggregators;
 
+import java.util.List;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.worker.WorkerInfo;
-import org.apache.hadoop.io.Writable;
-
-import java.util.List;
 
 /**
  * Class for aggregator constants and utility methods
@@ -36,6 +33,7 @@ public class AggregatorUtils {
    */
   public static final String SPECIAL_COUNT_AGGREGATOR =
       "__aggregatorRequestCount";
+
   /** How big a single aggregator request can be (in bytes) */
   public static final String MAX_BYTES_PER_AGGREGATOR_REQUEST =
       "giraph.maxBytesPerAggregatorRequest";
@@ -58,37 +56,6 @@ public class AggregatorUtils {
   private AggregatorUtils() { }
 
   /**
-   * Get aggregator class from class name, catch all exceptions.
-   *
-   * @param aggregatorClassName Class nam of aggregator class
-   * @return Aggregator class
-   */
-  public static Class<Aggregator<Writable>> getAggregatorClass(String
-      aggregatorClassName) {
-    try {
-      return (Class<Aggregator<Writable>>) Class.forName(aggregatorClassName);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalStateException("getAggregatorClass: " +
-          "ClassNotFoundException for aggregator class " + aggregatorClassName,
-          e);
-    }
-  }
-
-  /**
-   * Create new aggregator instance from aggregator class,
-   * catch all exceptions.
-   *
-   * @param aggregatorClass Class of aggregator
-   * @param conf Configuration
-   * @return New aggregator
-   */
-  public static Aggregator<Writable> newAggregatorInstance(
-      Class<Aggregator<Writable>> aggregatorClass,
-      ImmutableClassesGiraphConfiguration conf) {
-    return ReflectionUtils.newInstance(aggregatorClass, conf);
-  }
-
-  /**
    * Get owner of aggregator with selected name from the list of workers
    *
    * @param aggregatorName Name of the aggregators

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
index 177e738..effc9bf 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AllAggregatorServerData.java
@@ -18,10 +18,18 @@
 
 package org.apache.giraph.comm.aggregators;
 
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
 import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.master.MasterInfo;
+import org.apache.giraph.utils.Factory;
 import org.apache.giraph.utils.TaskIdsPermitsBarrier;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -29,12 +37,6 @@ import org.apache.log4j.Logger;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-
 /**
  * Accepts aggregators and their values from previous superstep from master
  * and workers which own aggregators. Keeps data received from master so it
@@ -49,16 +51,9 @@ public class AllAggregatorServerData {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(AllAggregatorServerData.class);
-  /**
-   * Map from aggregator class to aggregator object which we need in order
-   * to create initial aggregated values
-   */
-  private final
-  ConcurrentMap<Class<Aggregator<Writable>>, Aggregator<Writable>>
-  aggregatorTypesMap = Maps.newConcurrentMap();
-  /** Map of aggregator classes */
-  private final ConcurrentMap<String, Class<Aggregator<Writable>>>
-  aggregatorClassMap = Maps.newConcurrentMap();
+  /** Map of aggregator factories */
+  private final ConcurrentMap<String, WritableFactory<Aggregator<Writable>>>
+  aggregatorFactoriesMap = Maps.newConcurrentMap();
   /** Map of values of aggregators from previous superstep */
   private final ConcurrentMap<String, Writable>
   aggregatedValuesMap = Maps.newConcurrentMap();
@@ -104,16 +99,12 @@ public class AllAggregatorServerData {
   /**
    * Register the class of the aggregator, received by master or worker.
    *
-   * @param name            Aggregator name
-   * @param aggregatorClass Class of the aggregator
+   * @param name              Aggregator name
+   * @param aggregatorFactory Aggregator factory
    */
   public void registerAggregatorClass(String name,
-      Class<Aggregator<Writable>> aggregatorClass) {
-    aggregatorClassMap.put(name, aggregatorClass);
-    if (!aggregatorTypesMap.containsKey(aggregatorClass)) {
-      aggregatorTypesMap.putIfAbsent(aggregatorClass,
-          AggregatorUtils.newAggregatorInstance(aggregatorClass, conf));
-    }
+      WritableFactory<Aggregator<Writable>> aggregatorFactory) {
+    aggregatorFactoriesMap.put(name, aggregatorFactory);
     progressable.progress();
   }
 
@@ -139,10 +130,10 @@ public class AllAggregatorServerData {
    * @return Empty aggregated value for this aggregator
    */
   public Writable createAggregatorInitialValue(String name) {
-    Class<Aggregator<Writable>> aggregatorClass = aggregatorClassMap.get(name);
-    Aggregator<Writable> aggregator = aggregatorTypesMap.get(aggregatorClass);
-    synchronized (aggregator) {
-      return aggregator.createInitialValue();
+    WritableFactory<Aggregator<Writable>> aggregatorFactory =
+        aggregatorFactoriesMap.get(name);
+    synchronized (aggregatorFactory) {
+      return aggregatorFactory.create().createInitialValue();
     }
   }
 
@@ -211,29 +202,25 @@ public class AllAggregatorServerData {
    * @param workerIds All workers in the job apart from the current one
    * @param previousAggregatedValuesMap Map of values from previous
    *                                    superstep to fill out
-   * @param currentAggregatorMap Map of aggregators for current superstep to
-   *                             fill out. All aggregators in this map will
-   *                             be set to initial value.
+   * @param currentAggregatorFactoryMap Map of aggregators factories for
+   *                                    current superstep to fill out.
    */
   public void fillNextSuperstepMapsWhenReady(
       Set<Integer> workerIds,
       Map<String, Writable> previousAggregatedValuesMap,
-      Map<String, Aggregator<Writable>> currentAggregatorMap) {
+      Map<String, Factory<Aggregator<Writable>>> currentAggregatorFactoryMap) {
     workersBarrier.waitForRequiredPermits(workerIds);
     if (LOG.isDebugEnabled()) {
       LOG.debug("fillNextSuperstepMapsWhenReady: Aggregators ready");
     }
     previousAggregatedValuesMap.clear();
     previousAggregatedValuesMap.putAll(aggregatedValuesMap);
-    for (Map.Entry<String, Class<Aggregator<Writable>>> entry :
-        aggregatorClassMap.entrySet()) {
-      Aggregator<Writable> aggregator =
-          currentAggregatorMap.get(entry.getKey());
-      if (aggregator == null) {
-        currentAggregatorMap.put(entry.getKey(),
-            AggregatorUtils.newAggregatorInstance(entry.getValue(), conf));
-      } else {
-        aggregator.reset();
+    for (Map.Entry<String, WritableFactory<Aggregator<Writable>>> entry :
+        aggregatorFactoriesMap.entrySet()) {
+      Factory<Aggregator<Writable>> aggregatorFactory =
+          currentAggregatorFactoryMap.get(entry.getKey());
+      if (aggregatorFactory == null) {
+        currentAggregatorFactoryMap.put(entry.getKey(), entry.getValue());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
index eb25a2e..2f3d5e5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/OwnerAggregatorServerData.java
@@ -18,9 +18,14 @@
 
 package org.apache.giraph.comm.aggregators;
 
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
 import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.TaskIdsPermitsBarrier;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -29,11 +34,6 @@ import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
-import java.util.AbstractMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-
 /**
  * Class for holding aggregators which current worker owns,
  * and aggregating partial aggregator values from workers.
@@ -73,19 +73,14 @@ public class OwnerAggregatorServerData {
   private final TaskIdsPermitsBarrier workersBarrier;
   /** Progressable used to report progress */
   private final Progressable progressable;
-  /** Configuration */
-  private final ImmutableClassesGiraphConfiguration conf;
 
   /**
    * Constructor
    *
    * @param progressable Progressable used to report progress
-   * @param conf         Configuration
    */
-  public OwnerAggregatorServerData(Progressable progressable,
-      ImmutableClassesGiraphConfiguration conf) {
+  public OwnerAggregatorServerData(Progressable progressable) {
     this.progressable = progressable;
-    this.conf = conf;
     workersBarrier = new TaskIdsPermitsBarrier(progressable);
   }
 
@@ -93,15 +88,14 @@ public class OwnerAggregatorServerData {
    * Register an aggregator which current worker owns. Thread-safe.
    *
    * @param name Name of aggregator
-   * @param aggregatorClass Aggregator class
+   * @param aggregatorFactory Aggregator factory
    */
   public void registerAggregator(String name,
-      Class<Aggregator<Writable>> aggregatorClass) {
+      WritableFactory<Aggregator<Writable>> aggregatorFactory) {
     if (LOG.isDebugEnabled() && myAggregatorMap.isEmpty()) {
       LOG.debug("registerAggregator: The first registration after a reset()");
     }
-    myAggregatorMap.putIfAbsent(name,
-        AggregatorUtils.newAggregatorInstance(aggregatorClass, conf));
+    myAggregatorMap.putIfAbsent(name, aggregatorFactory.create());
     progressable.progress();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
index adc2aa8..8f880b4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/SendAggregatorCache.java
@@ -18,15 +18,16 @@
 
 package org.apache.giraph.comm.aggregators;
 
+import java.io.IOException;
+import java.util.Map;
+
 import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 
 import com.google.common.collect.Maps;
 
-import java.io.IOException;
-import java.util.Map;
-
 /**
  * Takes and serializes aggregators and keeps them grouped by owner
  * partition id, to be sent in bulk.
@@ -41,20 +42,20 @@ public class SendAggregatorCache extends CountingCache {
    *
    * @param taskId Task id of worker which owns the aggregator
    * @param aggregatorName Name of the aggregator
-   * @param aggregatorClass Class of the aggregator
+   * @param aggregatorFactory Aggregator factory
    * @param aggregatedValue Value of the aggregator
    * @return Number of bytes in serialized data for this worker
    * @throws IOException
    */
   public int addAggregator(Integer taskId, String aggregatorName,
-      Class<? extends Aggregator> aggregatorClass,
+      WritableFactory<? extends Aggregator> aggregatorFactory,
       Writable aggregatedValue) throws IOException {
     AggregatorOutputStream out = aggregatorMap.get(taskId);
     if (out == null) {
       out = new AggregatorOutputStream();
       aggregatorMap.put(taskId, out);
     }
-    return out.addAggregator(aggregatorName, aggregatorClass,
+    return out.addAggregator(aggregatorName, aggregatorFactory,
         aggregatedValue);
   }
 
@@ -86,6 +87,6 @@ public class SendAggregatorCache extends CountingCache {
     // current number of requests, plus one for the last flush
     long totalCount = getCount(taskId) + 1;
     addAggregator(taskId, AggregatorUtils.SPECIAL_COUNT_AGGREGATOR,
-        Aggregator.class, new LongWritable(totalCount));
+        null, new LongWritable(totalCount));
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/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 1218d29..51277c9 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
@@ -18,20 +18,21 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import java.io.IOException;
+
+import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 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.aggregators.Aggregator;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.giraph.worker.WorkerInfo;
-import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.util.Progressable;
 
-import java.io.IOException;
-
 /**
  * Netty implementation of {@link MasterClient}
  */
@@ -39,7 +40,7 @@ public class NettyMasterClient implements MasterClient {
   /** Netty client that does the actual I/O */
   private final NettyClient nettyClient;
   /** Worker information for current superstep */
-  private CentralizedServiceMaster<?, ?, ?> service;
+  private final CentralizedServiceMaster<?, ?, ?> service;
   /** Cached map of partition ids to serialized aggregator data */
   private final SendAggregatorCache sendAggregatorCache =
       new SendAggregatorCache();
@@ -78,12 +79,12 @@ public class NettyMasterClient implements MasterClient {
 
   @Override
   public void sendAggregator(String aggregatorName,
-      Class<? extends Aggregator> aggregatorClass,
+      WritableFactory<? extends Aggregator> aggregatorFactory,
       Writable aggregatedValue) throws IOException {
     WorkerInfo owner =
         AggregatorUtils.getOwner(aggregatorName, service.getWorkerInfoList());
     int currentSize = sendAggregatorCache.addAggregator(owner.getTaskId(),
-        aggregatorName, aggregatorClass, aggregatedValue);
+        aggregatorName, aggregatorFactory, aggregatedValue);
     if (currentSize >= maxBytesPerAggregatorRequest) {
       flushAggregatorsToWorker(owner);
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/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 e2681ee..10d8d02 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
@@ -18,16 +18,18 @@
 
 package org.apache.giraph.comm.requests;
 
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 
-import java.io.DataInput;
-import java.io.IOException;
-
 /**
  * Request to send final aggregatd values from master to worker which owns
  * the aggregators
@@ -59,23 +61,22 @@ public class SendAggregatorsToOwnerRequest
       int numAggregators = input.readInt();
       for (int i = 0; i < numAggregators; i++) {
         String aggregatorName = input.readUTF();
-        String aggregatorClassName = input.readUTF();
+        WritableFactory<Aggregator<Writable>> aggregatorFactory =
+            WritableUtils.readWritableObject(input, conf);
         if (aggregatorName.equals(AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)) {
           LongWritable count = new LongWritable(0);
           count.readFields(input);
           aggregatorData.receivedRequestCountFromMaster(count.get(),
               getSenderTaskId());
         } else {
-          Class<Aggregator<Writable>> aggregatorClass =
-              AggregatorUtils.getAggregatorClass(aggregatorClassName);
           aggregatorData.registerAggregatorClass(aggregatorName,
-              aggregatorClass);
+              aggregatorFactory);
           Writable aggregatorValue =
               aggregatorData.createAggregatorInitialValue(aggregatorName);
           aggregatorValue.readFields(input);
           aggregatorData.setAggregatorValue(aggregatorName, aggregatorValue);
           serverData.getOwnerAggregatorData().registerAggregator(
-              aggregatorName, aggregatorClass);
+              aggregatorName, aggregatorFactory);
         }
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/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 52e4cba..d469e96 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
@@ -18,16 +18,18 @@
 
 package org.apache.giraph.comm.requests;
 
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 
-import java.io.DataInput;
-import java.io.IOException;
-
 /**
  * Request to send final aggregated values from worker which owns them to
  * other workers
@@ -59,17 +61,16 @@ public class SendAggregatorsToWorkerRequest extends
       int numAggregators = input.readInt();
       for (int i = 0; i < numAggregators; i++) {
         String aggregatorName = input.readUTF();
-        String aggregatorClassName = input.readUTF();
+        WritableFactory<Aggregator<Writable>> aggregatorFactory =
+            WritableUtils.readWritableObject(input, conf);
         if (aggregatorName.equals(AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)) {
           LongWritable count = new LongWritable(0);
           count.readFields(input);
           aggregatorData.receivedRequestCountFromWorker(count.get(),
               getSenderTaskId());
         } else {
-          Class<Aggregator<Writable>> aggregatorClass =
-              AggregatorUtils.getAggregatorClass(aggregatorClassName);
           aggregatorData.registerAggregatorClass(aggregatorName,
-              aggregatorClass);
+              aggregatorFactory);
           Writable aggregatorValue =
               aggregatorData.createAggregatorInitialValue(aggregatorName);
           aggregatorValue.readFields(input);

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
index 14c8c0d..62ab7f1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
@@ -44,7 +44,7 @@ public abstract class WritableRequest<I extends WritableComparable,
   public static final int UNKNOWN_SIZE = -1;
 
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
+  protected ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Client id */
   private int clientId = -1;
   /** Request id */

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
index 325d91f..2bc08e9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
@@ -18,15 +18,24 @@
 
 package org.apache.giraph.master;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.bsp.SuperstepState;
-import org.apache.giraph.comm.MasterClient;
-import org.apache.giraph.comm.aggregators.AggregatorUtils;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Map;
+
 import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.aggregators.AggregatorWrapper;
 import org.apache.giraph.aggregators.AggregatorWriter;
+import org.apache.giraph.aggregators.ClassAggregatorFactory;
 import org.apache.giraph.bsp.BspService;
+import org.apache.giraph.bsp.SuperstepState;
+import org.apache.giraph.comm.MasterClient;
+import org.apache.giraph.comm.aggregators.AggregatorUtils;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.MasterLoggingAggregator;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -35,12 +44,6 @@ import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.AbstractMap;
-import java.util.Map;
-
 /** Handler for aggregators on master */
 public class MasterAggregatorHandler implements MasterAggregatorUsage,
     Writable {
@@ -106,7 +109,17 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
     checkAggregatorName(name);
-    return registerAggregator(name, aggregatorClass, false) != null;
+    ClassAggregatorFactory<A> aggregatorFactory =
+        new ClassAggregatorFactory<A>(aggregatorClass, conf);
+    return registerAggregator(name, aggregatorFactory, false) != null;
+  }
+
+  @Override
+  public <A extends Writable> boolean registerAggregator(String name,
+      WritableFactory<? extends Aggregator<A>> aggregator) throws
+      InstantiationException, IllegalAccessException {
+    checkAggregatorName(name);
+    return registerAggregator(name, aggregator, false) != null;
   }
 
   @Override
@@ -114,7 +127,9 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
     checkAggregatorName(name);
-    return registerAggregator(name, aggregatorClass, true) != null;
+    ClassAggregatorFactory<A> aggregatorFactory =
+        new ClassAggregatorFactory<A>(aggregatorClass, conf);
+    return registerAggregator(name, aggregatorFactory, true) != null;
   }
 
   /**
@@ -134,22 +149,22 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
   /**
    * Helper function for registering aggregators.
    *
-   * @param name            Name of the aggregator
-   * @param aggregatorClass Class of the aggregator
-   * @param persistent      Whether aggregator is persistent or not
-   * @param <A>             Aggregated value type
+   * @param name              Name of the aggregator
+   * @param aggregatorFactory Aggregator factory
+   * @param persistent        Whether aggregator is persistent or not
+   * @param <A>               Aggregated value type
    * @return Newly registered aggregator or aggregator which was previously
    *         created with selected name, if any
    */
   private <A extends Writable> AggregatorWrapper<A> registerAggregator
-  (String name, Class<? extends Aggregator<A>> aggregatorClass,
+  (String name, WritableFactory<? extends Aggregator<A>> aggregatorFactory,
       boolean persistent) throws InstantiationException,
       IllegalAccessException {
     AggregatorWrapper<A> aggregatorWrapper =
         (AggregatorWrapper<A>) aggregatorMap.get(name);
     if (aggregatorWrapper == null) {
       aggregatorWrapper =
-          new AggregatorWrapper<A>(aggregatorClass, persistent, conf);
+          new AggregatorWrapper<A>(aggregatorFactory, persistent, conf);
       aggregatorMap.put(name, (AggregatorWrapper<Writable>) aggregatorWrapper);
     }
     return aggregatorWrapper;
@@ -207,7 +222,7 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
       for (Map.Entry<String, AggregatorWrapper<Writable>> entry :
           aggregatorMap.entrySet()) {
         masterClient.sendAggregator(entry.getKey(),
-            entry.getValue().getAggregatorClass(),
+            entry.getValue().getAggregatorFactory(),
             entry.getValue().getPreviousAggregatedValue());
         progressable.progress();
       }
@@ -322,7 +337,7 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
     for (Map.Entry<String, AggregatorWrapper<Writable>> entry :
         aggregatorMap.entrySet()) {
       out.writeUTF(entry.getKey());
-      out.writeUTF(entry.getValue().getAggregatorClass().getName());
+      entry.getValue().getAggregatorFactory().write(out);
       out.writeBoolean(entry.getValue().isPersistent());
       entry.getValue().getPreviousAggregatedValue().write(out);
       progressable.progress();
@@ -336,15 +351,16 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
     try {
       for (int i = 0; i < numAggregators; i++) {
         String aggregatorName = in.readUTF();
-        String aggregatorClassName = in.readUTF();
+        WritableFactory<Aggregator<Writable>> aggregatorFactory =
+            WritableUtils.readWritableObject(in, conf);
         boolean isPersistent = in.readBoolean();
-        AggregatorWrapper<Writable> aggregator = registerAggregator(
+        AggregatorWrapper<Writable> aggregatorWrapper = registerAggregator(
             aggregatorName,
-            AggregatorUtils.getAggregatorClass(aggregatorClassName),
+            aggregatorFactory,
             isPersistent);
-        Writable value = aggregator.createInitialValue();
+        Writable value = aggregatorWrapper.createInitialValue();
         value.readFields(in);
-        aggregator.setPreviousAggregatedValue(value);
+        aggregatorWrapper.setPreviousAggregatedValue(value);
         progressable.progress();
       }
     } catch (InstantiationException e) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
index cadae67..91f5d24 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
@@ -20,6 +20,7 @@ package org.apache.giraph.master;
 
 import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.aggregators.AggregatorUsage;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -40,6 +41,21 @@ public interface MasterAggregatorUsage extends AggregatorUsage {
       InstantiationException, IllegalAccessException;
 
   /**
+   * Register an aggregator in preSuperstep() and/or preApplication(). This
+   * aggregator will have its value reset at the end of each super step.
+   *
+   * Aggregator should either implement Writable, or have no-arg constructor.
+   *
+   * @param name of aggregator
+   * @param aggregatorFactory aggregator factory
+   * @param <A> Aggregator type
+   * @return True iff aggregator wasn't already registered
+   */
+  <A extends Writable> boolean registerAggregator(String name,
+      WritableFactory<? extends Aggregator<A>> aggregatorFactory) throws
+      InstantiationException, IllegalAccessException;
+
+  /**
    * Register persistent aggregator in preSuperstep() and/or
    * preApplication(). This aggregator will not reset value at the end of
    * super step.

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
index d77a9b5..c2a1f9a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
@@ -23,6 +23,7 @@ import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.Computation;
 import org.apache.giraph.graph.GraphState;
+import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
 
@@ -195,6 +196,13 @@ public abstract class MasterCompute
   }
 
   @Override
+  public final <A extends Writable> boolean registerAggregator(
+    String name, WritableFactory<? extends Aggregator<A>> aggregator)
+    throws InstantiationException, IllegalAccessException {
+    return masterAggregatorUsage.registerAggregator(name, aggregator);
+  }
+
+  @Override
   public final <A extends Writable> boolean registerPersistentAggregator(
       String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/utils/ArrayWritable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ArrayWritable.java b/giraph-core/src/main/java/org/apache/giraph/utils/ArrayWritable.java
new file mode 100644
index 0000000..9ea24c3
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ArrayWritable.java
@@ -0,0 +1,100 @@
+/*
+ * 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.utils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Array;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A Writable for arrays containing instances of a class. The elements of this
+ * writable must all be instances of the same class.
+ *
+ * @param <T> element type
+ */
+public class ArrayWritable<T extends Writable> implements Writable {
+  /** Element type class */
+  private Class<T> valueClass;
+  /** Array */
+  private T[] values;
+
+  /** Constructor */
+  public ArrayWritable() {
+  }
+
+  /**
+   * Constructor
+   * @param valueClass Element type class
+   * @param values Array of elements
+   */
+  public ArrayWritable(Class<T> valueClass, T[] values) {
+    Preconditions.checkNotNull(valueClass,
+        "valueClass cannot be null in ArrayWritable");
+    this.valueClass = valueClass;
+    this.values = values;
+  }
+
+  /**
+   * Get element type class
+   * @return element type class
+   */
+  public Class<T> getValueClass() {
+    return valueClass;
+  }
+
+  /**
+   * Set array
+   * @param values array
+   */
+  public void set(T[] values) { this.values = values; }
+
+  /**
+   * Ger array
+   * @return array
+   */
+  public T[] get() { return values; }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    valueClass = WritableUtils.readClass(in);
+    values = (T[]) Array.newInstance(valueClass, in.readInt());
+
+    for (int i = 0; i < values.length; i++) {
+      T value = (T) WritableFactories.newInstance(valueClass);
+      value.readFields(in);                       // read a value
+      values[i] = value;                          // store it in values
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Preconditions.checkNotNull(valueClass,
+        "valueClass cannot be null in ArrayWritable");
+    WritableUtils.writeClass(valueClass, out);
+    out.writeInt(values.length);                 // write values
+    for (int i = 0; i < values.length; i++) {
+      values[i].write(out);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java
new file mode 100644
index 0000000..43bed7e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java
@@ -0,0 +1,28 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Factory that can be serialized.
+ * @param <T> Type of object factory creates
+ */
+public interface WritableFactory<T> extends Writable, Factory<T> {
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 3f8382e..763f59d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -18,6 +18,18 @@
 
 package org.apache.giraph.utils;
 
+import static org.apache.hadoop.util.ReflectionUtils.newInstance;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.OutEdges;
@@ -33,18 +45,6 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.hadoop.util.ReflectionUtils.newInstance;
-
 /**
  * Helper static methods for working with Writable objects.
  */
@@ -70,6 +70,23 @@ public class WritableUtils {
   }
 
   /**
+   * Instantiate a new Writable, checking for NullWritable along the way.
+   *
+   * @param klass Class
+   * @param configuration Configuration
+   * @param <W> type
+   * @return new instance of class
+   */
+  public static <W extends Writable> W createWritable(
+      Class<W> klass,
+      ImmutableClassesGiraphConfiguration configuration) {
+    W result = createWritable(klass);
+    ConfigurationUtils.configureIfPossible(result, configuration);
+    return result;
+  }
+
+
+  /**
    * Read fields from byteArray to a Writeable object.
    *
    * @param byteArray Byte array to find the fields in.
@@ -616,4 +633,47 @@ public class WritableUtils {
       return null;
     }
   }
+
+  /**
+   * Write object to output stream
+   * @param object Object
+   * @param output Output stream
+   * @throws IOException
+   */
+  public static void writeWritableObject(
+    Writable object, DataOutput output)
+    throws IOException {
+    output.writeBoolean(object != null);
+    if (object != null) {
+      output.writeUTF(object.getClass().getName());
+      object.write(output);
+    }
+  }
+
+  /**
+   * Reads object from input stream
+   * @param input Input stream
+   * @param conf Configuration
+   * @param <T> Object type
+   * @return Object
+   * @throws IOException
+   */
+  public static <T extends Writable>
+  T readWritableObject(DataInput input,
+      ImmutableClassesGiraphConfiguration conf) throws IOException {
+    if (input.readBoolean()) {
+      String className = input.readUTF();
+      try {
+        T object =
+            (T) ReflectionUtils.newInstance(Class.forName(className), conf);
+        object.readFields(input);
+        return object;
+      } catch (ClassNotFoundException e) {
+        throw new IllegalStateException("readWritableObject: No class found " +
+            className);
+      }
+    } else {
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
index 9bfd7b5..45ca665 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
@@ -18,14 +18,19 @@
 
 package org.apache.giraph.worker;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
-import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
 import org.apache.giraph.comm.aggregators.AggregatedValueOutputStream;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
 import org.apache.giraph.comm.aggregators.OwnerAggregatorServerData;
-import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.Factory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -33,10 +38,6 @@ import org.apache.log4j.Logger;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
 /**
  * Handler for aggregators on worker. Provides the aggregated values and
  * performs aggregations from user vertex code (thread-safe). Also has
@@ -58,10 +59,13 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
   private static final Logger LOG =
       Logger.getLogger(WorkerAggregatorHandler.class);
   /** Map of values from previous superstep */
-  private Map<String, Writable> previousAggregatedValueMap =
+  private final Map<String, Writable> previousAggregatedValueMap =
       Maps.newHashMap();
+  /** Map of aggregator factories for current superstep */
+  private final Map<String, Factory<Aggregator<Writable>>>
+  currentAggregatorFactoryMap = Maps.newHashMap();
   /** Map of aggregators for current superstep */
-  private Map<String, Aggregator<Writable>> currentAggregatorMap =
+  private final Map<String, Aggregator<Writable>> currentAggregatorMap =
       Maps.newHashMap();
   /** Service worker */
   private final CentralizedServiceWorker<?, ?, ?> serviceWorker;
@@ -143,7 +147,8 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
     // Wait for all other aggregators and store them
     allAggregatorData.fillNextSuperstepMapsWhenReady(
         getOtherWorkerIdsSet(), previousAggregatedValueMap,
-        currentAggregatorMap);
+        currentAggregatorFactoryMap);
+    fillAndInitAggregatorsMap(currentAggregatorMap);
     allAggregatorData.reset();
     if (LOG.isDebugEnabled()) {
       LOG.debug("prepareSuperstep: Aggregators prepared");
@@ -151,6 +156,25 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
   }
 
   /**
+   * Fills aggregators map from currentAggregatorFactoryMap.
+   * All aggregators in this map will be set to initial value.
+   * @param aggregatorMap Map to fill.
+   */
+  private void fillAndInitAggregatorsMap(
+      Map<String, Aggregator<Writable>> aggregatorMap) {
+    for (Map.Entry<String, Factory<Aggregator<Writable>>> entry :
+        currentAggregatorFactoryMap.entrySet()) {
+      Aggregator<Writable> aggregator =
+          aggregatorMap.get(entry.getKey());
+      if (aggregator == null) {
+        aggregatorMap.put(entry.getKey(), entry.getValue().create());
+      } else {
+        aggregator.reset();
+      }
+    }
+  }
+
+  /**
    * Send aggregators to their owners and in the end to the master
    *
    * @param requestProcessor Request processor for aggregators
@@ -286,13 +310,7 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
     public ThreadLocalWorkerAggregatorUsage() {
       threadAggregatorMap = Maps.newHashMapWithExpectedSize(
           WorkerAggregatorHandler.this.currentAggregatorMap.size());
-      for (Map.Entry<String, Aggregator<Writable>> entry :
-          WorkerAggregatorHandler.this.currentAggregatorMap.entrySet()) {
-        threadAggregatorMap.put(entry.getKey(),
-            AggregatorUtils.newAggregatorInstance(
-                (Class<Aggregator<Writable>>) entry.getValue().getClass(),
-                conf));
-      }
+      fillAndInitAggregatorsMap(threadAggregatorMap);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/ce97134d/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java b/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java
new file mode 100644
index 0000000..2898647
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.aggregators;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.giraph.utils.ArrayWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Test;
+
+public class TestArrayAggregator {
+  @Test
+  public void testMaxAggregator() {
+    Aggregator<ArrayWritable<LongWritable>> max = new ArrayAggregatorFactory<>(2, LongMaxAggregator.class).create();
+
+    ArrayWritable<LongWritable> tmp = max.createInitialValue();
+
+    tmp.get()[0].set(2);
+    max.aggregate(tmp);
+
+    tmp.get()[0].set(3);
+    tmp.get()[1].set(1);
+    max.aggregate(tmp);
+
+    assertEquals(3L, max.getAggregatedValue().get()[0].get());
+    assertEquals(1L, max.getAggregatedValue().get()[1].get());
+
+    tmp.get()[0].set(-1);
+    tmp.get()[1].set(-1);
+    max.setAggregatedValue(tmp);
+
+    assertEquals(-1L, max.getAggregatedValue().get()[0].get());
+    assertEquals(-1L, max.getAggregatedValue().get()[1].get());
+  }
+}


[24/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-937 - Upgrade Hive I/O to 0.26

Summary: As above

Test Plan: mvn compile

Reviewers: maja.kabiljo

Reviewed By: maja.kabiljo

Differential Revision: https://reviews.facebook.net/D21237


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/d8b44397
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/d8b44397
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/d8b44397

Branch: refs/heads/release-1.1
Commit: d8b443979d49517b40856009647231c1dfd17b4c
Parents: fc26773
Author: Avery Ching <ac...@fb.com>
Authored: Wed Aug 6 16:39:55 2014 -0700
Committer: Avery Ching <ac...@fb.com>
Committed: Wed Aug 6 17:01:33 2014 -0700

----------------------------------------------------------------------
 CHANGELOG | 2 ++
 pom.xml   | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/d8b44397/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 300215a..be3c59f 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-937 - Upgrade Hive I/O to 0.26. (aching)
+
   GIRAPH-893: Implement preLoad & postSave on workerObservers (pavanka)
 
   GIRAPH-936: AsyncMessageStoreWrapper threads are not daemonized (edunov via majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/d8b44397/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index de25499..ed2a98c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -296,7 +296,7 @@ under the License.
     <dep.hbase.version>0.94.16</dep.hbase.version>
     <dep.hcatalog.version>0.5.0-incubating</dep.hcatalog.version>
     <dep.hive.version>0.11.0</dep.hive.version>
-    <dep.hiveio.version>0.21</dep.hiveio.version>
+    <dep.hiveio.version>0.26</dep.hiveio.version>
     <dep.jaxb-impl.version>2.2.4-1</dep.jaxb-impl.version>
     <dep.jersey.version>1.17</dep.jersey.version>
     <dep.jetty.version>6.1.26</dep.jetty.version>


[08/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-923: Upgrade Netty version to a latest stable one (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/666d5fdd
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/666d5fdd
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/666d5fdd

Branch: refs/heads/release-1.1
Commit: 666d5fdd966c5111316519a862336144223652ba
Parents: 441ec8b
Author: Pavan Kumar <pa...@fb.com>
Authored: Mon Jul 7 16:31:29 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Mon Jul 7 16:31:29 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                                       | 2 ++
 .../main/java/org/apache/giraph/conf/GiraphConfiguration.java   | 5 ++++-
 pom.xml                                                         | 2 +-
 3 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/666d5fdd/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 2929c66..43aea7a 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-923: Upgrade Netty version to a latest stable one (pavanka)
+
   GIRAPH-916: Wrong number of vertices stored reported to command line (majakabiljo)
 
   GIRAPH-919: Add worker to worker communication (majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/666d5fdd/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 b6384dc..ee88b04 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
@@ -854,9 +854,12 @@ public class GiraphConfiguration extends Configuration
    */
   public ByteBufAllocator getNettyAllocator() {
     if (nettyBufferAllocator == null) {
+      int nArenas = Math.max(GiraphConstants.NETTY_CLIENT_THREADS.get(this),
+          GiraphConstants.NETTY_SERVER_THREADS.get(this));
       if (NETTY_USE_POOLED_ALLOCATOR.get(this)) { // Use pooled allocator
         nettyBufferAllocator = new PooledByteBufAllocator(
-          NETTY_USE_DIRECT_MEMORY.get(this));
+            NETTY_USE_DIRECT_MEMORY.get(this), nArenas, nArenas,
+            8192, 11, 0, 0, 0);
       } else { // Use un-pooled allocator
         // Note: Current default settings create un-pooled heap allocator
         nettyBufferAllocator = new UnpooledByteBufAllocator(

http://git-wip-us.apache.org/repos/asf/giraph/blob/666d5fdd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 183f2df..de25499 100644
--- a/pom.xml
+++ b/pom.xml
@@ -308,7 +308,7 @@ under the License.
     <dep.mockito.version>1.9.5</dep.mockito.version>
     <!-- note: old version of netty is required by hadoop_facebook for tests to succeed -->
     <dep.oldnetty.version>3.2.2.Final</dep.oldnetty.version>
-    <dep.netty.version>4.0.14.Final</dep.netty.version>
+    <dep.netty.version>4.0.21.Final</dep.netty.version>
     <dep.paranamer.version>2.3</dep.paranamer.version>
     <dep.slf4j.version>1.7.5</dep.slf4j.version>
     <dep.tinkerpop.rexter.version>2.4.0</dep.tinkerpop.rexter.version>


[10/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-925: Unit tests should pass even if zookeeper port not available (edunov via pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/7f9218ae
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/7f9218ae
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/7f9218ae

Branch: refs/heads/release-1.1
Commit: 7f9218aeb6410929ddada81b4fabb17bf8636a4c
Parents: 4223ccc
Author: Pavan Kumar <pa...@fb.com>
Authored: Tue Jul 8 10:58:09 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Tue Jul 8 10:59:50 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 +
 .../giraph/utils/InternalVertexRunner.java      | 63 ++++++++++++++++----
 2 files changed, 54 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/7f9218ae/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index ea2f911..834b45f 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-925: Unit tests should pass even if zookeeper port not available (edunov via pavanka)
+
   GIRAPH-713: Provide an option to do request compression (pavanka)
 
   GIRAPH-923: Upgrade Netty version to a latest stable one (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/7f9218ae/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
index 09dd46d..2c4606f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
@@ -38,6 +38,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.ServerSocket;
 import java.util.Properties;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -55,8 +56,10 @@ import java.util.concurrent.TimeUnit;
  */
 @SuppressWarnings("unchecked")
 public class InternalVertexRunner {
-  /** ZooKeeper port to use for tests */
-  public static final int LOCAL_ZOOKEEPER_PORT = 22182;
+  /** Range of ZooKeeper ports to use for tests */
+  public static final int LOCAL_ZOOKEEPER_PORT_FROM = 22182;
+  /** Range of ZooKeeper ports to use for tests */
+  public static final int LOCAL_ZOOKEEPER_PORT_TO = 65535;
 
   /** Logger */
   private static final Logger LOG =
@@ -166,11 +169,13 @@ public class InternalVertexRunner {
         FileUtils.writeLines(edgeInputFile, edgeInputData);
       }
 
+      int localZookeeperPort = findAvailablePort();
+
       conf.setWorkerConfiguration(1, 1, 100.0f);
       GiraphConstants.SPLIT_MASTER_WORKER.set(conf, false);
       GiraphConstants.LOCAL_TEST_MODE.set(conf, true);
       conf.setZookeeperList("localhost:" +
-        String.valueOf(LOCAL_ZOOKEEPER_PORT));
+          String.valueOf(localZookeeperPort));
 
       conf.set(GiraphConstants.ZOOKEEPER_DIR, zkDir.toString());
       GiraphConstants.ZOOKEEPER_MANAGER_DIRECTORY.set(conf,
@@ -190,10 +195,10 @@ public class InternalVertexRunner {
             new Path(edgeInputFile.toString()));
       }
       FileOutputFormat.setOutputPath(job.getInternalJob(),
-                                     new Path(outputDir.toString()));
+          new Path(outputDir.toString()));
 
       // Configure a local zookeeper instance
-      Properties zkProperties = configLocalZooKeeper(zkDir);
+      Properties zkProperties = configLocalZooKeeper(zkDir, localZookeeperPort);
 
       QuorumPeerConfig qpConfig = new QuorumPeerConfig();
       qpConfig.parseProperties(zkProperties);
@@ -227,8 +232,8 @@ public class InternalVertexRunner {
    * @throws Exception if anything goes wrong
    */
   public static <I extends WritableComparable,
-    V extends Writable,
-    E extends Writable> void run(
+      V extends Writable,
+      E extends Writable> void run(
       GiraphConfiguration conf,
       TestGraph<I, V, E> graph) throws Exception {
     File tmpDir = null;
@@ -247,11 +252,13 @@ public class InternalVertexRunner {
 
       InMemoryVertexInputFormat.setGraph(graph);
 
+      int localZookeeperPort = findAvailablePort();
+
       conf.setWorkerConfiguration(1, 1, 100.0f);
       GiraphConstants.SPLIT_MASTER_WORKER.set(conf, false);
       GiraphConstants.LOCAL_TEST_MODE.set(conf, true);
       GiraphConstants.ZOOKEEPER_LIST.set(conf, "localhost:" +
-          String.valueOf(LOCAL_ZOOKEEPER_PORT));
+          String.valueOf(localZookeeperPort));
 
       conf.set(GiraphConstants.ZOOKEEPER_DIR, zkDir.toString());
       GiraphConstants.ZOOKEEPER_MANAGER_DIRECTORY.set(conf,
@@ -259,7 +266,7 @@ public class InternalVertexRunner {
       GiraphConstants.CHECKPOINT_DIRECTORY.set(conf, checkpointsDir.toString());
 
       // Configure a local zookeeper instance
-      Properties zkProperties = configLocalZooKeeper(zkDir);
+      Properties zkProperties = configLocalZooKeeper(zkDir, localZookeeperPort);
 
       QuorumPeerConfig qpConfig = new QuorumPeerConfig();
       qpConfig.parseProperties(zkProperties);
@@ -298,14 +305,16 @@ public class InternalVertexRunner {
    * Configuration options for running local ZK.
    *
    * @param zkDir directory for ZK to hold files in.
+   * @param zookeeperPort port zookeeper will listen on
    * @return Properties configured for local ZK.
    */
-  private static Properties configLocalZooKeeper(File zkDir) {
+  private static Properties configLocalZooKeeper(File zkDir,
+                                                 int zookeeperPort) {
     Properties zkProperties = new Properties();
     zkProperties.setProperty("tickTime", "2000");
     zkProperties.setProperty("dataDir", zkDir.getAbsolutePath());
     zkProperties.setProperty("clientPort",
-        String.valueOf(LOCAL_ZOOKEEPER_PORT));
+        String.valueOf(zookeeperPort));
     zkProperties.setProperty("maxClientCnxns", "10000");
     zkProperties.setProperty("minSessionTimeout", "10000");
     zkProperties.setProperty("maxSessionTimeout", "100000");
@@ -316,6 +325,38 @@ public class InternalVertexRunner {
   }
 
   /**
+   * Scans for available port. Returns first port where
+   * we can open server socket.
+   * Note: if another process opened port with SO_REUSEPORT then this
+   * function may return port that is in use. It actually happens
+   * with NetCat on Mac.
+   * @return available port
+   */
+  private static int findAvailablePort() {
+    for (int port = LOCAL_ZOOKEEPER_PORT_FROM;
+         port < LOCAL_ZOOKEEPER_PORT_TO; port++) {
+      ServerSocket ss = null;
+      try {
+        ss = new ServerSocket(port);
+        ss.setReuseAddress(true);
+        return port;
+      } catch (IOException e) {
+        LOG.info("findAvailablePort: port " + port + " is in use.");
+      } finally {
+        if (ss != null && !ss.isClosed()) {
+          try {
+            ss.close();
+          } catch (IOException e) {
+            LOG.info("findAvailablePort: can't close test socket", e);
+          }
+        }
+      }
+    }
+    throw new RuntimeException("No port found in the range [ " +
+        LOCAL_ZOOKEEPER_PORT_FROM + ", " + LOCAL_ZOOKEEPER_PORT_TO + ")");
+  }
+
+  /**
    * Extension of {@link ZooKeeperServerMain} that allows programmatic shutdown
    */
   private static class InternalZooKeeper extends ZooKeeperServerMain {


[04/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-922: SimpleEdgeStore has a bug causing NPE (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/64428a9d
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/64428a9d
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/64428a9d

Branch: refs/heads/release-1.1
Commit: 64428a9d0f8bca0edb1ccf34d281e89747b295f7
Parents: 58576c8
Author: Pavan Kumar <pa...@fb.com>
Authored: Wed Jun 18 16:14:40 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Wed Jun 18 16:14:40 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                               |  2 ++
 .../java/org/apache/giraph/edge/AbstractEdgeStore.java  | 12 +++++-------
 .../java/org/apache/giraph/edge/SimpleEdgeStore.java    |  7 +++----
 .../org/apache/giraph/edge/primitives/IntEdgeStore.java |  7 +++----
 .../apache/giraph/edge/primitives/LongEdgeStore.java    |  7 +++----
 5 files changed, 16 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/64428a9d/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 659edfd..d315a9f 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-922: SimpleEdgeStore has a bug causing NPE (pavanka)
+
   GIRAPH-915: With BigDataIO some messages can get ignored (majakabiljo via pavanka)
 
   GIRAPH-918: GIRAPH-908 has a small bug reg counting entries (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/64428a9d/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
index ee53718..b40ac00 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
@@ -121,14 +121,12 @@ public abstract class AbstractEdgeStore<I extends WritableComparable,
   protected abstract Map<K, OutEdges<I, E>> getPartitionEdges(int partitionId);
 
   /**
-   * Remove and return the OutEdges for a given partition
+   * Return the OutEdges for a given partition
    *
    * @param entry for vertexId key
-   * @param partitionEdges map of out-edges for vertices in a partition
    * @return out edges
    */
-  protected abstract OutEdges<I, E> removePartitionEdges(Et entry,
-    Map<K, OutEdges<I, E>> partitionEdges);
+  protected abstract OutEdges<I, E> getPartitionEdges(Et entry);
 
   /**
    * Get iterator for partition edges
@@ -223,10 +221,10 @@ public abstract class AbstractEdgeStore<I extends WritableComparable,
               // process all vertices in given partition
               while (iterator.hasNext()) {
                 Et entry = iterator.next();
-                I vertexId = getVertexId(entry,
-                    representativeVertexId);
+                I vertexId = getVertexId(entry, representativeVertexId);
                 OutEdges<I, E> outEdges = convertInputToComputeEdges(
-                    removePartitionEdges(entry, partitionEdges));
+                  getPartitionEdges(entry));
+                iterator.remove();
                 Vertex<I, V, E> vertex = partition.getVertex(vertexId);
                 // If the source vertex doesn't exist, create it. Otherwise,
                 // just set the edges.

http://git-wip-us.apache.org/repos/asf/giraph/blob/64428a9d/giraph-core/src/main/java/org/apache/giraph/edge/SimpleEdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/SimpleEdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/SimpleEdgeStore.java
index a6a3356..3eb97d6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/SimpleEdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/SimpleEdgeStore.java
@@ -87,10 +87,9 @@ public class SimpleEdgeStore<I extends WritableComparable,
   }
 
   @Override
-  protected OutEdges<I, E> removePartitionEdges(
-      Map.Entry<I, OutEdges<I, E>> entry,
-      Map<I, OutEdges<I, E>> partitionEdges) {
-    return partitionEdges.put(entry.getKey(), null);
+  protected OutEdges<I, E> getPartitionEdges(
+    Map.Entry<I, OutEdges<I, E>> entry) {
+    return entry.getValue();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/64428a9d/giraph-core/src/main/java/org/apache/giraph/edge/primitives/IntEdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/primitives/IntEdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/primitives/IntEdgeStore.java
index 826c685..b138f49 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/primitives/IntEdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/primitives/IntEdgeStore.java
@@ -75,10 +75,9 @@ public class IntEdgeStore<V extends Writable, E extends Writable>
   }
 
   @Override
-  protected OutEdges<IntWritable, E> removePartitionEdges(
-    Int2ObjectMap.Entry<OutEdges<IntWritable, E>> entry,
-    Map<Integer, OutEdges<IntWritable, E>> partitionEdges) {
-    return partitionEdges.put(entry.getIntKey(), null);
+  protected OutEdges<IntWritable, E> getPartitionEdges(
+    Int2ObjectMap.Entry<OutEdges<IntWritable, E>> entry) {
+    return entry.getValue();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/64428a9d/giraph-core/src/main/java/org/apache/giraph/edge/primitives/LongEdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/primitives/LongEdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/primitives/LongEdgeStore.java
index 486410f..61f908a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/primitives/LongEdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/primitives/LongEdgeStore.java
@@ -76,10 +76,9 @@ public class LongEdgeStore<V extends Writable, E extends Writable>
 
 
   @Override
-  protected OutEdges<LongWritable, E> removePartitionEdges(
-      Long2ObjectMap.Entry<OutEdges<LongWritable, E>> entry,
-      Map<Long, OutEdges<LongWritable, E>> partitionEdges) {
-    return partitionEdges.put(entry.getLongKey(), null);
+  protected OutEdges<LongWritable, E> getPartitionEdges(
+    Long2ObjectMap.Entry<OutEdges<LongWritable, E>> entry) {
+    return entry.getValue();
   }
 
   @Override


[46/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-956. disable tests for a few components in hadoop_2 profile


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/d49edb82
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/d49edb82
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/d49edb82

Branch: refs/heads/release-1.1
Commit: d49edb82f13f1e9c295f21d966bcb246d0bae3bd
Parents: 54a1a8d
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Sat Oct 25 16:45:21 2014 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Sat Oct 25 16:45:21 2014 -0700

----------------------------------------------------------------------
 giraph-accumulo/pom.xml |  4 ++++
 giraph-hbase/pom.xml    |  4 ++++
 giraph-hive/pom.xml     | 10 ++++++++++
 pom.xml                 |  2 ++
 4 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/d49edb82/giraph-accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-accumulo/pom.xml b/giraph-accumulo/pom.xml
index b5e6121..7c2016f 100644
--- a/giraph-accumulo/pom.xml
+++ b/giraph-accumulo/pom.xml
@@ -67,6 +67,7 @@ under the License.
         <artifactId>maven-surefire-plugin</artifactId>
         <version>2.6</version>
         <configuration>
+          <skip>${surefire.skip}</skip>
           <systemProperties>
             <property>
               <name>prop.jarLocation</name>
@@ -112,6 +113,9 @@ under the License.
 
     <profile>
       <id>hadoop_2</id>
+      <properties>
+        <surefire.skip>true</surefire.skip>
+      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/giraph/blob/d49edb82/giraph-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-hbase/pom.xml b/giraph-hbase/pom.xml
index 3782d38..d6bdc92 100644
--- a/giraph-hbase/pom.xml
+++ b/giraph-hbase/pom.xml
@@ -71,6 +71,7 @@ under the License.
         <artifactId>maven-surefire-plugin</artifactId>
         <version>2.6</version>
         <configuration>
+          <skip>${surefire.skip}</skip>
           <systemProperties>
             <property>
               <name>prop.jarLocation</name>
@@ -116,6 +117,9 @@ under the License.
 
     <profile>
       <id>hadoop_2</id>
+      <properties>
+        <surefire.skip>true</surefire.skip>
+      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/giraph/blob/d49edb82/giraph-hive/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-hive/pom.xml b/giraph-hive/pom.xml
index c20c308..dbe93e2 100644
--- a/giraph-hive/pom.xml
+++ b/giraph-hive/pom.xml
@@ -37,6 +37,15 @@ under the License.
     <top.dir>${project.basedir}/..</top.dir>
   </properties>
 
+  <profiles>
+    <profile>
+      <id>hadoop_2</id>
+      <properties>
+        <surefire.skip>true</surefire.skip>
+      </properties>
+    </profile>
+  </profiles>
+
   <build>
     <plugins>
       <plugin>
@@ -67,6 +76,7 @@ under the License.
         <artifactId>maven-surefire-plugin</artifactId>
         <version>2.6</version>
         <configuration>
+          <skip>${surefire.skip}</skip>
           <systemProperties>
             <property>
               <name>prop.jarLocation</name>

http://git-wip-us.apache.org/repos/asf/giraph/blob/d49edb82/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6804de9..1833011 100644
--- a/pom.xml
+++ b/pom.xml
@@ -272,6 +272,8 @@ under the License.
     <project.build.targetJdk>1.7</project.build.targetJdk>
     <giraph.maven.dependency.plugin.skip>false</giraph.maven.dependency.plugin.skip>
     <giraph.maven.duplicate.finder.skip>false</giraph.maven.duplicate.finder.skip>
+    <!-- This lets modules skip unit tests. More details: GIRAPH-957 --> 
+    <surefire.skip>false</surefire.skip>
 
     <dep.avro.version>1.7.6</dep.avro.version>
     <dep.accumulo.version>1.4.0</dep.accumulo.version>


[27/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-943: Perf regression due to netty 4.0.21 (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/bc9f823e
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/bc9f823e
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/bc9f823e

Branch: refs/heads/release-1.1
Commit: bc9f823e23d110d3c54d6eb0f5ccf7eff155a6b7
Parents: 181ce39
Author: Pavan Kumar <pa...@fb.com>
Authored: Tue Aug 12 14:09:48 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Tue Aug 12 14:09:48 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                                       | 2 ++
 .../main/java/org/apache/giraph/conf/GiraphConfiguration.java   | 5 +----
 pom.xml                                                         | 2 +-
 3 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/bc9f823e/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 08abaaa..66136b2 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-943: Perf regression due to netty 4.0.21 (pavanka)
+
   GIRAPH-935: Loosen modifiers when needed (ikabiljo via majakabiljo)
 
   GIRAPH-937 - Upgrade Hive I/O to 0.26. (aching)

http://git-wip-us.apache.org/repos/asf/giraph/blob/bc9f823e/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 f35fbc1..7f1a764 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
@@ -865,12 +865,9 @@ public class GiraphConfiguration extends Configuration
    */
   public ByteBufAllocator getNettyAllocator() {
     if (nettyBufferAllocator == null) {
-      int nArenas = Math.max(GiraphConstants.NETTY_CLIENT_THREADS.get(this),
-          GiraphConstants.NETTY_SERVER_THREADS.get(this));
       if (NETTY_USE_POOLED_ALLOCATOR.get(this)) { // Use pooled allocator
         nettyBufferAllocator = new PooledByteBufAllocator(
-            NETTY_USE_DIRECT_MEMORY.get(this), nArenas, nArenas,
-            8192, 11, 0, 0, 0);
+          NETTY_USE_DIRECT_MEMORY.get(this));
       } else { // Use un-pooled allocator
         // Note: Current default settings create un-pooled heap allocator
         nettyBufferAllocator = new UnpooledByteBufAllocator(

http://git-wip-us.apache.org/repos/asf/giraph/blob/bc9f823e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ed2a98c..672ec44 100644
--- a/pom.xml
+++ b/pom.xml
@@ -308,7 +308,7 @@ under the License.
     <dep.mockito.version>1.9.5</dep.mockito.version>
     <!-- note: old version of netty is required by hadoop_facebook for tests to succeed -->
     <dep.oldnetty.version>3.2.2.Final</dep.oldnetty.version>
-    <dep.netty.version>4.0.21.Final</dep.netty.version>
+    <dep.netty.version>4.0.14.Final</dep.netty.version>
     <dep.paranamer.version>2.3</dep.paranamer.version>
     <dep.slf4j.version>1.7.5</dep.slf4j.version>
     <dep.tinkerpop.rexter.version>2.4.0</dep.tinkerpop.rexter.version>


[14/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-929: setIfUnset for EnumConfOption (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/25d912cd
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/25d912cd
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/25d912cd

Branch: refs/heads/release-1.1
Commit: 25d912cd910712a7c1e7da98db657f6dce877e7e
Parents: f31e9a3
Author: Pavan Kumar <pa...@fb.com>
Authored: Mon Jul 14 10:18:37 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Mon Jul 14 10:18:37 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                               |  2 ++
 .../java/org/apache/giraph/conf/EnumConfOption.java     | 12 ++++++++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/25d912cd/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 0263749..e1c1227 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-929: setIfUnset for EnumConfOption (pavanka)  
+
   GIRAPH-912: Support succinct representation of messages in messagestores (pavanka)
 
   GIRAPH-903: Detect crashes of Netty threads (edunov via pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/25d912cd/giraph-core/src/main/java/org/apache/giraph/conf/EnumConfOption.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/EnumConfOption.java b/giraph-core/src/main/java/org/apache/giraph/conf/EnumConfOption.java
index 2a65ec0..a0effe1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/EnumConfOption.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/EnumConfOption.java
@@ -95,4 +95,16 @@ public class EnumConfOption<T extends Enum<T>> extends AbstractConfOption {
   public void set(Configuration conf, Enum<T> value) {
     conf.set(getKey(), value.name());
   }
+
+  /**
+   * Set value if it's not already present
+   *
+   * @param conf Configuration
+   * @param value to set
+   */
+  public void setIfUnset(Configuration conf, Enum<T> value) {
+    if (!contains(conf)) {
+      set(conf, value);
+    }
+  }
 }


[47/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Preparing for the 1.1.0 release


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/4c139ee3
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/4c139ee3
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/4c139ee3

Branch: refs/heads/release-1.1
Commit: 4c139ee36df19dd138be21f65349eb2cebbbc593
Parents: d49edb8
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Thu Jun 26 14:15:21 2014 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Sat Oct 25 18:21:09 2014 -0700

----------------------------------------------------------------------
 giraph-accumulo/pom.xml                  | 2 +-
 giraph-core/pom.xml                      | 2 +-
 giraph-dist/pom.xml                      | 2 +-
 giraph-examples/pom.xml                  | 4 ++--
 giraph-gora/pom.xml                      | 2 +-
 giraph-hbase/pom.xml                     | 2 +-
 giraph-hcatalog/pom.xml                  | 2 +-
 giraph-hive/pom.xml                      | 2 +-
 giraph-rexster/giraph-kibble/pom.xml     | 2 +-
 giraph-rexster/giraph-rexster-io/pom.xml | 2 +-
 giraph-rexster/pom.xml                   | 2 +-
 pom.xml                                  | 2 +-
 src/site/xdoc/quick_start.xml            | 8 ++++----
 src/site/xdoc/rexster.xml                | 2 +-
 14 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-accumulo/pom.xml b/giraph-accumulo/pom.xml
index 7c2016f..f35b041 100644
--- a/giraph-accumulo/pom.xml
+++ b/giraph-accumulo/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-accumulo</artifactId>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-core/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-core/pom.xml b/giraph-core/pom.xml
index 23f6666..85b9377 100644
--- a/giraph-core/pom.xml
+++ b/giraph-core/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-core</artifactId>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-dist/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-dist/pom.xml b/giraph-dist/pom.xml
index bac8a24..3767793 100644
--- a/giraph-dist/pom.xml
+++ b/giraph-dist/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
 
   <artifactId>giraph-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-examples/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-examples/pom.xml b/giraph-examples/pom.xml
index f95edcb..a2f2ef5 100644
--- a/giraph-examples/pom.xml
+++ b/giraph-examples/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-examples</artifactId>
   <packaging>jar</packaging>
@@ -429,7 +429,7 @@ under the License.
     <dependency>
       <groupId>org.apache.giraph</groupId>
       <artifactId>giraph-core</artifactId>
-      <version>1.1.0-SNAPSHOT</version>
+      <version>1.1.0</version>
       <type>test-jar</type>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-gora/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-gora/pom.xml b/giraph-gora/pom.xml
index 7f499c6..8c30d30 100644
--- a/giraph-gora/pom.xml
+++ b/giraph-gora/pom.xml
@@ -16,7 +16,7 @@
     <parent>
         <groupId>org.apache.giraph</groupId>
         <artifactId>giraph-parent</artifactId>
-        <version>1.1.0-SNAPSHOT</version>
+        <version>1.1.0</version>
     </parent>
     <artifactId>giraph-gora</artifactId>
     <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-hbase/pom.xml b/giraph-hbase/pom.xml
index d6bdc92..fbea640 100644
--- a/giraph-hbase/pom.xml
+++ b/giraph-hbase/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-hbase</artifactId>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-hcatalog/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/pom.xml b/giraph-hcatalog/pom.xml
index af084ac..e1a191d 100644
--- a/giraph-hcatalog/pom.xml
+++ b/giraph-hcatalog/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-hcatalog</artifactId>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-hive/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-hive/pom.xml b/giraph-hive/pom.xml
index dbe93e2..5364f13 100644
--- a/giraph-hive/pom.xml
+++ b/giraph-hive/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-hive</artifactId>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-rexster/giraph-kibble/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-rexster/giraph-kibble/pom.xml b/giraph-rexster/giraph-kibble/pom.xml
index 710d50c..c4f6d3d 100644
--- a/giraph-rexster/giraph-kibble/pom.xml
+++ b/giraph-rexster/giraph-kibble/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-rexster</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
 
   <artifactId>giraph-kibble</artifactId>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-rexster/giraph-rexster-io/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-rexster/giraph-rexster-io/pom.xml b/giraph-rexster/giraph-rexster-io/pom.xml
index 6bbb236..b44fa93 100644
--- a/giraph-rexster/giraph-rexster-io/pom.xml
+++ b/giraph-rexster/giraph-rexster-io/pom.xml
@@ -24,7 +24,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-rexster</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
   <artifactId>giraph-rexster-io</artifactId>
   <packaging>jar</packaging>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/giraph-rexster/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-rexster/pom.xml b/giraph-rexster/pom.xml
index 260d3de..3f9e6b7 100644
--- a/giraph-rexster/pom.xml
+++ b/giraph-rexster/pom.xml
@@ -25,7 +25,7 @@ under the License.
   <parent>
     <groupId>org.apache.giraph</groupId>
     <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.0</version>
   </parent>
 
   <artifactId>giraph-rexster</artifactId>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1833011..7941ce8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -31,7 +31,7 @@ under the License.
   <groupId>org.apache.giraph</groupId>
   <artifactId>giraph-parent</artifactId>
   <packaging>pom</packaging>
-  <version>1.1.0-SNAPSHOT</version>
+  <version>1.1.0</version>
 
   <name>Apache Giraph Parent</name>
   <url>http://giraph.apache.org/</url>

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/src/site/xdoc/quick_start.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/quick_start.xml b/src/site/xdoc/quick_start.xml
index cdef668..6793638 100644
--- a/src/site/xdoc/quick_start.xml
+++ b/src/site/xdoc/quick_start.xml
@@ -52,7 +52,7 @@ under the License.
           <li>Network mask: <tt>255.255.255.0</tt></li>
         </ul>
         <li>Apache Hadoop 0.20.203.0-RC1</li>
-        <li>Apache Giraph 1.1.0-SNAPSHOT</li>
+        <li>Apache Giraph 1.1.0</li>
       </ul>
     </section>
     <section name="Deploying Hadoop" id="qs_section_2">
@@ -195,7 +195,7 @@ su - hduser</source>
 source $HOME/.bashrc
 cd $GIRAPH_HOME
 mvn package -DskipTests</source>
-    <p>The argument <tt>-DskipTests</tt> will skip the testing phase. This may take a while on the first run because Maven is downloading the most recent artifacts (plugin JARs and other files) into your local repository. You may also need to execute the command a couple of times before it succeeds. This is because the remote server may time out before your downloads are complete. Once the packaging is successful, you will have the Giraph core JAR <tt>$GIRAPH_HOME/giraph-core/target/giraph-1.1.0-SNAPSHOT-for-hadoop-0.20.203.0-jar-with-dependencies.jar</tt> and Giraph examples JAR <tt>$GIRAPH_HOME/giraph-examples/target/giraph-examples-1.1.0-SNAPSHOT-for-hadoop-0.20.203.0-jar-with-dependencies.jar</tt>. You are done with deploying Giraph.</p>
+    <p>The argument <tt>-DskipTests</tt> will skip the testing phase. This may take a while on the first run because Maven is downloading the most recent artifacts (plugin JARs and other files) into your local repository. You may also need to execute the command a couple of times before it succeeds. This is because the remote server may time out before your downloads are complete. Once the packaging is successful, you will have the Giraph core JAR <tt>$GIRAPH_HOME/giraph-core/target/giraph-1.1.0-for-hadoop-0.20.203.0-jar-with-dependencies.jar</tt> and Giraph examples JAR <tt>$GIRAPH_HOME/giraph-examples/target/giraph-examples-1.1.0-for-hadoop-0.20.203.0-jar-with-dependencies.jar</tt>. You are done with deploying Giraph.</p>
     </section>
   
     <section name="Running a Giraph job" id="qs_section_6">
@@ -212,9 +212,9 @@ $HADOOP_HOME/bin/hadoop dfs -copyFromLocal /tmp/tiny_graph.txt /user/hduser/inpu
 $HADOOP_HOME/bin/hadoop dfs -ls /user/hduser/input</source>
       <p>We will use <tt>IdWithValueTextOutputFormat</tt> output file format, where each line consists of <tt>source_id length</tt> for each node in the input graph (the source node has a length of 0, by convention). You can now run the example by:</p>
       <source>
-$HADOOP_HOME/bin/hadoop jar $GIRAPH_HOME/giraph-examples/target/giraph-examples-1.1.0-SNAPSHOT-for-hadoop-0.20.203.0-jar-with-dependencies.jar org.apache.giraph.GiraphRunner org.apache.giraph.examples.SimpleShortestPathsComputation -vif org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat -vip /user/hduser/input/tiny_graph.txt -vof org.apache.giraph.io.formats.IdWithValueTextOutputFormat -op /user/hduser/output/shortestpaths -w 1</source>
+$HADOOP_HOME/bin/hadoop jar $GIRAPH_HOME/giraph-examples/target/giraph-examples-1.1.0-for-hadoop-0.20.203.0-jar-with-dependencies.jar org.apache.giraph.GiraphRunner org.apache.giraph.examples.SimpleShortestPathsComputation -vif org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat -vip /user/hduser/input/tiny_graph.txt -vof org.apache.giraph.io.formats.IdWithValueTextOutputFormat -op /user/hduser/output/shortestpaths -w 1</source>
       <p>Notice that the job is computed using a single worker using the argument <tt>-w</tt>. To get more information about running a Giraph job, run the following command:</p>
-      <source>$HADOOP_HOME/bin/hadoop jar $GIRAPH_HOME/giraph-examples/target/giraph-examples-1.1.0-SNAPSHOT-for-hadoop-0.20.203.0-jar-with-dependencies.jar org.apache.giraph.GiraphRunner -h</source>
+      <source>$HADOOP_HOME/bin/hadoop jar $GIRAPH_HOME/giraph-examples/target/giraph-examples-1.1.0-for-hadoop-0.20.203.0-jar-with-dependencies.jar org.apache.giraph.GiraphRunner -h</source>
       <p>This will output the following:</p>
       <source>
 usage: org.apache.giraph.utils.ConfigurationUtils [-aw &lt;arg&gt;] [-c &lt;arg&gt;]

http://git-wip-us.apache.org/repos/asf/giraph/blob/4c139ee3/src/site/xdoc/rexster.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/rexster.xml b/src/site/xdoc/rexster.xml
index cc55eab..bfa8311 100644
--- a/src/site/xdoc/rexster.xml
+++ b/src/site/xdoc/rexster.xml
@@ -61,7 +61,7 @@
 # Configuration
 export REXSTER_VERSION=2.4.0
 export HADOOP_VERSION=1.0.2
-export GIRAPH_VERSION=1.1.0-SNAPSHOT
+export GIRAPH_VERSION=1.1.0
 export GIRAPH_DIR=/path/to/giraph
 export REXSTER_DIR=/path/to/rexster
 export HADOOP_DIR=/path/to/hadoop


[15/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-921: Create ByteValueVertex to store vertex values as bytes without object instance (akyrola via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/0a901771
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/0a901771
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/0a901771

Branch: refs/heads/release-1.1
Commit: 0a90177161d65ff2802d3926599561adc985dc65
Parents: 25d912c
Author: Maja Kabiljo <ma...@fb.com>
Authored: Wed Jul 16 10:21:22 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Wed Jul 16 10:21:22 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../org/apache/giraph/conf/GiraphClasses.java   |  16 ++-
 .../apache/giraph/conf/GiraphConfiguration.java |  11 ++
 .../org/apache/giraph/conf/GiraphConstants.java |  12 +-
 .../org/apache/giraph/conf/GiraphTypes.java     |  11 ++
 .../ImmutableClassesGiraphConfiguration.java    |  11 +-
 .../apache/giraph/graph/ByteValueVertex.java    | 114 +++++++++++++++++++
 .../apache/giraph/utils/ConfigurationUtils.java |   1 +
 .../giraph/graph/TestByteValueVertex.java       |  85 ++++++++++++++
 .../apache/giraph/graph/TestVertexAndEdges.java |   4 +-
 10 files changed, 255 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index e1c1227..7287490 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-921: Create ByteValueVertex to store vertex values as bytes without object instance (akyrola via majakabiljo)
+
   GIRAPH-929: setIfUnset for EnumConfOption (pavanka)  
 
   GIRAPH-912: Support succinct representation of messages in messagestores (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/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 e7b18aa..7fe85f1 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
@@ -25,10 +25,11 @@ import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.factories.ComputationFactory;
 import org.apache.giraph.factories.DefaultComputationFactory;
 import org.apache.giraph.graph.Computation;
-import org.apache.giraph.graph.DefaultVertexValueCombiner;
 import org.apache.giraph.graph.DefaultVertexResolver;
-import org.apache.giraph.graph.VertexValueCombiner;
 import org.apache.giraph.graph.VertexResolver;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.graph.VertexValueCombiner;
+import org.apache.giraph.graph.DefaultVertexValueCombiner;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.EdgeOutputFormat;
 import org.apache.giraph.io.MappingInputFormat;
@@ -236,6 +237,17 @@ public class GiraphClasses<I extends WritableComparable,
     return giraphTypes.getVertexIdClass();
   }
 
+
+  /**
+   * Get Vertex implementation class
+   *
+   * @return Vertex implementation class
+   */
+  public Class<? extends Vertex> getVertexClass() {
+    return giraphTypes.getVertexClass();
+  }
+
+
   /**
    * Get Vertex Value class
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/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 953f49f..f35fbc1 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
@@ -23,6 +23,7 @@ import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.ReuseObjectsOutEdges;
 import org.apache.giraph.factories.ComputationFactory;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexValueCombiner;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.factories.VertexValueFactory;
@@ -173,6 +174,16 @@ public class GiraphConfiguration extends Configuration
   }
 
   /**
+   * Set the vertex implementation class
+   *
+   * @param vertexClass class of the vertex implementation
+   */
+  public final void setVertexClass(Class<? extends Vertex> vertexClass) {
+    VERTEX_CLASS.set(this, vertexClass);
+  }
+
+
+  /**
    * Set the vertex edges class used during edge-based input (if different
    * from the one used during computation)
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 ab0570f..81c0e0b 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
@@ -39,11 +39,13 @@ import org.apache.giraph.factories.MessageValueFactory;
 import org.apache.giraph.factories.VertexIdFactory;
 import org.apache.giraph.factories.VertexValueFactory;
 import org.apache.giraph.graph.Computation;
-import org.apache.giraph.graph.DefaultVertexValueCombiner;
+import org.apache.giraph.graph.DefaultVertex;
 import org.apache.giraph.graph.DefaultVertexResolver;
+import org.apache.giraph.graph.DefaultVertexValueCombiner;
 import org.apache.giraph.graph.Language;
-import org.apache.giraph.graph.VertexValueCombiner;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
+import org.apache.giraph.graph.VertexValueCombiner;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.EdgeOutputFormat;
 import org.apache.giraph.io.MappingInputFormat;
@@ -274,7 +276,11 @@ public interface GiraphConstants {
       ClassConfOption.create("giraph.vertexInputFilterClass",
           DefaultVertexInputFilter.class, VertexInputFilter.class,
           "VertexInputFilter class");
-
+  /** Vertex class */
+  ClassConfOption<Vertex> VERTEX_CLASS =
+      ClassConfOption.create("giraph.vertexClass",
+          DefaultVertex.class, Vertex.class,
+          "Vertex class");
   /** VertexOutputFormat class */
   ClassConfOption<VertexOutputFormat> VERTEX_OUTPUT_FORMAT_CLASS =
       ClassConfOption.create("giraph.vertexOutputFormatClass", null,

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/giraph-core/src/main/java/org/apache/giraph/conf/GiraphTypes.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphTypes.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphTypes.java
index 0727270..6c854f3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphTypes.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphTypes.java
@@ -17,6 +17,8 @@
  */
 package org.apache.giraph.conf;
 
+import org.apache.giraph.graph.DefaultVertex;
+import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -28,6 +30,7 @@ import static org.apache.giraph.conf.GiraphConstants.INCOMING_MESSAGE_VALUE_CLAS
 import static org.apache.giraph.conf.GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS;
 import static org.apache.giraph.conf.GiraphConstants.VERTEX_ID_CLASS;
 import static org.apache.giraph.conf.GiraphConstants.VERTEX_VALUE_CLASS;
+import static org.apache.giraph.conf.GiraphConstants.VERTEX_CLASS;
 import static org.apache.giraph.utils.ConfigurationUtils.getTypesHolderClass;
 import static org.apache.giraph.utils.ReflectionUtils.getTypeArguments;
 
@@ -50,6 +53,9 @@ public class GiraphTypes<I extends WritableComparable, V extends Writable,
   private Class<? extends Writable> incomingMessageValueClass;
   /** Outgoing message value class */
   private Class<? extends Writable> outgoingMessageValueClass;
+  /** Vertex implementation class */
+  private Class<? extends Vertex> vertexClass = DefaultVertex.class;
+
 
   /**
    * Empty Constructor
@@ -128,6 +134,7 @@ public class GiraphTypes<I extends WritableComparable, V extends Writable,
     edgeValueClass = (Class<E>) EDGE_VALUE_CLASS.get(conf);
     incomingMessageValueClass = INCOMING_MESSAGE_VALUE_CLASS.get(conf);
     outgoingMessageValueClass = OUTGOING_MESSAGE_VALUE_CLASS.get(conf);
+    vertexClass = VERTEX_CLASS.get(conf);
   }
 
   /**
@@ -189,6 +196,10 @@ public class GiraphTypes<I extends WritableComparable, V extends Writable,
     return vertexValueClass;
   }
 
+  public Class<? extends Vertex> getVertexClass() {
+    return vertexClass;
+  }
+
   public void setEdgeValueClass(Class<E> edgeValueClass) {
     this.edgeValueClass = edgeValueClass;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/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 3121fa8..32d08f8 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
@@ -39,7 +39,6 @@ import org.apache.giraph.factories.ValueFactories;
 import org.apache.giraph.factories.VertexIdFactory;
 import org.apache.giraph.factories.VertexValueFactory;
 import org.apache.giraph.graph.Computation;
-import org.apache.giraph.graph.DefaultVertex;
 import org.apache.giraph.graph.Language;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
@@ -90,6 +89,7 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.util.Progressable;
 
+
 /**
  * The classes set here are immutable, the remaining configuration is mutable.
  * Classes are immutable and final to provide the best performance for
@@ -113,6 +113,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   /** Whether values (IVEMM) need Jython wrappers */
   private final PerGraphTypeBoolean valueNeedsWrappers;
 
+
   /**
    * Use unsafe serialization? Cached for fast access to instantiate the
    * extended data input/output classes
@@ -675,12 +676,12 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    * @return Instantiated vertex
    */
   public Vertex<I, V, E> createVertex() {
-    Vertex<I, V, E> vertex = new DefaultVertex<I, V, E>();
-    vertex.setConf(this);
-    return vertex;
+    Class vertexClass = classes.getVertexClass();
+    return (Vertex<I, V, E>) ReflectionUtils.newInstance(vertexClass, this);
   }
 
-  /**
+
+ /**
    * Get the user's subclassed vertex index class.
    *
    * @return User's vertex index class

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/giraph-core/src/main/java/org/apache/giraph/graph/ByteValueVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/ByteValueVertex.java b/giraph-core/src/main/java/org/apache/giraph/graph/ByteValueVertex.java
new file mode 100644
index 0000000..fdabc81
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/ByteValueVertex.java
@@ -0,0 +1,114 @@
+/*
+ * 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.giraph.edge.Edge;
+import org.apache.giraph.utils.UnsafeByteArrayInputStream;
+import org.apache.giraph.utils.UnsafeByteArrayOutputStream;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+
+/**
+ * Special version of vertex that holds the value in raw byte form to save
+ * memory.
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ */
+public class ByteValueVertex<I extends WritableComparable,
+        V extends Writable, E extends Writable>
+        extends DefaultVertex<I, V, E> {
+
+  /** Vertex value stored as raw bytes */
+  private byte[] valueBytes;
+  /** Value as an cached object that is only valid during the vertex update */
+  private V cachedValue = null;
+
+  @Override
+  public V getValue() {
+    if (cachedValue != null) {
+      return cachedValue; // Return always same instance
+    }
+    DataInput dis = new UnsafeByteArrayInputStream(valueBytes);
+    cachedValue = getConf().createVertexValue();
+    try {
+      cachedValue.readFields(dis);
+    } catch (IOException ioe) {
+      throw new RuntimeException("Could not deserialize vertex value", ioe);
+    }
+    // Forget the serialized data, because we have cached the object
+    valueBytes = null;
+    return cachedValue;
+  }
+
+  /**
+   * Serializes the value to bytes, stored in field valueBytes
+   * @param value new vertex value
+   */
+  private void setSerializedValue(V value) {
+    UnsafeByteArrayOutputStream bos = new UnsafeByteArrayOutputStream();
+    try {
+      value.write(bos);
+      bos.close();
+    } catch (IOException ioe) {
+      throw new RuntimeException("Could not serialize vertex value", ioe);
+    }
+    this.valueBytes = bos.toByteArray();
+    cachedValue = null;
+  }
+
+  @Override
+  public void setValue(V value) {
+    if (cachedValue != null) {
+      cachedValue = value;
+    } else {
+      setSerializedValue(value);
+    }
+  }
+
+  @Override
+  public void initialize(I id, V value, Iterable<Edge<I, E>> edges) {
+    // Set the parent's value to null, and instead use our own setter
+    super.initialize(id, null, edges);
+    setValue(value);
+  }
+
+  @Override
+  public void initialize(I id, V value) {
+    super.initialize(id, null);
+    setValue(value);
+  }
+
+  @Override
+  public void unwrapMutableEdges() {
+    // This method is called always after compute(vertex), so
+    // we use this to commit the vertex value.
+    if (cachedValue != null) {
+      // This means the value has been requested from vertex
+      // and possible mutated -- so we need to update the byte array
+      setSerializedValue(cachedValue);
+      cachedValue = null;  // Uncache the value
+    }
+    super.unwrapMutableEdges();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index da4b98e..917f881 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -433,6 +433,7 @@ public final class ConfigurationUtils {
         }
       }
     }
+
     // YARN-ONLY OPTIONS
     if (cmd.hasOption("yj")) {
       conf.setYarnLibJars(cmd.getOptionValue("yj"));

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/giraph-core/src/test/java/org/apache/giraph/graph/TestByteValueVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/graph/TestByteValueVertex.java b/giraph-core/src/test/java/org/apache/giraph/graph/TestByteValueVertex.java
new file mode 100644
index 0000000..740e944
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/graph/TestByteValueVertex.java
@@ -0,0 +1,85 @@
+/*
+ * 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.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.ArrayListEdges;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests whether ByteValueVertex works -- same test as for DefaultVertex
+ * but with different factory method for vertices.
+ */
+public class TestByteValueVertex extends TestVertexAndEdges {
+
+    protected Vertex<LongWritable, FloatWritable, DoubleWritable>
+    instantiateVertex(Class<? extends OutEdges> edgesClass) {
+      GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
+      giraphConfiguration.setComputationClass(TestComputation.class);
+      giraphConfiguration.setOutEdgesClass(edgesClass);
+      giraphConfiguration.setVertexClass(ByteValueVertex.class);
+
+      ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
+                new ImmutableClassesGiraphConfiguration(giraphConfiguration);
+      Vertex bv = immutableClassesGiraphConfiguration.createVertex();
+      assertTrue(bv instanceof ByteValueVertex);
+      return bv;
+    }
+
+    @Test
+    public void testCachedValue() {
+      ByteValueVertex<LongWritable, FloatWritable, DoubleWritable> byteValueVertex =
+        (ByteValueVertex<LongWritable, FloatWritable, DoubleWritable>)
+          instantiateVertex(ArrayListEdges.class);
+
+      FloatWritable origValue = new FloatWritable(492.2f);
+      byteValueVertex.setValue(origValue);
+
+      // Check value is correct
+      assertEquals(492.2f, byteValueVertex.getValue().get(), 0.0f);
+
+      // Change value and see it is reflected correctly
+      FloatWritable gotValue = byteValueVertex.getValue();
+      gotValue.set(33.3f);
+      assertEquals(33.3f, byteValueVertex.getValue().get(), 0.0f);
+
+      // Change the object and set that the cached value also changes
+      FloatWritable newValue = new FloatWritable(99.9f);
+      byteValueVertex.setValue(newValue);
+      assertEquals(99.9f, byteValueVertex.getValue().get(), 0.0f);
+
+      // Reference should be now newValue
+      assertTrue(newValue == byteValueVertex.getValue());
+
+      // Commit the changes... (called after vertex update)
+      byteValueVertex.unwrapMutableEdges();
+
+      // Now the value reference should be new
+      assertFalse(newValue == byteValueVertex.getValue());
+
+      // But value data should be correct
+      assertEquals(99.9f, byteValueVertex.getValue().get(), 0.0f);
+    }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/0a901771/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
index 86d75a3..01c2613 100644
--- a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
@@ -161,7 +161,7 @@ public class TestVertexAndEdges {
     edgesClasses.add(LongDoubleHashMapEdges.class);
   }
 
-  private Vertex<LongWritable, FloatWritable, DoubleWritable>
+  protected Vertex<LongWritable, FloatWritable, DoubleWritable>
   instantiateVertex(Class<? extends OutEdges> edgesClass) {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
     giraphConfiguration.setComputationClass(TestComputation.class);
@@ -359,7 +359,7 @@ public class TestVertexAndEdges {
     }
   }
 
-  private Vertex<LongWritable, FloatWritable, DoubleWritable>
+  protected Vertex<LongWritable, FloatWritable, DoubleWritable>
   buildVertex(Class<? extends OutEdges> edgesClass) {
     Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         instantiateVertex(edgesClass);


[05/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
fix build error (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/b218d72c
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/b218d72c
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/b218d72c

Branch: refs/heads/release-1.1
Commit: b218d72cedc52467e691c6002e596e482d8583e4
Parents: 64428a9
Author: Pavan Kumar <pa...@fb.com>
Authored: Wed Jun 18 19:16:32 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Wed Jun 18 19:16:32 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/b218d72c/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
index b40ac00..dc7a5b4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/AbstractEdgeStore.java
@@ -224,7 +224,6 @@ public abstract class AbstractEdgeStore<I extends WritableComparable,
                 I vertexId = getVertexId(entry, representativeVertexId);
                 OutEdges<I, E> outEdges = convertInputToComputeEdges(
                   getPartitionEdges(entry));
-                iterator.remove();
                 Vertex<I, V, E> vertex = partition.getVertex(vertexId);
                 // If the source vertex doesn't exist, create it. Otherwise,
                 // just set the edges.
@@ -250,6 +249,7 @@ public abstract class AbstractEdgeStore<I extends WritableComparable,
                   // require us to put back the vertex after modifying it.
                   partition.saveVertex(vertex);
                 }
+                iterator.remove();
               }
               // Some PartitionStore implementations
               // (e.g. DiskBackedPartitionStore) require us to put back the


[42/47] GIRAPH-946. Upgrade to Gora 0.5 (Renato Javier Marroquín Mogrovejo via rvs)

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertex.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertex.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertex.java
index efd7b95..81b7c9a 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertex.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertex.java
@@ -15,67 +15,59 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.giraph.io.gora.generated;
 
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.util.Utf8;
-import org.apache.gora.persistency.StateManager;
-import org.apache.gora.persistency.StatefulHashMap;
-import org.apache.gora.persistency.impl.PersistentBase;
-import org.apache.gora.persistency.impl.StateManagerImpl;
-
 /**
- * Example class for defining a Giraph-Vertex.
+ * Example class for defining a Giraph-vertex.
  */
 @SuppressWarnings("all")
-public class GVertex extends PersistentBase {
-  /**
-   * Schema used for the class.
-   */
-  public static final Schema OBJ_SCHEMA = Schema.parse(
-      "{\"type\":\"record\",\"name\":\"Vertex\"," +
-      "\"namespace\":\"org.apache.giraph.gora.generated\"," +
-      "\"fields\":[{\"name\":\"vertexId\",\"type\":\"string\"}," +
-      "{\"name\":\"value\",\"type\":\"float\"},{\"name\":\"edges\"," +
-      "\"type\":{\"type\":\"map\",\"values\":\"string\"}}]}");
+public class GVertex extends org.apache.gora.persistency.impl.PersistentBase
+    implements org.apache.avro.specific.SpecificRecord,
+    org.apache.gora.persistency.Persistent {
 
   /**
-   * Field enum
+   * Schema used for the class.
    */
+  public static final org.apache.avro.Schema SCHEMAS =
+      new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\"," +
+            "\"name\":\"GVertex\"," +
+            "\"namespace\":\"org.apache.giraph.io.gora.generated\"," +
+            "\"fields\":[{\"name\":\"vertexId\",\"type\":\"string\"}," +
+            "{\"name\":\"vertexValue\",\"type\":\"float\"}," +
+            "{\"name\":\"edges\",\"type\":" +
+            "{\"type\":\"map\",\"values\":\"string\"}}]}");
+
+  /** Enum containing all data bean's fields. */
   public static enum Field {
     /**
-     * VertexId
+     * Vertex id.
      */
     VERTEX_ID(0, "vertexId"),
 
     /**
-     * Field value
+     * Vertex value.
      */
-    VALUE(1, "value"),
+    VERTEX_VALUE(1, "vertexValue"),
 
     /**
-     * Edges
+     * Vertex edges.
      */
     EDGES(2, "edges");
 
     /**
-     * Field index
+     * Field's index.
      */
     private int index;
 
     /**
-     * Field name
+     * Field's name.
      */
     private String name;
 
     /**
-     * Field constructor
-     * @param index of attribute
-     * @param name of attribute
+     * Field's constructor
+     * @param index field's index.
+     * @param name field's name.
      */
     Field(int index, String name) {
       this.index = index;
@@ -83,24 +75,24 @@ public class GVertex extends PersistentBase {
     }
 
     /**
-     * Gets index
-     * @return int of attribute.
+     * Gets field's index.
+     * @return int field's index.
      */
     public int getIndex() {
       return index;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's name.
+     * @return String field's name.
      */
     public String getName() {
       return name;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's attributes to string.
+     * @return String field's attributes to string.
      */
     public String toString() {
       return name;
@@ -111,170 +103,563 @@ public class GVertex extends PersistentBase {
    * Array containing all fields/
    */
   private static final String[] ALL_FIELDS = {
-    "vertexId", "value", "edges"
-  };
-
-  static {
-    PersistentBase.registerFields(GVertex.class, ALL_FIELDS);
-  }
+    "vertexId", "vertexValue", "edges", };
 
   /**
-   * Vertex Id
+   * Tombstone.
    */
-  private Utf8 vertexId;
+  private static final Tombstone TOMBSTONE = new Tombstone();
 
   /**
-   * Value
+   * vertexId.
    */
-  private float value;
+  private java.lang.CharSequence vertexId;
 
   /**
-   * Edges
+   * vertexValue.
    */
-  private Map<Utf8, Utf8> edges;
+  private float vertexValue;
 
   /**
-   * Default constructor
+   * edges.
    */
-  public GVertex() {
-    this(new StateManagerImpl());
-  }
+  private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> edges;
 
   /**
-   * Constructor
-   * @param stateManager from which the object will be created.
+   * Gets the total field count.
+   * @return int field count
    */
-  public GVertex(StateManager stateManager) {
-    super(stateManager);
-    edges = new StatefulHashMap<Utf8, Utf8>();
+  public int getFieldsCount() {
+    return GVertex.ALL_FIELDS.length;
   }
 
   /**
-   * Creates a new instance
-   * @param stateManager from which the object will be created.
-   * @return GVertex created
+   * Gets the schema
+   * @return Schema
    */
-  public GVertex newInstance(StateManager stateManager) {
-    return new GVertex(stateManager);
-  }
-
-  /**
-   * Gets the object schema
-   * @return Schema of the object.
-   */
-  public Schema getSchema() {
-    return OBJ_SCHEMA;
+  public org.apache.avro.Schema getSchema() {
+    return SCHEMAS;
   }
 
   /**
    * Gets field
-   * @param fieldIndex index of field to be used.
+   * @param field index field.
    * @return Object from an index.
    */
-  public Object get(int fieldIndex) {
-    switch (fieldIndex) {
+  public java.lang.Object get(int field) {
+    switch (field) {
     case 0:
       return vertexId;
     case 1:
-      return value;
+      return vertexValue;
     case 2:
       return edges;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
    * Puts a value into a field.
-   * @param fieldIndex index of field used.
-   * @param fieldValue value of field used.
+   * @param field index of field used.
+   * @param value value of field used.
    */
   @SuppressWarnings(value = "unchecked")
-  public void put(int fieldIndex, Object fieldValue) {
-    if (isFieldEqual(fieldIndex, fieldValue)) {
-      return;
-    }
-    getStateManager().setDirty(this, fieldIndex);
-    switch (fieldIndex) {
+  public void put(int field, java.lang.Object value) {
+    switch (field) {
     case 0:
-      vertexId = (Utf8) fieldValue; break;
+      vertexId = (java.lang.CharSequence) value;
+      break;
     case 1:
-      value = (Float) fieldValue; break;
+      vertexValue = (java.lang.Float) value;
+      break;
     case 2:
-      edges = (Map<Utf8, Utf8>) fieldValue; break;
+      edges = (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>)
+        ((value instanceof org.apache.gora.persistency.Dirtyable) ? value :
+        new org.apache.gora.persistency.impl.DirtyMapWrapper((java.util.Map)
+            value));
+      break;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
-   * Gets vertexId
-   * @return Utf8 vertexId
+   * Gets the value of the 'vertexId' field.
+   * @return CharSequence
+   */
+  public java.lang.CharSequence getVertexId() {
+    return vertexId;
+  }
+
+  /**
+   * Sets the value of the 'vertexId' field.
+   * @param value the value to set.
+   */
+  public void setVertexId(java.lang.CharSequence value) {
+    this.vertexId = value;
+    setDirty(0);
+  }
+
+  /**
+   * Checks the dirty status of the 'vertexId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
+   */
+  public boolean isVertexIdDirty(java.lang.CharSequence value) {
+    return isDirty(0);
+  }
+
+  /**
+   * Gets the value of the 'vertexValue' field.
+   * @return Float
+   */
+  public java.lang.Float getVertexValue() {
+    return vertexValue;
+  }
+
+  /**
+   * Sets the value of the 'vertexValue' field.
+   * @param value the value to set.
    */
-  public Utf8 getVertexId() {
-    return (Utf8) get(0);
+  public void setVertexValue(java.lang.Float value) {
+    this.vertexValue = value;
+    setDirty(1);
   }
 
   /**
-   * Sets vertexId
-   * @param value vertexId
+   * Checks the dirty status of the 'vertexValue' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public void setVertexId(Utf8 value) {
-    put(0, value);
+  public boolean isVertexValueDirty(java.lang.Float value) {
+    return isDirty(1);
   }
 
   /**
-   * Gets value
-   * @return String of value.
+   * Gets the value of the 'edges' field.
+   * @return java.util.Map
    */
-  public float getValue() {
-    return (Float) get(1);
+  public java.util.Map<java.lang.CharSequence, java.lang.CharSequence>
+  getEdges() {
+    return edges;
   }
 
   /**
-   * Sets value
-   * @param value .
+   * Sets the value of the 'edges' field.
+   * @param value the value to set.
    */
-  public void setValue(float value) {
-    put(1, value);
+  public void setEdges(
+      java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+    this.edges =
+      (value instanceof org.apache.gora.persistency.Dirtyable) ? value :
+      new org.apache.gora.persistency.impl.DirtyMapWrapper(value);
+    setDirty(2);
   }
 
   /**
-   * Get edges.
-   * @return Map of edges.
+   * Checks the dirty status of the 'edges' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public Map<Utf8, Utf8> getEdges() {
-    return (Map<Utf8, Utf8>) get(2);
+  public boolean isEdgesDirty(
+      java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+    return isDirty(2);
   }
 
   /**
-   * Gets value from edge.
-   * @param key Edge key.
-   * @return Utf8 containing the value of edge.
+   * Creates a new GVertex RecordBuilder.
+   * @return GVertex.Builder
    */
-  public Utf8 getFromEdges(Utf8 key) {
-    if (edges == null) { return null; }
-    return edges.get(key);
+  public static org.apache.giraph.io.gora.generated.GVertex.Builder
+  newBuilder() {
+    return new org.apache.giraph.io.gora.generated.GVertex.Builder();
   }
 
   /**
-   * Puts a new edge.
-   * @param key of new edge.
-   * @param value of new edge.
+   * Creates a new GVertex RecordBuilder by copying an existing Builder.
+   * @param other GVertex.Builder
+   * @return GVertex.Builder
    */
-  public void putToEdges(Utf8 key, Utf8 value) {
-    getStateManager().setDirty(this, 2);
-    edges.put(key, value);
+  public static org.apache.giraph.io.gora.generated.GVertex.Builder newBuilder(
+      org.apache.giraph.io.gora.generated.GVertex.Builder other) {
+    return new org.apache.giraph.io.gora.generated.GVertex.Builder(other);
   }
 
   /**
-   * Remove from edges
-   * @param key of edge to be deleted.
-   * @return Utf8 containing value of deleted key.
+   * Creates a new GVertex RecordBuilder by copying
+   * an existing GVertex instance
+   * @param other GVertex
+   * @return GVertex.Builder
    */
-  public Utf8 removeFromEdges(Utf8 key) {
-    if (edges == null) { return null; }
-    getStateManager().setDirty(this, 2);
-    return edges.remove(key);
+  public static org.apache.giraph.io.gora.generated.GVertex.Builder newBuilder(
+      org.apache.giraph.io.gora.generated.GVertex other) {
+    return new org.apache.giraph.io.gora.generated.GVertex.Builder(other);
+  }
+
+  /**
+   * Makes a deep copy from a bytebuffer.
+   * @param input ByteBuffer
+   * @return ByteBuffer
+   */
+  private static java.nio.ByteBuffer deepCopyToReadOnlyBuffer(
+      java.nio.ByteBuffer input) {
+    java.nio.ByteBuffer copy = java.nio.ByteBuffer.allocate(input.capacity());
+    int position = input.position();
+    input.reset();
+    int mark = input.position();
+    int limit = input.limit();
+    input.rewind();
+    input.limit(input.capacity());
+    copy.put(input);
+    input.rewind();
+    copy.rewind();
+    input.position(mark);
+    input.mark();
+    copy.position(mark);
+    copy.mark();
+    input.position(position);
+    copy.position(position);
+    input.limit(limit);
+    copy.limit(limit);
+    return copy.asReadOnlyBuffer();
+  }
+
+  /**
+   * RecordBuilder for GVertex instances.
+   */
+  public static class Builder extends
+      org.apache.avro.specific.SpecificRecordBuilderBase<GVertex> implements
+      org.apache.avro.data.RecordBuilder<GVertex> {
+
+    /**
+     * vertexId
+     */
+    private java.lang.CharSequence vertexId;
+
+    /**
+     * vertexValue
+     */
+    private float vertexValue;
+
+    /**
+     * edges
+     */
+    private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> edges;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.giraph.io.gora.generated.GVertex.SCHEMAS);
+    }
+
+    /**
+     * Creates a Builder by copying an existing Builder.
+     * @param other GVertex
+     */
+    private Builder(org.apache.giraph.io.gora.generated.GVertex.Builder other) {
+      super(other);
+    }
+
+    /**
+     * Creates a Builder by copying an existing GVertex instance.
+     * @param other GVertex
+     */
+    // CHECKSTYLE: stop Indentation
+    private Builder(org.apache.giraph.io.gora.generated.GVertex other) {
+      super(org.apache.giraph.io.gora.generated.GVertex.SCHEMAS);
+      if (isValidValue(fields()[0], other.vertexId)) {
+        this.vertexId = (java.lang.CharSequence) data().deepCopy(
+            fields()[0].schema(), other.vertexId);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.vertexValue)) {
+        this.vertexValue = (java.lang.Float) data().deepCopy(
+            fields()[1].schema(), other.vertexValue);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.edges)) {
+        this.edges =
+            (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>)
+            data().deepCopy(fields()[2].schema(), other.edges);
+        fieldSetFlags()[2] = true;
+      }
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexId' field.
+     * @return CharSsequence
+     */
+    public java.lang.CharSequence getVertexId() {
+      return vertexId;
+    }
+
+    /**
+     * Sets the value of the 'vertexId' field.
+     * @param value CharSequence
+     * @return GVertex.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertex.Builder setVertexId(
+        java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.vertexId = value;
+      fieldSetFlags()[0] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexId' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexId() {
+      return fieldSetFlags()[0];
+    }
+
+    /**
+     * Clears the value of the 'vertexId' field
+     * @return GVertex.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertex.Builder clearVertexId() {
+      vertexId = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+   // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexValue' field
+     * @return Float
+     */
+    public java.lang.Float getVertexValue() {
+      return vertexValue;
+    }
+
+    /**
+     * Sets the value of the 'vertexValue' field.
+     * @param value float
+     * @return GVertex.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertex.Builder setVertexValue(
+        float value) {
+      validate(fields()[1], value);
+      this.vertexValue = value;
+      fieldSetFlags()[1] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexValue' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexValue() {
+      return fieldSetFlags()[1];
+    }
+
+    /**
+     * Clears the value of the 'vertexValue' field.
+     * @return GVertex.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertex.Builder
+    clearVertexValue() {
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'edges' field.
+     * @return java.util.Map
+     */
+    public java.util.Map<java.lang.CharSequence, java.lang.CharSequence>
+    getEdges() {
+      return edges;
+    }
+
+    /**
+     * Sets the value of the 'edges' field.
+     * @param value java.util.Map
+     * @return GVertex.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertex.Builder setEdges(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      validate(fields()[2], value);
+      this.edges = value;
+      fieldSetFlags()[2] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'edges' field has been set.
+     * @return boolean
+     */
+    public boolean hasEdges() {
+      return fieldSetFlags()[2];
+    }
+
+    /**
+     * Clears the value of the 'edges' field.
+     * @return org.apache.giraph.io.gora.generated.GVertex.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertex.Builder clearEdges() {
+      edges = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    @Override
+    /**
+     * Builds a GVertex.
+     * @return GVertex
+     */
+    // CHECKSTYLE: stop IllegalCatch
+    public GVertex build() {
+      try {
+        GVertex record = new GVertex();
+        record.vertexId = fieldSetFlags()[0] ? this.vertexId :
+          (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.vertexValue = fieldSetFlags()[1] ? this.vertexValue :
+          (java.lang.Float) defaultValue(fields()[1]);
+        record.edges = fieldSetFlags()[2] ? this.edges :
+          (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>)
+          new org.apache.gora.persistency.impl.DirtyMapWrapper(
+            (java.util.Map) defaultValue(fields()[2]));
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+    // CHECKSTYLE: resume IllegalCatch
+  }
+
+  /**
+   * Gets tombstone
+   * @return GVertex.Tombstone
+   */
+  public GVertex.Tombstone getTombstone() {
+    return TOMBSTONE;
+  }
+
+  /**
+   * Gets a new instance
+   * @return GVertex.
+   */
+  public GVertex newInstance() {
+    return newBuilder().build();
+  }
+
+  /**
+   * Tombstone class.
+   */
+  public static final class Tombstone extends GVertex implements
+      org.apache.gora.persistency.Tombstone {
+
+    /**
+     * Default constructor.
+     */
+    private Tombstone() {
+    }
+
+    /**
+     * Gets the value of the 'vertexId' field.
+     * @return java.lang.CharSequence
+     */
+    public java.lang.CharSequence getVertexId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexId' field.
+     * @param value the value to set.
+     */
+    public void setVertexId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexId' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'vertexValue' field.
+     * @return Float
+     */
+    public java.lang.Float getVertexValue() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexValue' field.
+     * @param value the value to set.
+     */
+    public void setVertexValue(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexValue' field. A field is dirty if
+     * it represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexValueDirty(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'edges' field.
+     * @return java.util.Map
+     */
+    public java.util.Map<java.lang.CharSequence, java.lang.CharSequence>
+    getEdges() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'edges' field.
+     * @param value the value to set.
+     */
+    public void setEdges(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'edges' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isEdgesDirty(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertexResult.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertexResult.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertexResult.java
index 2c1952d..394fb8b 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertexResult.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GVertexResult.java
@@ -15,67 +15,60 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.giraph.io.gora.generated;
 
-import java.util.Map;
-
-import org.apache.avro.Schema;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.util.Utf8;
-import org.apache.gora.persistency.StateManager;
-import org.apache.gora.persistency.StatefulHashMap;
-import org.apache.gora.persistency.impl.PersistentBase;
-import org.apache.gora.persistency.impl.StateManagerImpl;
-
 /**
- * Example class for defining a Giraph-Vertex.
+ * Example class for defining a Giraph-vertex result.
  */
 @SuppressWarnings("all")
-public class GVertexResult extends PersistentBase {
-  /**
-   * Schema used for the class.
-   */
-  public static final Schema OBJ_SCHEMA = Schema.parse(
-      "{\"type\":\"record\",\"name\":\"Vertex\"," +
-      "\"namespace\":\"org.apache.giraph.gora.generated\"," +
-      "\"fields\":[{\"name\":\"vertexId\",\"type\":\"string\"}," +
-      "{\"name\":\"value\",\"type\":\"float\"},{\"name\":\"edges\"," +
-      "\"type\":{\"type\":\"map\",\"values\":\"string\"}}]}");
+public class GVertexResult extends
+    org.apache.gora.persistency.impl.PersistentBase implements
+    org.apache.avro.specific.SpecificRecord,
+    org.apache.gora.persistency.Persistent {
 
   /**
-   * Field enum
+   * Schema used for the class.
    */
+  public static final org.apache.avro.Schema SCHEMAS =
+      new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\"," +
+            "\"name\":\"GVertexResult\"," +
+            "\"namespace\":\"org.apache.giraph.io.gora.generated\"," +
+            "\"fields\":[{\"name\":\"vertexId\",\"type\":\"string\"}," +
+            "{\"name\":\"vertexValue\",\"type\":\"float\"}," +
+            "{\"name\":\"edges\",\"type\":" +
+            "{\"type\":\"map\",\"values\":\"string\"}}]}");
+
+  /** Enum containing all data bean's fields. */
   public static enum Field {
     /**
-     * VertexId
+     * Vertex id.
      */
     VERTEX_ID(0, "vertexId"),
 
     /**
-     * Field value
+     * Vertex value.
      */
-    VALUE(1, "value"),
+    VERTEX_VALUE(1, "vertexValue"),
 
     /**
-     * Edges
+     * Vertex edges.
      */
     EDGES(2, "edges");
 
     /**
-     * Field index
+     * Field's index.
      */
     private int index;
 
     /**
-     * Field name
+     * Field's name.
      */
     private String name;
 
     /**
-     * Field constructor
-     * @param index of attribute
-     * @param name of attribute
+     * Field's constructor
+     * @param index field's index.
+     * @param name field's name.
      */
     Field(int index, String name) {
       this.index = index;
@@ -83,24 +76,24 @@ public class GVertexResult extends PersistentBase {
     }
 
     /**
-     * Gets index
-     * @return int of attribute.
+     * Gets field's index.
+     * @return int field's index.
      */
     public int getIndex() {
       return index;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's name.
+     * @return String field's name.
      */
     public String getName() {
       return name;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's attributes to string.
+     * @return String field's attributes to string.
      */
     public String toString() {
       return name;
@@ -111,170 +104,565 @@ public class GVertexResult extends PersistentBase {
    * Array containing all fields/
    */
   private static final String[] ALL_FIELDS = {
-    "vertexId", "value", "edges"
-  };
-
-  static {
-    PersistentBase.registerFields(GVertexResult.class, ALL_FIELDS);
-  }
+    "vertexId", "vertexValue", "edges", };
 
   /**
-   * Vertex Id
+   * Tombstone.
    */
-  private Utf8 vertexId;
+  private static final Tombstone TOMBSTONE = new Tombstone();
 
   /**
-   * Value
+   * vertexId.
    */
-  private float value;
+  private java.lang.CharSequence vertexId;
 
   /**
-   * Edges
+   * vertexValue.
    */
-  private Map<Utf8, Utf8> edges;
+  private float vertexValue;
 
   /**
-   * Default constructor
+   * edges.
    */
-  public GVertexResult() {
-    this(new StateManagerImpl());
-  }
+  private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> edges;
 
   /**
-   * Constructor
-   * @param stateManager from which the object will be created.
+   * Gets the total field count.
+   * @return int field count
    */
-  public GVertexResult(StateManager stateManager) {
-    super(stateManager);
-    edges = new StatefulHashMap<Utf8, Utf8>();
+  public int getFieldsCount() {
+    return GVertexResult.ALL_FIELDS.length;
   }
 
   /**
-   * Creates a new instance
-   * @param stateManager from which the object will be created.
-   * @return GVertex created
+   * Gets the schema
+   * @return Schema
    */
-  public GVertexResult newInstance(StateManager stateManager) {
-    return new GVertexResult(stateManager);
-  }
-
-  /**
-   * Gets the object schema
-   * @return Schema of the object.
-   */
-  public Schema getSchema() {
-    return OBJ_SCHEMA;
+  public org.apache.avro.Schema getSchema() {
+    return SCHEMAS;
   }
 
   /**
    * Gets field
-   * @param fieldIndex index of field to be used.
+   * @param field index field.
    * @return Object from an index.
    */
-  public Object get(int fieldIndex) {
-    switch (fieldIndex) {
+  public java.lang.Object get(int field) {
+    switch (field) {
     case 0:
       return vertexId;
     case 1:
-      return value;
+      return vertexValue;
     case 2:
       return edges;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
    * Puts a value into a field.
-   * @param fieldIndex index of field used.
-   * @param fieldValue value of field used.
+   * @param field index of field used.
+   * @param value value of field used.
    */
   @SuppressWarnings(value = "unchecked")
-  public void put(int fieldIndex, Object fieldValue) {
-    if (isFieldEqual(fieldIndex, fieldValue)) {
-      return;
-    }
-    getStateManager().setDirty(this, fieldIndex);
-    switch (fieldIndex) {
+  public void put(int field, java.lang.Object value) {
+    switch (field) {
     case 0:
-      vertexId = (Utf8) fieldValue; break;
+      vertexId = (java.lang.CharSequence) value;
+      break;
     case 1:
-      value = (Float) fieldValue; break;
+      vertexValue = (java.lang.Float) value;
+      break;
     case 2:
-      edges = (Map<Utf8, Utf8>) fieldValue; break;
+      edges = (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>)
+        ((value instanceof org.apache.gora.persistency.Dirtyable) ? value :
+        new org.apache.gora.persistency.impl.DirtyMapWrapper((java.util.Map)
+            value));
+      break;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
-   * Gets vertexId
-   * @return Utf8 vertexId
+   * Gets the value of the 'vertexId' field.
+   * @return CharSequence
+   */
+  public java.lang.CharSequence getVertexId() {
+    return vertexId;
+  }
+
+  /**
+   * Sets the value of the 'vertexId' field.
+   * @param value the value to set.
+   */
+  public void setVertexId(java.lang.CharSequence value) {
+    this.vertexId = value;
+    setDirty(0);
+  }
+
+  /**
+   * Checks the dirty status of the 'vertexId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
+   */
+  public boolean isVertexIdDirty(java.lang.CharSequence value) {
+    return isDirty(0);
+  }
+
+  /**
+   * Gets the value of the 'vertexValue' field.
+   * @return Float
+   */
+  public java.lang.Float getVertexValue() {
+    return vertexValue;
+  }
+
+  /**
+   * Sets the value of the 'vertexValue' field.
+     * @param value the value to set.
    */
-  public Utf8 getVertexId() {
-    return (Utf8) get(0);
+  public void setVertexValue(java.lang.Float value) {
+    this.vertexValue = value;
+    setDirty(1);
   }
 
   /**
-   * Sets vertexId
-   * @param value vertexId
+   * Checks the dirty status of the 'vertexValue' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public void setVertexId(Utf8 value) {
-    put(0, value);
+  public boolean isVertexValueDirty(java.lang.Float value) {
+    return isDirty(1);
   }
 
   /**
-   * Gets value
-   * @return String of value.
+   * Gets the value of the 'edges' field.
+   * @return Edges
    */
-  public float getValue() {
-    return (Float) get(1);
+  public java.util.Map<java.lang.CharSequence, java.lang.CharSequence>
+  getEdges() {
+    return edges;
   }
 
   /**
-   * Sets value
-   * @param value .
+   * Sets the value of the 'edges' field.
+   * @param value the value to set.
    */
-  public void setValue(float value) {
-    put(1, value);
+  public void setEdges(
+      java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+    this.edges = (value instanceof org.apache.gora.persistency.Dirtyable) ?
+        value : new org.apache.gora.persistency.impl.DirtyMapWrapper(value);
+    setDirty(2);
   }
 
   /**
-   * Get edges.
-   * @return Map of edges.
+   * Checks the dirty status of the 'edges' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public Map<Utf8, Utf8> getEdges() {
-    return (Map<Utf8, Utf8>) get(2);
+  public boolean isEdgesDirty(
+      java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+    return isDirty(2);
   }
 
   /**
-   * Gets value from edge.
-   * @param key Edge key.
-   * @return Utf8 containing the value of edge.
+   * Creates a new GVertexResult RecordBuilder
+   * @return GVertexResult.Builder
    */
-  public Utf8 getFromEdges(Utf8 key) {
-    if (edges == null) { return null; }
-    return edges.get(key);
+  public static org.apache.giraph.io.gora.generated.GVertexResult.Builder
+  newBuilder() {
+    return new org.apache.giraph.io.gora.generated.GVertexResult.Builder();
   }
 
   /**
-   * Puts a new edge.
-   * @param key of new edge.
-   * @param value of new edge.
+   * Creates a new GVertexResult RecordBuilder by copying an existing Builder.
+   * @param other GVertexResult.Builder
+   * @return GVertexResult.Builder
    */
-  public void putToEdges(Utf8 key, Utf8 value) {
-    getStateManager().setDirty(this, 2);
-    edges.put(key, value);
+  public static org.apache.giraph.io.gora.generated.GVertexResult.Builder
+  newBuilder(org.apache.giraph.io.gora.generated.GVertexResult.Builder other) {
+    return new org.apache.giraph.io.gora.generated.GVertexResult.Builder(other);
   }
 
   /**
-   * Remove from edges
-   * @param key of edge to be deleted.
-   * @return Utf8 containing value of deleted key.
+   * Creates a new GVertexResult RecordBuilder by copying an existing
+   * GVertexResult instance
+   * @param other GVertexResult
+   * @return GVertexResult.Builder
    */
-  public Utf8 removeFromEdges(Utf8 key) {
-    if (edges == null) { return null; }
-    getStateManager().setDirty(this, 2);
-    return edges.remove(key);
+  public static org.apache.giraph.io.gora.generated.GVertexResult.Builder
+  newBuilder(org.apache.giraph.io.gora.generated.GVertexResult other) {
+    return new org.apache.giraph.io.gora.generated.GVertexResult.Builder(other);
+  }
+
+  /**
+   * Makes a deep copy from a bytebuffer.
+   * @param input ByteBuffer
+   * @return ByteBuffer
+   */
+  private static java.nio.ByteBuffer deepCopyToReadOnlyBuffer(
+      java.nio.ByteBuffer input) {
+    java.nio.ByteBuffer copy = java.nio.ByteBuffer.allocate(input.capacity());
+    int position = input.position();
+    input.reset();
+    int mark = input.position();
+    int limit = input.limit();
+    input.rewind();
+    input.limit(input.capacity());
+    copy.put(input);
+    input.rewind();
+    copy.rewind();
+    input.position(mark);
+    input.mark();
+    copy.position(mark);
+    copy.mark();
+    input.position(position);
+    copy.position(position);
+    input.limit(limit);
+    copy.limit(limit);
+    return copy.asReadOnlyBuffer();
+  }
+
+  /**
+   * RecordBuilder for GVertexResult instances.
+   */
+  public static class Builder extends
+      org.apache.avro.specific.SpecificRecordBuilderBase<GVertexResult>
+      implements org.apache.avro.data.RecordBuilder<GVertexResult> {
+
+    /**
+     * vertexId
+     */
+    private java.lang.CharSequence vertexId;
+
+    /**
+     * vertexValue
+     */
+    private float vertexValue;
+
+    /**
+     * edges
+     */
+    private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> edges;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.giraph.io.gora.generated.GVertexResult.SCHEMAS);
+    }
+
+    /**
+     * Creates a Builder by copying an existing Builder.
+     * @param other GVertexResult.Builder
+     */
+    private Builder(
+        org.apache.giraph.io.gora.generated.GVertexResult.Builder other) {
+      super(other);
+    }
+
+    /**
+     * Creates a Builder by copying an existing GVertexResult instance.
+     * @param other GVertexResult
+     */
+    // CHECKSTYLE: stop Indentation
+    private Builder(org.apache.giraph.io.gora.generated.GVertexResult other) {
+      super(org.apache.giraph.io.gora.generated.GVertexResult.SCHEMAS);
+      if (isValidValue(fields()[0], other.vertexId)) {
+        this.vertexId = (java.lang.CharSequence) data().deepCopy(
+            fields()[0].schema(), other.vertexId);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.vertexValue)) {
+        this.vertexValue = (java.lang.Float) data().deepCopy(
+            fields()[1].schema(), other.vertexValue);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.edges)) {
+        this.edges =
+            (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>)
+            data().deepCopy(fields()[2].schema(), other.edges);
+        fieldSetFlags()[2] = true;
+      }
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexId() {
+      return vertexId;
+    }
+
+    /**
+     * Sets the value of the 'vertexId' field.
+     * @param value CharSequence
+     * @return GVertexResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertexResult.Builder
+    setVertexId(java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.vertexId = value;
+      fieldSetFlags()[0] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexId' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexId() {
+      return fieldSetFlags()[0];
+    }
+
+    /**
+     * Clears the value of the 'vertexId' field
+     * @return GVertexResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertexResult.Builder
+    clearVertexId() {
+      vertexId = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexValue' field.
+     * @return Float
+     */
+    public java.lang.Float getVertexValue() {
+      return vertexValue;
+    }
+
+    /**
+     * Sets the value of the 'vertexValue' field.
+     * @param value float
+     * @return GVertexResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertexResult.Builder
+    setVertexValue(float value) {
+      validate(fields()[1], value);
+      this.vertexValue = value;
+      fieldSetFlags()[1] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexValue' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexValue() {
+      return fieldSetFlags()[1];
+    }
+
+    /**
+     * Clears the value of the 'vertexValue' field.
+     * @return GVertexResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertexResult.Builder
+    clearVertexValue() {
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'edges' field.
+     * @return java.util.Map
+     */
+    public java.util.Map<java.lang.CharSequence, java.lang.CharSequence>
+    getEdges() {
+      return edges;
+    }
+
+    /**
+     * Sets the value of the 'edges' field
+     * @param value java.util.Map
+     * @return GVertexResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertexResult.Builder setEdges(
+    java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      validate(fields()[2], value);
+      this.edges = value;
+      fieldSetFlags()[2] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'edges' field has been set.
+     * @return boolean
+     */
+    public boolean hasEdges() {
+      return fieldSetFlags()[2];
+    }
+
+    /**
+     * Clears the value of the 'edges' field.
+     * @return org.apache.giraph.io.gora.generated.GVertexResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GVertexResult.Builder
+    clearEdges() {
+      edges = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    @Override
+    /**
+     * Builds a GVertexResult.
+     * @return GVertexResult
+     */
+    // CHECKSTYLE: stop IllegalCatch
+    public GVertexResult build() {
+      try {
+        GVertexResult record = new GVertexResult();
+        record.vertexId = fieldSetFlags()[0] ? this.vertexId :
+          (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.vertexValue = fieldSetFlags()[1] ? this.vertexValue :
+          (java.lang.Float) defaultValue(fields()[1]);
+        record.edges = fieldSetFlags()[2] ? this.edges :
+          (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>)
+          new org.apache.gora.persistency.impl.DirtyMapWrapper(
+            (java.util.Map) defaultValue(fields()[2]));
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+    // CHECKSTYLE: resume IllegalCatch
+  }
+
+  /**
+   * Gets tombstone
+   * @return GVertex.Tombstone
+   */
+  public GVertexResult.Tombstone getTombstone() {
+    return TOMBSTONE;
+  }
+
+  /**
+   * Gets a new instance
+   * @return GVertexResult.
+   */
+  public GVertexResult newInstance() {
+    return newBuilder().build();
+  }
+
+  /**
+   * Tombstone class.
+   */
+  public static final class Tombstone extends GVertexResult implements
+      org.apache.gora.persistency.Tombstone {
+
+    /**
+     * Default constructor.
+     */
+    private Tombstone() {
+    }
+
+    /**
+     * Gets the value of the 'vertexId' field.
+     * @return java.lang.CharSequence
+     */
+    public java.lang.CharSequence getVertexId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexId' field.
+     * @param value the value to set.
+     */
+    public void setVertexId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexId' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'vertexValue' field.
+     * @return Float
+     */
+    public java.lang.Float getVertexValue() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexValue' field.
+     * @param value the value to set.
+     */
+    public void setVertexValue(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexValue' field. A field is dirty if
+     * it represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexValueDirty(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'edges' field.
+     * @return java.util.Map
+     */
+    public java.util.Map<java.lang.CharSequence, java.lang.CharSequence>
+    getEdges() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'edges' field.
+     * @param value the value to set.
+     */
+    public void setEdges(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'edges' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isEdgesDirty(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/ExtraGoraInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/ExtraGoraInputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/ExtraGoraInputFormat.java
index e11f910..b40a361 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/ExtraGoraInputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/ExtraGoraInputFormat.java
@@ -28,6 +28,7 @@ import org.apache.gora.persistency.Persistent;
 import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.PartitionQuery;
 import org.apache.gora.query.Query;
+import org.apache.gora.query.impl.PartitionQueryImpl;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.util.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -83,17 +84,21 @@ public class ExtraGoraInputFormat<K, T extends PersistentBase>
     return new GoraRecordReader<K, T>(partitionQuery, context);
   }
 
+  /**
+   * Gets splits.
+   * @param context for the job.
+   * @return List<InputSplit> splits found
+   */
   @Override
   public List<InputSplit> getSplits(JobContext context) throws IOException,
       InterruptedException {
     List<PartitionQuery<K, T>> queries =
         getDataStore().getPartitions(getQuery());
     List<InputSplit> splits = new ArrayList<InputSplit>(queries.size());
-
     for (PartitionQuery<K, T> partQuery : queries) {
+      ((PartitionQueryImpl) partQuery).setConf(context.getConfiguration());
       splits.add(new GoraInputSplit(context.getConfiguration(), partQuery));
     }
-
     return splits;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/GoraUtils.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/GoraUtils.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/GoraUtils.java
index c3fc268..932e2f3 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/GoraUtils.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/utils/GoraUtils.java
@@ -20,6 +20,7 @@ package org.apache.giraph.io.gora.utils;
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
+import org.apache.gora.query.impl.QueryBase;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.util.GoraException;
@@ -36,11 +37,6 @@ public class GoraUtils {
   private static Class<? extends DataStore> DATASTORECLASS;
 
   /**
-   * Attribute handling configuration for data stores.
-   */
-  private static Configuration CONF = new Configuration();
-
-  /**
    * The default constructor is set to be private by default so that the
    * class is not instantiated.
    */
@@ -49,6 +45,7 @@ public class GoraUtils {
   /**
    * Creates a generic data store using the data store class.
    * set using the class property
+   * @param conf Configuration
    * @param <K> key class
    * @param <T> value class
    * @param keyClass key class used
@@ -58,20 +55,22 @@ public class GoraUtils {
    */
   @SuppressWarnings("unchecked")
   public static <K, T extends Persistent> DataStore<K, T>
-  createDataStore(Class<K> keyClass, Class<T> persistentClass)
+  createDataStore(Configuration conf,
+      Class<K> keyClass, Class<T> persistentClass)
     throws GoraException {
     DataStoreFactory.createProps();
     DataStore<K, T> dataStore =
         DataStoreFactory.createDataStore((Class<? extends DataStore<K, T>>)
                                           DATASTORECLASS,
                                           keyClass, persistentClass,
-                                          getConf());
+                                          conf);
 
     return dataStore;
   }
 
   /**
    * Creates a specific data store specified by.
+   * @param conf Configuration
    * @param <K> key class
    * @param <T> value class
    * @param dataStoreClass  Defines the type of data store used.
@@ -81,10 +80,11 @@ public class GoraUtils {
    * @throws GoraException  if an error occurs.
    */
   public static <K, T extends Persistent> DataStore<K, T>
-  createSpecificDataStore(Class<? extends DataStore> dataStoreClass,
+  createSpecificDataStore(Configuration conf,
+      Class<? extends DataStore> dataStoreClass,
       Class<K> keyClass, Class<T> persistentClass) throws GoraException {
     DATASTORECLASS = dataStoreClass;
-    return createDataStore(keyClass, persistentClass);
+    return createDataStore(conf, keyClass, persistentClass);
   }
 
   /**
@@ -98,7 +98,7 @@ public class GoraUtils {
    */
   public static <K, T extends Persistent> Result<K, T>
   getRequest(DataStore<K, T> pDataStore, K pStartKey, K pEndKey) {
-    Query<K, T> query = getQuery(pDataStore, pStartKey, pEndKey);
+    QueryBase query = getQuery(pDataStore, pStartKey, pEndKey);
     return getRequest(pDataStore, query);
   }
 
@@ -137,9 +137,9 @@ public class GoraUtils {
    * @param <T> value class
    * @return range query object.
    */
-  public static <K, T extends Persistent> Query<K, T>
-  getQuery(DataStore<K, T> pDataStore, K pStartKey, K pEndKey) {
-    Query<K, T> query = pDataStore.newQuery();
+  public static <K, T extends Persistent> QueryBase
+  getQuery(DataStore pDataStore, K pStartKey, K pEndKey) {
+    QueryBase query = (QueryBase) pDataStore.newQuery();
     query.setStartKey(pStartKey);
     query.setEndKey(pEndKey);
     return query;
@@ -175,20 +175,4 @@ public class GoraUtils {
     query.setEndKey(null);
     return query;
   }
-
-  /**
-   * Gets the configuration object.
-   * @return the configuration object.
-   */
-  public static Configuration getConf() {
-    return CONF;
-  }
-
-  /**
-   * Sets the configuration object.
-   * @param conf to be set as the configuration object.
-   */
-  public static void setConf(Configuration conf) {
-    CONF = conf;
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeInputFormat.java b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeInputFormat.java
index ba71ce4..c339eb0 100644
--- a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeInputFormat.java
+++ b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeInputFormat.java
@@ -19,10 +19,8 @@ package org.apache.giraph.io.gora;
 
 import java.io.IOException;
 
-import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.io.gora.GoraEdgeInputFormat;
 import org.apache.giraph.io.gora.generated.GEdge;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -82,10 +80,10 @@ public class GoraTestEdgeInputFormat
   private static GEdge createEdge(String id, String vertexInId,
       String vertexOutId, String edgeLabel, float edgeWeight) {
     GEdge newEdge = new GEdge();
-    newEdge.setEdgeId(new Utf8(id));
-    newEdge.setVertexInId(new Utf8(vertexInId));
-    newEdge.setVertexOutId(new Utf8(vertexOutId));
-    newEdge.setLabel(new Utf8(edgeLabel));
+    newEdge.setEdgeId(id);
+    newEdge.setVertexInId(vertexInId);
+    newEdge.setVertexOutId(vertexOutId);
+    newEdge.setLabel(edgeLabel);
     newEdge.setEdgeWeight(edgeWeight);
     return newEdge;
   }
@@ -109,11 +107,11 @@ public class GoraTestEdgeInputFormat
       Edge<LongWritable, FloatWritable> edge = null;
       GEdge goraEdge = (GEdge) goraObject;
       Long dest;
-      Long value;
+      Float value;
       dest = Long.valueOf(goraEdge.getVertexOutId().toString());
       this.sourceId = new LongWritable();
       this.sourceId.set(Long.valueOf(goraEdge.getVertexInId().toString()));
-      value = (long) goraEdge.getEdgeWeight();
+      value = (float) goraEdge.getEdgeWeight();
       edge = EdgeFactory.create(new LongWritable(dest),
           new FloatWritable(value));
       return edge;

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeOutputFormat.java b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeOutputFormat.java
index 0254498..f2e88c6 100644
--- a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeOutputFormat.java
+++ b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestEdgeOutputFormat.java
@@ -19,11 +19,8 @@ package org.apache.giraph.io.gora;
 
 import java.io.IOException;
 
-import junit.framework.Assert;
-
 import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
-import org.apache.giraph.io.gora.GoraEdgeOutputFormat;
 import org.apache.giraph.io.gora.generated.GEdge;
 import org.apache.giraph.io.gora.generated.GEdgeResult;
 import org.apache.gora.persistency.Persistent;
@@ -31,6 +28,7 @@ import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.Assert;
 
 /**
  * Implementation of a specific writer for a generated data bean.
@@ -63,11 +61,11 @@ public class GoraTestEdgeOutputFormat
       GEdgeResult tmpGEdge = new GEdgeResult();
       Utf8 keyLabel = new Utf8(srcId.toString() + "-" +
       edge.getTargetVertexId().toString());
-      tmpGEdge.setEdgeId(keyLabel);
+      tmpGEdge.setEdgeId(keyLabel.toString());
       tmpGEdge.setEdgeWeight(edge.getValue().get());
-      tmpGEdge.setVertexInId(new Utf8(srcId.toString()));
-      tmpGEdge.setVertexOutId(new Utf8(edge.getTargetVertexId().toString()));
-      tmpGEdge.setLabel(keyLabel);
+      tmpGEdge.setVertexInId(srcId.toString());
+      tmpGEdge.setVertexOutId(edge.getTargetVertexId().toString());
+      tmpGEdge.setLabel(keyLabel.toString());
       getLogger().debug("GoraObject created: " + tmpGEdge.toString());
       return tmpGEdge;
     }
@@ -80,6 +78,7 @@ public class GoraTestEdgeOutputFormat
       return goraKey;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void writeEdge(LongWritable srcId, DoubleWritable srcValue,
         Edge<LongWritable, FloatWritable> edge)
@@ -105,13 +104,13 @@ public class GoraTestEdgeOutputFormat
      * @param edgeWeight Edge wight.
      * @return GEdge created.
      */
-    private GEdge createEdge(String id, String vertexInId,
+    private GEdgeResult createEdge(String id, String vertexInId,
         String vertexOutId, String edgeLabel, float edgeWeight) {
-      GEdge newEdge = new GEdge();
-      newEdge.setEdgeId(new Utf8(id));
-      newEdge.setVertexInId(new Utf8(vertexInId));
-      newEdge.setVertexOutId(new Utf8(vertexOutId));
-      newEdge.setLabel(new Utf8(edgeLabel));
+      GEdgeResult newEdge = new GEdgeResult();
+      newEdge.setEdgeId(id);
+      newEdge.setVertexInId(vertexInId);
+      newEdge.setVertexOutId(vertexOutId);
+      newEdge.setLabel(edgeLabel);
       newEdge.setEdgeWeight(edgeWeight);
       return newEdge;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexInputFormat.java b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexInputFormat.java
index 7de9346..d79a8fa 100644
--- a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexInputFormat.java
+++ b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexInputFormat.java
@@ -18,11 +18,9 @@
 package org.apache.giraph.io.gora;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
@@ -78,10 +76,10 @@ public class GoraTestVertexInputFormat
    */
   public static GVertex createVertex(String id, Map<String, String> edges) {
     GVertex newVrtx = new GVertex();
-    newVrtx.setVertexId(new Utf8(id));
+    newVrtx.setVertexId(id);
     if (edges != null) {
       for (String edgeId : edges.keySet())
-        newVrtx.putToEdges(new Utf8(edgeId), new Utf8(edges.get(edgeId)));
+        newVrtx.getEdges().put(edgeId, edges.get(edgeId));
     }
     return newVrtx;
   }
@@ -106,11 +104,11 @@ public class GoraTestVertexInputFormat
 
       LongWritable vrtxId = new LongWritable(
           Long.parseLong(tmpGVertex.getVertexId().toString()));
-      DoubleWritable vrtxValue = new DoubleWritable(tmpGVertex.getValue());
+      DoubleWritable vrtxValue = new DoubleWritable(tmpGVertex.getVertexValue());
       vertex.initialize(vrtxId, vrtxValue);
       if (tmpGVertex.getEdges() != null && !tmpGVertex.getEdges().isEmpty()) {
-        Set<Utf8> keyIt = tmpGVertex.getEdges().keySet();
-        for (Utf8 key : keyIt) {
+        Set<CharSequence> keyIt = tmpGVertex.getEdges().keySet();
+        for (CharSequence key : keyIt) {
           String keyVal = key.toString();
           String valVal = tmpGVertex.getEdges().get(key).toString();
           Edge<LongWritable, FloatWritable> edge;

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexOutputFormat.java b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexOutputFormat.java
index 5170d03..1cb1f14 100644
--- a/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexOutputFormat.java
+++ b/giraph-gora/src/test/java/org/apache/giraph/io/gora/GoraTestVertexOutputFormat.java
@@ -21,9 +21,6 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map;
 
-import junit.framework.Assert;
-
-import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexWriter;
@@ -34,6 +31,7 @@ import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.Assert;
 
 /**
  * Implementation of a specific reader for a generated data bean.
@@ -64,20 +62,21 @@ public class GoraTestVertexOutputFormat
     protected Persistent getGoraVertex(
         Vertex<LongWritable, DoubleWritable, FloatWritable> vertex) {
       GVertexResult tmpGVertex = new GVertexResult();
-      tmpGVertex.setVertexId(new Utf8(vertex.getId().toString()));
-      tmpGVertex.setValue(Float.parseFloat(vertex.getValue().toString()));
+      tmpGVertex.setVertexId(vertex.getId().toString());
+      tmpGVertex.setVertexValue(Float.parseFloat(vertex.getValue().toString()));
       Iterator<Edge<LongWritable, FloatWritable>> it =
           vertex.getEdges().iterator();
       while (it.hasNext()) {
         Edge<LongWritable, FloatWritable> edge = it.next();
-        tmpGVertex.putToEdges(
-            new Utf8(edge.getTargetVertexId().toString()),
-            new Utf8(edge.getValue().toString()));
+        tmpGVertex.getEdges().put(
+            edge.getTargetVertexId().toString(),
+            edge.getValue().toString());
       }
       getLogger().debug("GoraObject created: " + tmpGVertex.toString());
       return tmpGVertex;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void writeVertex(
       Vertex<LongWritable, DoubleWritable, FloatWritable> vertex)
@@ -94,12 +93,12 @@ public class GoraTestVertexOutputFormat
      * @param edges Set of edges.
      * @return GVertex created.
      */
-    public GVertex createVertex(String id, Map<String, String> edges) {
-      GVertex newVrtx = new GVertex();
-      newVrtx.setVertexId(new Utf8(id));
+    public GVertexResult createVertex(String id, Map<String, String> edges) {
+      GVertexResult newVrtx = new GVertexResult();
+      newVrtx.setVertexId(id);
       if (edges != null) {
         for (String edgeId : edges.keySet())
-          newVrtx.putToEdges(new Utf8(edgeId), new Utf8(edges.get(edgeId)));
+          newVrtx.getEdges().put(edgeId, edges.get(edgeId));
       }
       return newVrtx;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeInputFormat.java b/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeInputFormat.java
index a01fbd3..cf2df34 100644
--- a/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeInputFormat.java
+++ b/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeInputFormat.java
@@ -60,8 +60,8 @@ public class TestGoraEdgeInputFormat {
     GIRAPH_GORA_START_KEY.set(conf,"1");
     GIRAPH_GORA_END_KEY.set(conf,"3");
     conf.set("io.serializations",
-        "org.apache.hadoop.io.serializer.WritableSerialization," +
-        "org.apache.hadoop.io.serializer.JavaSerialization");
+        "org.apache.hadoop.io.serializer.JavaSerialization," +
+        "org.apache.hadoop.io.serializer.WritableSerialization");
     conf.setComputationClass(EmptyComputation.class);
     conf.setEdgeInputFormatClass(GoraGEdgeEdgeInputFormat.class);
     results = InternalVertexRunner.run(conf, new String[0], new String[0]);

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeOutputFormat.java b/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeOutputFormat.java
index c9ac38a..7de6d01 100644
--- a/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeOutputFormat.java
+++ b/giraph-gora/src/test/java/org/apache/giraph/io/gora/TestGoraEdgeOutputFormat.java
@@ -68,7 +68,7 @@ public class TestGoraEdgeOutputFormat {
     set(conf, "org.apache.gora.memory.store.MemStore");
     GIRAPH_GORA_OUTPUT_KEY_CLASS.set(conf, "java.lang.String");
     GIRAPH_GORA_OUTPUT_PERSISTENT_CLASS.
-    set(conf,"org.apache.giraph.io.gora.generated.GEdge");
+    set(conf,"org.apache.giraph.io.gora.generated.GEdgeResult");
     conf.setEdgeOutputFormatClass(GoraTestEdgeOutputFormat.class);
     results = InternalVertexRunner.run(conf, new String[0], new String[0]);
     Assert.assertNotNull(results);

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e654ff3..6804de9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -273,10 +273,10 @@ under the License.
     <giraph.maven.dependency.plugin.skip>false</giraph.maven.dependency.plugin.skip>
     <giraph.maven.duplicate.finder.skip>false</giraph.maven.duplicate.finder.skip>
 
-    <dep.avro.version>1.3.3</dep.avro.version>
+    <dep.avro.version>1.7.6</dep.avro.version>
     <dep.accumulo.version>1.4.0</dep.accumulo.version>
     <dep.asm.version>3.2</dep.asm.version>
-    <dep.gora.version>0.3</dep.gora.version>
+    <dep.gora.version>0.5</dep.gora.version>
     <dep.airline.version>0.5</dep.airline.version>
     <dep.base64.version>2.3.8</dep.base64.version>
     <dep.cli-parser.version>1.1</dep.cli-parser.version>


[18/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-932 - Adding .arcconfig to GIRAPH for Arcanist support

Summary: As above

Test Plan: This diff.

Reviewers: maja.kabiljo

Reviewed By: maja.kabiljo

Subscribers: maja.kabiljo

Differential Revision: https://reviews.facebook.net/D20439


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/93035222
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/93035222
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/93035222

Branch: refs/heads/release-1.1
Commit: 930352220d5b201007e886ad8f92f9796d828a6a
Parents: 969a488
Author: Avery Ching <ac...@fb.com>
Authored: Tue Jul 22 17:20:07 2014 -0700
Committer: Avery Ching <ac...@fb.com>
Committed: Tue Jul 22 17:34:06 2014 -0700

----------------------------------------------------------------------
 .arcconfig | 7 +++++++
 CHANGELOG  | 2 ++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/93035222/.arcconfig
----------------------------------------------------------------------
diff --git a/.arcconfig b/.arcconfig
new file mode 100644
index 0000000..c598de7
--- /dev/null
+++ b/.arcconfig
@@ -0,0 +1,7 @@
+{
+  "project_id" : "giraph",
+  "conduit_uri" : "https://reviews.facebook.net/",
+  "copyright_holder" : "Apache Software Foundation",
+  "jira_project" : "GIRAPH",
+  "jira_api_url" : "https://issues.apache.org/jira/browse/GIRAPH"
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/93035222/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index d6d998c..54ed3a3 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-932: Adding .arcconfig to GIRAPH for Arcanist support (aching)
+
   GIRAPH-927: Decouple netty server threads from message processing (edunov via pavanka)  
 
   GIRAPH-924: Fix checkpointing (edunov via majakabiljo)


[26/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-888. Please create a DOAP file for your TLP


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/181ce395
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/181ce395
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/181ce395

Branch: refs/heads/release-1.1
Commit: 181ce39561c10aca2f4064047309d4a2ed66638f
Parents: b914fec
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Sat Aug 9 13:40:51 2014 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Sat Aug 9 13:40:51 2014 -0700

----------------------------------------------------------------------
 src/site/resources/doap_Giraph.rdf | 50 +++++++++++++++++++++++++++++++++
 1 file changed, 50 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/181ce395/src/site/resources/doap_Giraph.rdf
----------------------------------------------------------------------
diff --git a/src/site/resources/doap_Giraph.rdf b/src/site/resources/doap_Giraph.rdf
new file mode 100644
index 0000000..676c4da
--- /dev/null
+++ b/src/site/resources/doap_Giraph.rdf
@@ -0,0 +1,50 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl"?>
+<rdf:RDF xml:lang="en"
+         xmlns="http://usefulinc.com/ns/doap#" 
+         xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#" 
+         xmlns:asfext="http://projects.apache.org/ns/asfext#"
+         xmlns:foaf="http://xmlns.com/foaf/0.1/">
+<!--
+    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.
+-->
+  <Project rdf:about="http://giraph.apache.org">
+    <created>2014-06-30</created>
+    <license rdf:resource="http://spdx.org/licenses/Apache-2.0" />
+    <name>Apache Giraph</name>
+    <homepage rdf:resource="http://giraph.apache.org" />
+    <asfext:pmc rdf:resource="http://giraph.apache.org" />
+    <shortdesc>Apache Giraph is an iterative graph processing system built for high scalability. </shortdesc>
+    <description>Apache Giraph is an iterative graph processing system built for high scalability. For example, it is currently used at Facebook to analyze the social graph formed by users and their connections. </description>
+    <bug-database rdf:resource="https://issues.apache.org/jira/browse/GIRAPH" />
+    <mailing-list rdf:resource="http://giraph.apache.org/mail-lists.html" />
+    <download-page rdf:resource="http://www.apache.org/dyn/closer.cgi/giraph" />
+    <programming-language>Java</programming-language>
+    <category rdf:resource="http://projects.apache.org/category/big-data" />
+    <repository>
+      <GitRepository>
+        <location rdf:resource="https://git-wip-us.apache.org/repos/asf/giraph.git"/>
+        <browse rdf:resource="https://git-wip-us.apache.org/repos/asf/giraph.git"/>
+      </GitRepository>
+    </repository>
+    <maintainer>
+      <foaf:Person>
+        <foaf:name>Avery Ching</foaf:name>
+          <foaf:mbox rdf:resource="mailto:aching@apache.org"/>
+      </foaf:Person>
+    </maintainer>
+  </Project>
+</rdf:RDF>


[36/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-952: Limit job runtime


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/61db6891
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/61db6891
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/61db6891

Branch: refs/heads/release-1.1
Commit: 61db689128679b886753fed0b8f310b0ece9e0cf
Parents: da3c7b2
Author: Maja Kabiljo <ma...@fb.com>
Authored: Thu Oct 2 15:28:23 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Fri Oct 3 09:38:22 2014 -0700

----------------------------------------------------------------------
 .../org/apache/giraph/conf/GiraphConstants.java | 10 +++++
 .../java/org/apache/giraph/job/GiraphJob.java   |  3 ++
 .../giraph/job/JobProgressTrackerService.java   | 42 ++++++++++++++++++++
 3 files changed, 55 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/61db6891/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 d1fdf57..e78eb42 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
@@ -251,6 +251,16 @@ public interface GiraphConstants {
           "Class which decides whether a failed job should be retried - " +
               "optional");
 
+  /**
+   * Maximum allowed time for job to run after getting all resources before it
+   * will be killed, in milliseconds (-1 if it has no limit)
+   */
+  LongConfOption MAX_ALLOWED_JOB_TIME_MS =
+      new LongConfOption("giraph.maxAllowedJobTimeMilliseconds", -1,
+          "Maximum allowed time for job to run after getting all resources " +
+              "before it will be killed, in milliseconds " +
+              "(-1 if it has no limit)");
+
   // At least one of the input format classes is required.
   /** VertexInputFormat class */
   ClassConfOption<VertexInputFormat> VERTEX_INPUT_FORMAT_CLASS =

http://git-wip-us.apache.org/repos/asf/giraph/blob/61db6891/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
index 491d3d2..ca1ad1c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
@@ -254,6 +254,9 @@ public class GiraphJob {
       submittedJob.setMapperClass(GraphMapper.class);
       submittedJob.setInputFormatClass(BspInputFormat.class);
       submittedJob.setOutputFormatClass(BspOutputFormat.class);
+      if (jobProgressTrackerService != null) {
+        jobProgressTrackerService.setJob(submittedJob);
+      }
 
       GiraphJobObserver jobObserver = conf.getJobObserver();
       jobObserver.launchingJob(submittedJob);

http://git-wip-us.apache.org/repos/asf/giraph/blob/61db6891/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java b/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
index 3a896e2..49610de 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/JobProgressTrackerService.java
@@ -19,7 +19,9 @@
 package org.apache.giraph.job;
 
 import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.worker.WorkerProgress;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.log4j.Logger;
 
 import com.facebook.swift.codec.ThriftCodecManager;
@@ -28,6 +30,7 @@ import com.facebook.swift.service.ThriftServer;
 import com.facebook.swift.service.ThriftServerConfig;
 import com.facebook.swift.service.ThriftServiceProcessor;
 
+import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Map;
@@ -58,6 +61,8 @@ public class JobProgressTrackerService implements JobProgressTracker {
   /** Map of worker progresses */
   private final Map<Integer, WorkerProgress> workerProgresses =
       new ConcurrentHashMap<>();
+  /** Job */
+  private Job job;
 
   /**
    * Constructor
@@ -107,12 +112,49 @@ public class JobProgressTrackerService implements JobProgressTracker {
     writerThread.start();
   }
 
+  public void setJob(Job job) {
+    this.job = job;
+  }
+
+  /**
+   * Called when job got all mappers, used to check MAX_ALLOWED_JOB_TIME_MS
+   * and potentially start a thread which will kill the job after this time
+   */
+  private void jobGotAllMappers() {
+    final long maxAllowedJobTimeMs =
+        GiraphConstants.MAX_ALLOWED_JOB_TIME_MS.get(conf);
+    if (maxAllowedJobTimeMs > 0) {
+      // Start a thread which will kill the job if running for too long
+      new Thread(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            Thread.sleep(maxAllowedJobTimeMs);
+            try {
+              LOG.warn("Killing job because it took longer than " +
+                  maxAllowedJobTimeMs + " milliseconds");
+              job.killJob();
+            } catch (IOException e) {
+              LOG.warn("Failed to kill job", e);
+            }
+          } catch (InterruptedException e) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Thread checking for jobs max allowed time " +
+                  "interrupted");
+            }
+          }
+        }
+      }).start();
+    }
+  }
+
   @Override
   public synchronized void mapperStarted() {
     mappersStarted++;
     if (LOG.isInfoEnabled()) {
       if (mappersStarted == conf.getMaxWorkers() + 1) {
         LOG.info("Got all " + mappersStarted + " mappers");
+        jobGotAllMappers();
       } else {
         if (System.currentTimeMillis() - lastTimeMappersStartedLogged >
             UPDATE_MILLISECONDS) {


[17/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-927: Decouple netty server threads from message processing (edunov via pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/969a4881
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/969a4881
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/969a4881

Branch: refs/heads/release-1.1
Commit: 969a488183a42dedbde8dcec7ac00595a836974c
Parents: 02d9e6c
Author: Pavan Kumar <pa...@fb.com>
Authored: Fri Jul 18 16:20:17 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Fri Jul 18 16:20:17 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../java/org/apache/giraph/comm/ServerData.java |  11 +
 .../messages/InMemoryMessageStoreFactory.java   |  13 +-
 .../queue/AsyncMessageStoreWrapper.java         | 238 +++++++++++++++++++
 .../comm/messages/queue/PartitionMessage.java   |  71 ++++++
 .../comm/messages/queue/package-info.java       |  22 ++
 .../org/apache/giraph/conf/GiraphConstants.java |   8 +
 .../apache/giraph/graph/GraphTaskManager.java   |   5 +-
 .../apache/giraph/worker/BspServiceWorker.java  |   8 +
 .../queue/AsyncMessageStoreWrapperTest.java     | 123 ++++++++++
 .../org/apache/giraph/TestCheckpointing.java    |  13 +-
 11 files changed, 508 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 4207339..d6d998c 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-927: Decouple netty server threads from message processing (edunov via pavanka)  
+
   GIRAPH-924: Fix checkpointing (edunov via majakabiljo)
 
   GIRAPH-921: Create ByteValueVertex to store vertex values as bytes without object instance (akyrola via majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
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 036510e..29488fc 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
@@ -23,6 +23,7 @@ import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
 import org.apache.giraph.comm.aggregators.OwnerAggregatorServerData;
 import org.apache.giraph.comm.messages.MessageStore;
 import org.apache.giraph.comm.messages.MessageStoreFactory;
+import org.apache.giraph.comm.messages.queue.AsyncMessageStoreWrapper;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.EdgeStore;
@@ -202,6 +203,16 @@ public class ServerData<I extends WritableComparable,
   }
 
   /**
+   * In case of async message store we have to wait for all messages
+   * to be processed before going into next superstep.
+   */
+  public void waitForComplete() {
+    if (incomingMessageStore instanceof AsyncMessageStoreWrapper) {
+      ((AsyncMessageStoreWrapper) incomingMessageStore).waitToComplete();
+    }
+  }
+
+  /**
    * Get the vertex mutations (synchronize on the values)
    *
    * @return Vertex mutations

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
index db22503..02ea7b2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
@@ -24,8 +24,9 @@ import org.apache.giraph.comm.messages.primitives.IntByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.IntFloatMessageStore;
 import org.apache.giraph.comm.messages.primitives.long_id.LongByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.LongDoubleMessageStore;
-import org.apache.giraph.comm.messages.primitives.long_id.LongPointerListMessageStore;
+import org.apache.giraph.comm.messages.queue.AsyncMessageStoreWrapper;
 import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.comm.messages.primitives.long_id.LongPointerListMessageStore;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.factories.MessageValueFactory;
 import org.apache.hadoop.io.DoubleWritable;
@@ -155,6 +156,16 @@ public class InMemoryMessageStoreFactory<I extends WritableComparable,
           (conf.useMessageCombiner() ? " message combiner " +
               conf.getMessageCombinerClass() : " no combiner"));
     }
+
+    int asyncMessageStoreThreads =
+        GiraphConstants.ASYNC_MESSAGE_STORE_THREADS_COUNT.get(conf);
+    if (asyncMessageStoreThreads > 0) {
+      messageStore = new AsyncMessageStoreWrapper(
+          messageStore,
+          service.getPartitionStore().getPartitionIds(),
+          asyncMessageStoreThreads);
+    }
+
     return messageStore;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
new file mode 100644
index 0000000..a62834f
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
@@ -0,0 +1,238 @@
+/*
+ * 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.comm.messages.queue;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import it.unimi.dsi.fastutil.ints.Int2IntArrayMap;
+import it.unimi.dsi.fastutil.ints.Int2IntMap;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.log4j.Logger;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
+
+/**
+ * This class decouples message receiving and processing
+ * into separate threads thus reducing contention.
+ * It does not provide message store functionality itself, rather
+ * providing a wrapper around existing message stores that
+ * can now be used in async mode with only slight modifications.
+ * @param <I> Vertex id
+ * @param <M> Message data
+ */
+public final class AsyncMessageStoreWrapper<I extends WritableComparable,
+    M extends Writable> implements MessageStore<I, M> {
+
+  /** Logger */
+  private static final Logger LOG =
+      Logger.getLogger(AsyncMessageStoreWrapper.class);
+  /** Pass this id to clear the queues and shutdown all threads
+   * started by this processor */
+  private static final PartitionMessage SHUTDOWN_QUEUE_MESSAGE =
+      new PartitionMessage(-1, null);
+  /** Pass this message to clear the queues but keep threads alive */
+  private static final PartitionMessage CLEAR_QUEUE_MESSAGE =
+      new PartitionMessage(-1, null);
+  /** Executor that processes messages in background */
+  private static final ExecutorService EXECUTOR_SERVICE =
+      Executors.newCachedThreadPool(
+          new ThreadFactoryBuilder()
+              .setNameFormat("AsyncMessageStoreWrapper-%d").build());
+
+  /** Number of threads that will process messages in background */
+  private final int threadsCount;
+  /** Queue that temporary stores messages */
+  private final BlockingQueue<PartitionMessage<I, M>>[] queues;
+  /** Map from partition id to thread that process this partition */
+  private final Int2IntMap partition2Queue;
+  /** Signals that all procesing is done */
+  private Semaphore completionSemaphore;
+  /** Underlying message store */
+  private final MessageStore<I, M> store;
+
+  /**
+   * Constructs async wrapper around existing message store
+   * object. Requires partition list and number of threads
+   * to properly initialize background threads and assign partitions.
+   * Partitions are assigned to threads in round-robin fashion.
+   * It guarantees that all threads have almost the same number of
+   * partitions (+-1) no matter how partitions are assigned to this worker.
+   * @param store underlying message store to be used in computation
+   * @param partitions partitions assigned to this worker
+   * @param threadCount number of threads that will be used to process
+   *                    messages.
+   */
+  public AsyncMessageStoreWrapper(MessageStore<I, M> store,
+                                  Iterable<Integer> partitions,
+                                  int threadCount) {
+    this.store = store;
+    this.threadsCount = threadCount;
+    completionSemaphore = new Semaphore(1 - threadsCount);
+    queues = new BlockingQueue[threadsCount];
+    partition2Queue = new Int2IntArrayMap();
+    LOG.info("AsyncMessageStoreWrapper enabled. Threads= " + threadsCount);
+
+    for (int i = 0; i < threadsCount; i++) {
+      queues[i] = new LinkedBlockingQueue<>();
+      EXECUTOR_SERVICE.submit(new MessageStoreQueueWorker(queues[i]));
+    }
+
+    int cnt = 0;
+    for (int partitionId : partitions) {
+      partition2Queue.put(partitionId, cnt++ % threadsCount);
+    }
+
+  }
+
+  @Override
+  public boolean isPointerListEncoding() {
+    return store.isPointerListEncoding();
+  }
+
+  @Override
+  public Iterable<M> getVertexMessages(I vertexId) throws IOException {
+    return store.getVertexMessages(vertexId);
+  }
+
+  @Override
+  public void clearVertexMessages(I vertexId) throws IOException {
+    store.clearVertexMessages(vertexId);
+  }
+
+  @Override
+  public void clearAll() throws IOException {
+    try {
+      for (BlockingQueue<PartitionMessage<I, M>> queue : queues) {
+        queue.put(SHUTDOWN_QUEUE_MESSAGE);
+      }
+      completionSemaphore.acquire();
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+    store.clearAll();
+  }
+
+  @Override
+  public boolean hasMessagesForVertex(I vertexId) {
+    return store.hasMessagesForVertex(vertexId);
+  }
+
+  @Override
+  public void addPartitionMessages(
+      int partitionId, VertexIdMessages<I, M> messages) throws IOException {
+    int hash = partition2Queue.get(partitionId);
+    try {
+      queues[hash].put(new PartitionMessage<>(partitionId, messages));
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void finalizeStore() {
+    store.finalizeStore();
+  }
+
+  @Override
+  public Iterable<I> getPartitionDestinationVertices(int partitionId) {
+    return store.getPartitionDestinationVertices(partitionId);
+  }
+
+  @Override
+  public void clearPartition(int partitionId) throws IOException {
+    store.clearPartition(partitionId);
+  }
+
+  @Override
+  public void writePartition(DataOutput out, int partitionId)
+    throws IOException {
+    store.writePartition(out, partitionId);
+  }
+
+  @Override
+  public void readFieldsForPartition(DataInput in, int partitionId)
+    throws IOException {
+    store.readFieldsForPartition(in, partitionId);
+  }
+
+  /**
+   * Wait till all messages are processed and all queues are empty.
+   */
+  public void waitToComplete() {
+    try {
+      for (BlockingQueue<PartitionMessage<I, M>> queue : queues) {
+        queue.put(CLEAR_QUEUE_MESSAGE);
+      }
+      completionSemaphore.acquire();
+      completionSemaphore = new Semaphore(1 - threadsCount);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * This runnable has logic for background thread
+   * that actually does message processing.
+   */
+  private class MessageStoreQueueWorker implements Runnable {
+    /**
+     * Queue assigned to this background thread.
+     */
+    private final BlockingQueue<PartitionMessage<I, M>> queue;
+
+    /**
+     * Constructs runnable.
+     * @param queue where messages are put by client
+     */
+    private MessageStoreQueueWorker(
+        BlockingQueue<PartitionMessage<I, M>> queue) {
+      this.queue = queue;
+    }
+
+    @Override
+    public void run() {
+      PartitionMessage<I, M> message = null;
+      while (true) {
+        try {
+          message = queue.take();
+          if (message.getMessage() != null) {
+            int partitionId = message.getPartitionId();
+            store.addPartitionMessages(partitionId, message.getMessage());
+          } else {
+            completionSemaphore.release();
+            if (message == SHUTDOWN_QUEUE_MESSAGE) {
+              return;
+            }
+          }
+        } catch (IOException | InterruptedException e) {
+          LOG.error("MessageStoreQueueWorker.run: " + message, e);
+          return;
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/PartitionMessage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/PartitionMessage.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/PartitionMessage.java
new file mode 100644
index 0000000..8c884ce
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/PartitionMessage.java
@@ -0,0 +1,71 @@
+/*
+ * 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.comm.messages.queue;
+
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Small wrapper that holds a reference to vertex message
+ * and knows partition id.
+ * @param <I> vertexId type parameter
+ * @param <M> message type parameter
+ */
+public class PartitionMessage<I extends WritableComparable,
+    M extends Writable> {
+  /** partition id */
+  private int partitionId;
+  /** vertext message */
+  private VertexIdMessages<I, M> message;
+
+  /**
+   * Constructs wrapper from partitino id and vertext message
+   * object.
+   * @param partitionId destination partition id
+   * @param message message object
+   */
+  public PartitionMessage(int partitionId, VertexIdMessages<I, M> message) {
+    this.partitionId = partitionId;
+    this.message = message;
+  }
+
+  /**
+   * Partition id
+   * @return destination partition id.
+   */
+  public int getPartitionId() {
+    return partitionId;
+  }
+
+  /**
+   * Message
+   * @return vertex message
+   */
+  public VertexIdMessages<I, M> getMessage() {
+    return message;
+  }
+
+  @Override
+  public String toString() {
+    return "PartitionMessage{" +
+        "partitionId=" + partitionId +
+        ", message=" + message +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/package-info.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/package-info.java
new file mode 100644
index 0000000..e54f9f2
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 for message store queue, that decouples netty threads from
+ * threads processing messages.
+ */
+package org.apache.giraph.comm.messages.queue;

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 3d16e9c..0424a47 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
@@ -1143,5 +1143,13 @@ public interface GiraphConstants {
           "org.apache.hadoop.io.compress.DefaultCodec",
           "Defines compression algorithm we will be using for " +
               "storing checkpoint");
+
+  /** Number of threads to use in async message store, 0 means
+   * we should not use async message processing */
+  IntConfOption ASYNC_MESSAGE_STORE_THREADS_COUNT =
+      new IntConfOption("giraph.async.message.store.threads", 0,
+          "Number of threads to be used in async message store.");
+
+
 }
 // CHECKSTYLE: resume InterfaceIsTypeCheck

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 b2a5c84..684f4eb 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
@@ -38,8 +38,6 @@ import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionOwner;
 import org.apache.giraph.partition.PartitionStats;
 import org.apache.giraph.partition.PartitionStore;
-import org.apache.giraph.time.SystemTime;
-import org.apache.giraph.time.Time;
 import org.apache.giraph.utils.CallableFactory;
 import org.apache.giraph.utils.MemoryUtils;
 import org.apache.giraph.utils.ProgressableUtils;
@@ -102,8 +100,6 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   /** Name of metric for time from first message till last message flushed */
   public static final String TIMER_COMMUNICATION_TIME = "communication-time-ms";
 
-  /** Time instance used for timing in this class */
-  private static final Time TIME = SystemTime.get();
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(GraphTaskManager.class);
   /** Coordination service worker */
@@ -304,6 +300,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
       }
       finishedSuperstepStats = completeSuperstepAndCollectStats(
         partitionStatsList, superstepTimerContext);
+
       // END of superstep compute loop
     }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index 0d90a59..d2d24ee 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -26,6 +26,8 @@ import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
 import org.apache.giraph.comm.WorkerServer;
 import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.comm.messages.queue.AsyncMessageStoreWrapper;
 import org.apache.giraph.comm.netty.NettyWorkerAggregatorRequestProcessor;
 import org.apache.giraph.comm.netty.NettyWorkerClient;
 import org.apache.giraph.comm.netty.NettyWorkerClientRequestProcessor;
@@ -891,6 +893,12 @@ public class BspServiceWorker<I extends WritableComparable,
 
     aggregatorHandler.finishSuperstep(workerAggregatorRequestProcessor);
 
+    MessageStore<I, Writable> incomingMessageStore =
+        getServerData().getIncomingMessageStore();
+    if (incomingMessageStore instanceof AsyncMessageStoreWrapper) {
+      ((AsyncMessageStoreWrapper) incomingMessageStore).waitToComplete();
+    }
+
     if (LOG.isInfoEnabled()) {
       LOG.info("finishSuperstep: Superstep " + getSuperstep() +
           ", messages = " + workerSentMessages + " " +

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-core/src/test/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapperTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapperTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapperTest.java
new file mode 100644
index 0000000..ca1031a
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapperTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.comm.messages.queue;
+
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.factories.TestMessageValueFactory;
+import org.apache.giraph.utils.ByteArrayVertexIdMessages;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test case for AsyncMessageStoreWrapper
+ */
+public class AsyncMessageStoreWrapperTest {
+
+
+  @Test
+  public void testAsyncQueue() throws IOException {
+    TestMessageStore store = new TestMessageStore();
+
+    AsyncMessageStoreWrapper<LongWritable, IntWritable> queue =
+        new AsyncMessageStoreWrapper<>(store,
+        Arrays.asList(0, 1, 2, 3, 4), 2);
+
+    for (int i = 0; i < 1000; i++) {
+      queue.addPartitionMessages(i % 5, new ByteArrayVertexIdMessages<LongWritable, IntWritable>(new TestMessageValueFactory<>(IntWritable.class)));
+    }
+
+    queue.waitToComplete();
+
+    assertArrayEquals(new int[] {200, 200, 200, 200, 200}, store.counters);
+
+    queue.clearAll();
+  }
+
+
+  static class TestMessageStore implements MessageStore<LongWritable, IntWritable> {
+
+    private int counters[] = new int[5];
+
+    @Override
+    public void addPartitionMessages(int partition, VertexIdMessages messages) throws IOException {
+      assertNotNull(messages);
+      counters[partition]++;
+    }
+
+    @Override
+    public boolean isPointerListEncoding() {
+      return false;
+    }
+
+    @Override
+    public Iterable<IntWritable> getVertexMessages(LongWritable vertexId) throws IOException {
+      return null;
+    }
+
+    @Override
+    public void clearVertexMessages(LongWritable vertexId) throws IOException {
+
+    }
+
+    @Override
+    public void clearAll() throws IOException {
+
+    }
+
+    @Override
+    public boolean hasMessagesForVertex(LongWritable vertexId) {
+      return false;
+    }
+
+    @Override
+    public void finalizeStore() {
+
+    }
+
+    @Override
+    public Iterable<LongWritable> getPartitionDestinationVertices(int partitionId) {
+      return null;
+    }
+
+    @Override
+    public void clearPartition(int partitionId) throws IOException {
+
+    }
+
+    @Override
+    public void writePartition(DataOutput out, int partitionId) throws IOException {
+
+    }
+
+    @Override
+    public void readFieldsForPartition(DataInput in, int partitionId) throws IOException {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/969a4881/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java b/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
index 387b937..2939af7 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
@@ -71,9 +71,17 @@ public class TestCheckpointing extends BspCase {
     super(TestCheckpointing.class.getName());
   }
 
+  @Test
+  public void testBspCheckpoint() throws InterruptedException, IOException, ClassNotFoundException {
+    testBspCheckpoint(false);
+  }
 
   @Test
-  public void testBspCheckpoint()
+  public void testAsyncMessageStoreCheckpoint() throws InterruptedException, IOException, ClassNotFoundException {
+    testBspCheckpoint(true);
+  }
+
+  public void testBspCheckpoint(boolean useAsyncMessageStore)
       throws IOException, InterruptedException, ClassNotFoundException {
     Path checkpointsDir = getTempPath("checkpointing");
     Path outputPath = getTempPath(getCallingMethodName());
@@ -88,6 +96,9 @@ public class TestCheckpointing extends BspCase {
     conf.setVertexOutputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat.class);
     conf.set("mapred.job.id", TEST_JOB_ID);
     conf.set(KEY_MIN_SUPERSTEP, "0");
+    if (useAsyncMessageStore) {
+      GiraphConstants.ASYNC_MESSAGE_STORE_THREADS_COUNT.set(conf, 2);
+    }
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
 
     GiraphConfiguration configuration = job.getConfiguration();


[22/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Fix GIRAPH-934


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/d455270e
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/d455270e
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/d455270e

Branch: refs/heads/release-1.1
Commit: d455270e2a8d5ec34d63d20ccb0f99d3629efa5e
Parents: 24309fe
Author: Maja Kabiljo <ma...@fb.com>
Authored: Mon Aug 4 13:46:51 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Mon Aug 4 13:46:51 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/giraph/master/MasterAggregatorHandler.java    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/d455270e/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
index 2bc08e9..2b0cdd6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
@@ -337,7 +337,8 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
     for (Map.Entry<String, AggregatorWrapper<Writable>> entry :
         aggregatorMap.entrySet()) {
       out.writeUTF(entry.getKey());
-      entry.getValue().getAggregatorFactory().write(out);
+      WritableUtils.writeWritableObject(
+          entry.getValue().getAggregatorFactory(), out);
       out.writeBoolean(entry.getValue().isPersistent());
       entry.getValue().getPreviousAggregatedValue().write(out);
       progressable.progress();


[20/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-936: AsyncMessageStoreWrapper threads are not daemonized (edunov via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/6d4af60e
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/6d4af60e
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/6d4af60e

Branch: refs/heads/release-1.1
Commit: 6d4af60e729a6b2153c220046cf6a3804427f1c1
Parents: ce97134
Author: Maja Kabiljo <ma...@fb.com>
Authored: Fri Aug 1 11:46:52 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Fri Aug 1 11:46:52 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                                          | 2 ++
 .../giraph/comm/messages/queue/AsyncMessageStoreWrapper.java       | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/6d4af60e/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index dbb134a..3c8e155 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-936: AsyncMessageStoreWrapper threads are not daemonized (edunov via majakabiljo)
+
   GIRAPH-934: Allow having state in aggregators (ikabiljo via majakabiljo)
 
   GIRAPH-932: Adding .arcconfig to GIRAPH for Arcanist support (aching)

http://git-wip-us.apache.org/repos/asf/giraph/blob/6d4af60e/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
index a62834f..252ee39 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/queue/AsyncMessageStoreWrapper.java
@@ -60,7 +60,7 @@ public final class AsyncMessageStoreWrapper<I extends WritableComparable,
   /** Executor that processes messages in background */
   private static final ExecutorService EXECUTOR_SERVICE =
       Executors.newCachedThreadPool(
-          new ThreadFactoryBuilder()
+          new ThreadFactoryBuilder().setDaemon(true)
               .setNameFormat("AsyncMessageStoreWrapper-%d").build());
 
   /** Number of threads that will process messages in background */


[32/47] GIRAPH-938: Allow fast working with primitives generically (ikabiljo via pavanka)

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicArrayList.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicArrayList.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicArrayList.java
new file mode 100644
index 0000000..df5ca24
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicArrayList.java
@@ -0,0 +1,632 @@
+/*
+ * 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.types.ops.collections;
+
+import it.unimi.dsi.fastutil.booleans.BooleanArrayList;
+import it.unimi.dsi.fastutil.bytes.ByteArrayList;
+import it.unimi.dsi.fastutil.doubles.DoubleArrayList;
+import it.unimi.dsi.fastutil.floats.FloatArrayList;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import org.apache.giraph.types.ops.BooleanTypeOps;
+import org.apache.giraph.types.ops.ByteTypeOps;
+import org.apache.giraph.types.ops.DoubleTypeOps;
+import org.apache.giraph.types.ops.FloatTypeOps;
+import org.apache.giraph.types.ops.IntTypeOps;
+import org.apache.giraph.types.ops.LongTypeOps;
+import org.apache.giraph.types.ops.PrimitiveTypeOps;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.ByteWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * BasicArrayList with only basic set of operations.
+ *
+ * @param <T> Element type
+ */
+public abstract class BasicArrayList<T> implements Writable {
+  /** Removes all of the elements from this list. */
+  public abstract void clear();
+  /**
+   * Number of elements in this list
+   * @return size
+   */
+  public abstract int size();
+  /**
+   * Capacity of currently allocated memory
+   * @return capacity
+   */
+  public abstract int capacity();
+  /**
+   * Forces allocated memory to hold exactly N values
+   * @param n new capacity
+   */
+  public abstract void setCapacity(int n);
+  /**
+   * Add value to the end of the array
+   * @param value Value
+   */
+  public abstract void add(T value);
+  /**
+   * Pop value from the end of the array, storing it into 'to' argument
+   * @param to Object to store value into
+   */
+  public abstract void popInto(T to);
+  /**
+   * Get element at given index in the array, storing it into 'to' argument
+   * @param index Index
+   * @param to Object to store value into
+   */
+  public abstract void getInto(int index, T to);
+  /**
+   * Set element at given index in the array
+   * @param index Index
+   * @param value Value
+   */
+  public abstract void set(int index, T value);
+
+  /**
+   * TypeOps for type of elements this object holds
+   * @return TypeOps
+   */
+  public abstract PrimitiveTypeOps<T> getElementTypeOps();
+
+  /**
+   * Fast iterator over BasicArrayList object, which doesn't allocate new
+   * element for each returned element, and can be iterated multiple times
+   * using reset().
+   *
+   * Object returned by next() is only valid until next() is called again,
+   * because it is reused.
+   *
+   * @return RessettableIterator
+   */
+  public ResettableIterator<T> fastIterator() {
+    return new ResettableIterator<T>() {
+      private final T value = getElementTypeOps().create();
+      private int pos;
+
+      @Override
+      public boolean hasNext() {
+        return pos < size();
+      }
+
+      @Override
+      public T next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        getInto(pos, value);
+        pos++;
+        return value;
+      }
+
+      @Override
+      public void reset() {
+        pos = 0;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  /** BooleanWritable implementation of BasicArrayList */
+  public static final class BasicBooleanArrayList
+      extends BasicArrayList<BooleanWritable> {
+    /** List */
+    private final BooleanArrayList list;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicBooleanArrayList(int capacity) {
+      list = new BooleanArrayList(capacity);
+    }
+
+    @Override
+    public PrimitiveTypeOps<BooleanWritable> getElementTypeOps() {
+      return BooleanTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public int size() {
+      return list.size();
+    }
+
+    @Override
+    public int capacity() {
+      return list.elements().length;
+    }
+
+    @Override
+    public void setCapacity(int n) {
+      if (n >= list.elements().length) {
+        list.ensureCapacity(n);
+      } else {
+        list.trim(n);
+      }
+    }
+
+    @Override
+    public void add(BooleanWritable value) {
+      list.add(value.get());
+    }
+
+    @Override
+    public void getInto(int index, BooleanWritable to) {
+      to.set(list.getBoolean(index));
+    }
+
+    @Override
+    public void popInto(BooleanWritable to) {
+      to.set(list.popBoolean());
+    }
+
+    @Override
+    public void set(int index, BooleanWritable value) {
+      list.set(index, value.get());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(list.size());
+      for (int i = 0; i < list.size(); i++) {
+        out.writeBoolean(list.getBoolean(i));
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      list.clear();
+      setCapacity(size);
+      for (int i = 0; i < size; ++i) {
+        list.add(in.readBoolean());
+      }
+    }
+  }
+
+  /** ByteWritable implementation of BasicArrayList */
+  public static final class BasicByteArrayList
+      extends BasicArrayList<ByteWritable> {
+    /** List */
+    private final ByteArrayList list;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicByteArrayList(int capacity) {
+      list = new ByteArrayList(capacity);
+    }
+
+    @Override
+    public PrimitiveTypeOps<ByteWritable> getElementTypeOps() {
+      return ByteTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public int size() {
+      return list.size();
+    }
+
+    @Override
+    public int capacity() {
+      return list.elements().length;
+    }
+
+    @Override
+    public void setCapacity(int n) {
+      if (n >= list.elements().length) {
+        list.ensureCapacity(n);
+      } else {
+        list.trim(n);
+      }
+    }
+
+    @Override
+    public void add(ByteWritable value) {
+      list.add(value.get());
+    }
+
+    @Override
+    public void getInto(int index, ByteWritable to) {
+      to.set(list.getByte(index));
+    }
+
+    @Override
+    public void popInto(ByteWritable to) {
+      to.set(list.popByte());
+    }
+
+    @Override
+    public void set(int index, ByteWritable value) {
+      list.set(index, value.get());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(list.size());
+      for (int i = 0; i < list.size(); i++) {
+        out.writeByte(list.getByte(i));
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      list.clear();
+      setCapacity(size);
+      for (int i = 0; i < size; ++i) {
+        list.add(in.readByte());
+      }
+    }
+  }
+
+  /** IntWritable implementation of BasicArrayList */
+  public static final class BasicIntArrayList
+      extends BasicArrayList<IntWritable> {
+    /** List */
+    private final IntArrayList list;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicIntArrayList(int capacity) {
+      list = new IntArrayList(capacity);
+    }
+
+    @Override
+    public PrimitiveTypeOps<IntWritable> getElementTypeOps() {
+      return IntTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public int size() {
+      return list.size();
+    }
+
+    @Override
+    public int capacity() {
+      return list.elements().length;
+    }
+
+    @Override
+    public void setCapacity(int n) {
+      if (n >= list.elements().length) {
+        list.ensureCapacity(n);
+      } else {
+        list.trim(n);
+      }
+    }
+
+    @Override
+    public void add(IntWritable value) {
+      list.add(value.get());
+    }
+
+    @Override
+    public void getInto(int index, IntWritable to) {
+      to.set(list.getInt(index));
+    }
+
+    @Override
+    public void popInto(IntWritable to) {
+      to.set(list.popInt());
+    }
+
+    @Override
+    public void set(int index, IntWritable value) {
+      list.set(index, value.get());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(list.size());
+      for (int i = 0; i < list.size(); i++) {
+        out.writeInt(list.getInt(i));
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      list.clear();
+      setCapacity(size);
+      for (int i = 0; i < size; ++i) {
+        list.add(in.readInt());
+      }
+    }
+  }
+
+  /** LongWritable implementation of BasicArrayList */
+  public static final class BasicLongArrayList
+      extends BasicArrayList<LongWritable> {
+    /** List */
+    private final LongArrayList list;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicLongArrayList(int capacity) {
+      list = new LongArrayList(capacity);
+    }
+
+    @Override
+    public PrimitiveTypeOps<LongWritable> getElementTypeOps() {
+      return LongTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public int size() {
+      return list.size();
+    }
+
+    @Override
+    public int capacity() {
+      return list.elements().length;
+    }
+
+    @Override
+    public void setCapacity(int n) {
+      if (n >= list.elements().length) {
+        list.ensureCapacity(n);
+      } else {
+        list.trim(n);
+      }
+    }
+
+    @Override
+    public void add(LongWritable value) {
+      list.add(value.get());
+    }
+
+    @Override
+    public void getInto(int index, LongWritable to) {
+      to.set(list.getLong(index));
+    }
+
+    @Override
+    public void popInto(LongWritable to) {
+      to.set(list.popLong());
+    }
+
+    @Override
+    public void set(int index, LongWritable value) {
+      list.set(index, value.get());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(list.size());
+      for (int i = 0; i < list.size(); i++) {
+        out.writeLong(list.getLong(i));
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      list.clear();
+      setCapacity(size);
+      for (int i = 0; i < size; ++i) {
+        list.add(in.readLong());
+      }
+    }
+  }
+
+  /** FloatWritable implementation of BasicArrayList */
+  public static final class BasicFloatArrayList
+      extends BasicArrayList<FloatWritable> {
+    /** List */
+    private final FloatArrayList list;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicFloatArrayList(int capacity) {
+      list = new FloatArrayList(capacity);
+    }
+
+    @Override
+    public PrimitiveTypeOps<FloatWritable> getElementTypeOps() {
+      return FloatTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public int size() {
+      return list.size();
+    }
+
+    @Override
+    public int capacity() {
+      return list.elements().length;
+    }
+
+    @Override
+    public void setCapacity(int n) {
+      if (n >= list.elements().length) {
+        list.ensureCapacity(n);
+      } else {
+        list.trim(n);
+      }
+    }
+
+    @Override
+    public void add(FloatWritable value) {
+      list.add(value.get());
+    }
+
+    @Override
+    public void getInto(int index, FloatWritable to) {
+      to.set(list.getFloat(index));
+    }
+
+    @Override
+    public void popInto(FloatWritable to) {
+      to.set(list.popFloat());
+    }
+
+    @Override
+    public void set(int index, FloatWritable value) {
+      list.set(index, value.get());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(list.size());
+      for (int i = 0; i < list.size(); i++) {
+        out.writeFloat(list.getFloat(i));
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      list.clear();
+      setCapacity(size);
+      for (int i = 0; i < size; ++i) {
+        list.add(in.readFloat());
+      }
+    }
+  }
+
+  /** DoubleWritable implementation of BasicArrayList */
+  public static final class BasicDoubleArrayList
+      extends BasicArrayList<DoubleWritable> {
+    /** List */
+    private final DoubleArrayList list;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicDoubleArrayList(int capacity) {
+      list = new DoubleArrayList(capacity);
+    }
+
+    @Override
+    public PrimitiveTypeOps<DoubleWritable> getElementTypeOps() {
+      return DoubleTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void clear() {
+      list.clear();
+    }
+
+    @Override
+    public int size() {
+      return list.size();
+    }
+
+    @Override
+    public int capacity() {
+      return list.elements().length;
+    }
+
+    @Override
+    public void setCapacity(int n) {
+      if (n >= list.elements().length) {
+        list.ensureCapacity(n);
+      } else {
+        list.trim(n);
+      }
+    }
+
+    @Override
+    public void add(DoubleWritable value) {
+      list.add(value.get());
+    }
+
+    @Override
+    public void getInto(int index, DoubleWritable to) {
+      to.set(list.getDouble(index));
+    }
+
+    @Override
+    public void popInto(DoubleWritable to) {
+      to.set(list.popDouble());
+    }
+
+    @Override
+    public void set(int index, DoubleWritable value) {
+      list.set(index, value.get());
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(list.size());
+      for (int i = 0; i < list.size(); i++) {
+        out.writeDouble(list.getDouble(i));
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      list.clear();
+      setCapacity(size);
+      for (int i = 0; i < size; ++i) {
+        list.add(in.readDouble());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicSet.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicSet.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicSet.java
new file mode 100644
index 0000000..c8cd72e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/BasicSet.java
@@ -0,0 +1,206 @@
+/*
+ * 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.types.ops.collections;
+
+import it.unimi.dsi.fastutil.ints.IntIterator;
+import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
+import it.unimi.dsi.fastutil.longs.LongIterator;
+import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.types.ops.IntTypeOps;
+import org.apache.giraph.types.ops.LongTypeOps;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * BasicSet with only basic set of operations.
+ * All operations that return object T are returning reusable object,
+ * which is modified after calling any other function.
+ *
+ * @param <T> Element type
+ */
+public interface BasicSet<T> extends Writable {
+  /** Removes all of the elements from this list. */
+  void clear();
+  /**
+   * Number of elements in this list
+   * @return size
+   */
+  int size();
+  /**
+   * Makes sure set is not using space with capacity more than
+   * max(n,size()) entries.
+   * @param n the threshold for the trimming.
+   */
+  void trim(int n);
+  /**
+   * Adds value to the set.
+   * Returns <tt>true</tt> if set changed as a
+   * result of the call.
+   *
+   * @param value Value to add
+   * @return true if set was changed.
+   */
+  boolean add(T value);
+  /**
+   * Checks whether set contains given value
+   * @param value Value to check
+   * @return true if value is present in the set
+   */
+  boolean contains(T value);
+
+  /**
+   * TypeOps for type of elements this object holds
+   * @return TypeOps
+   */
+  PrimitiveIdTypeOps<T> getElementTypeOps();
+
+  /** IntWritable implementation of BasicSet */
+  public static final class BasicIntOpenHashSet
+      implements BasicSet<IntWritable> {
+    /** Set */
+    private final IntOpenHashSet set;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicIntOpenHashSet(int capacity) {
+      set = new IntOpenHashSet(capacity);
+    }
+
+    @Override
+    public void clear() {
+      set.clear();
+    }
+
+    @Override
+    public int size() {
+      return set.size();
+    }
+
+    @Override
+    public void trim(int n) {
+      set.trim(Math.max(set.size(), n));
+    }
+
+    @Override
+    public boolean add(IntWritable value) {
+      return set.add(value.get());
+    }
+
+    @Override
+    public boolean contains(IntWritable value) {
+      return set.contains(value.get());
+    }
+
+    @Override
+    public PrimitiveIdTypeOps<IntWritable> getElementTypeOps() {
+      return IntTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(set.size());
+      IntIterator iter = set.iterator();
+      while (iter.hasNext()) {
+        out.writeInt(iter.nextInt());
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      set.clear();
+      set.trim(size);
+      for (int i = 0; i < size; ++i) {
+        set.add(in.readInt());
+      }
+    }
+  }
+
+  /** LongWritable implementation of BasicSet */
+  public static final class BasicLongOpenHashSet
+      implements BasicSet<LongWritable> {
+    /** Set */
+    private final LongOpenHashSet set;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicLongOpenHashSet(int capacity) {
+      set = new LongOpenHashSet(capacity);
+    }
+
+    @Override
+    public void clear() {
+      set.clear();
+    }
+
+    @Override
+    public int size() {
+      return set.size();
+    }
+
+    @Override
+    public void trim(int n) {
+      set.trim(Math.max(set.size(), n));
+    }
+
+    @Override
+    public boolean add(LongWritable value) {
+      return set.add(value.get());
+    }
+
+    @Override
+    public boolean contains(LongWritable value) {
+      return set.contains(value.get());
+    }
+
+    @Override
+    public PrimitiveIdTypeOps<LongWritable> getElementTypeOps() {
+      return LongTypeOps.INSTANCE;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeInt(set.size());
+      LongIterator iter = set.iterator();
+      while (iter.hasNext()) {
+        out.writeLong(iter.nextLong());
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int size = in.readInt();
+      set.clear();
+      trim(size);
+      for (int i = 0; i < size; ++i) {
+        set.add(in.readLong());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/ResettableIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/ResettableIterator.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/ResettableIterator.java
new file mode 100644
index 0000000..30e1b21
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/ResettableIterator.java
@@ -0,0 +1,32 @@
+/*
+ * 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.types.ops.collections;
+
+import java.util.Iterator;
+
+/**
+ * Defines an iterator that can iterated over multiple times, by
+ * rewinding it via reset() function.
+ *
+ * @param <T> Element type
+ */
+public interface ResettableIterator<T> extends Iterator<T> {
+  /** Rewinds iterator to the beginning. */
+  void reset();
+}
+

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/WritableWriter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/WritableWriter.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/WritableWriter.java
new file mode 100644
index 0000000..cd29a56
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/WritableWriter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.types.ops.collections;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Handler for knowing how to serialize/deserialize type T
+ *
+ * @param <T> Type of object to be serialized.
+ */
+public interface WritableWriter<T> {
+  /**
+   * Serialize the fields of <code>value</code> to <code>out</code>.
+   *
+   * @param out <code>DataOuput</code> to serialize object into.
+   * @param value Object to serialize
+   * @throws IOException
+   */
+  void write(DataOutput out, T value) throws IOException;
+
+  /**
+   * Deserialize the fields of object from <code>in</code>.
+   *
+   * @param in <code>DataInput</code> to deseriablize object from.
+   * @return Deserialized object.
+   * @throws IOException
+   */
+  T readFields(DataInput in) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/package-info.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/package-info.java
new file mode 100644
index 0000000..ae0231e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Collection interfaces and implementations provided by TypeOps classes.
+ */
+package org.apache.giraph.types.ops.collections;

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/package-info.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/package-info.java
new file mode 100644
index 0000000..f656711
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Type coercion, inference, and reflection.
+ */
+package org.apache.giraph.types.ops;


[09/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-713: Provide an option to do request compression (pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/4223ccc0
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/4223ccc0
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/4223ccc0

Branch: refs/heads/release-1.1
Commit: 4223ccc08bcd3689bddb310dddedab0485f7a6bd
Parents: 666d5fd
Author: Pavan Kumar <pa...@fb.com>
Authored: Mon Jul 7 16:48:08 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Mon Jul 7 16:48:08 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 +
 .../apache/giraph/comm/netty/NettyClient.java   | 21 +++++++-
 .../apache/giraph/comm/netty/NettyServer.java   | 20 ++++++++
 .../org/apache/giraph/conf/GiraphConstants.java |  5 ++
 .../ImmutableClassesGiraphConfiguration.java    | 54 ++++++++++++++++++++
 5 files changed, 101 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/4223ccc0/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 43aea7a..ea2f911 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-713: Provide an option to do request compression (pavanka)
+
   GIRAPH-923: Upgrade Netty version to a latest stable one (pavanka)
 
   GIRAPH-916: Wrong number of vertices stored reported to command line (majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/4223ccc0/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
index ae40c3b..5bb5545 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
@@ -268,11 +268,20 @@ public class NettyClient {
               PipelineUtils.addLastWithExecutorCheck("clientInboundByteCounter",
                   inboundByteCounter, handlerToUseExecutionGroup,
                   executionGroup, ch);
+              if (conf.doCompression()) {
+                PipelineUtils.addLastWithExecutorCheck("compressionDecoder",
+                    conf.getNettyCompressionDecoder(),
+                    handlerToUseExecutionGroup, executionGroup, ch);
+              }
               PipelineUtils.addLastWithExecutorCheck(
                   "clientOutboundByteCounter",
                   outboundByteCounter, handlerToUseExecutionGroup,
                   executionGroup, ch);
-
+              if (conf.doCompression()) {
+                PipelineUtils.addLastWithExecutorCheck("compressionEncoder",
+                    conf.getNettyCompressionEncoder(),
+                    handlerToUseExecutionGroup, executionGroup, ch);
+              }
               // The following pipeline component is needed to decode the
               // server's SASL tokens. It is replaced with a
               // FixedLengthFrameDecoder (same as used with the
@@ -303,10 +312,20 @@ public class NettyClient {
               PipelineUtils.addLastWithExecutorCheck("clientInboundByteCounter",
                   inboundByteCounter, handlerToUseExecutionGroup,
                   executionGroup, ch);
+              if (conf.doCompression()) {
+                PipelineUtils.addLastWithExecutorCheck("compressionDecoder",
+                    conf.getNettyCompressionDecoder(),
+                    handlerToUseExecutionGroup, executionGroup, ch);
+              }
               PipelineUtils.addLastWithExecutorCheck(
                   "clientOutboundByteCounter",
                   outboundByteCounter, handlerToUseExecutionGroup,
                   executionGroup, ch);
+              if (conf.doCompression()) {
+                PipelineUtils.addLastWithExecutorCheck("compressionEncoder",
+                    conf.getNettyCompressionEncoder(),
+                    handlerToUseExecutionGroup, executionGroup, ch);
+              }
               PipelineUtils.addLastWithExecutorCheck(
                   "fixed-length-frame-decoder",
                   new FixedLengthFrameDecoder(

http://git-wip-us.apache.org/repos/asf/giraph/blob/4223ccc0/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
index 14d4ea8..8162857 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
@@ -240,8 +240,18 @@ public class NettyServer {
           // configuration except for the presence of the Authorize component.
           PipelineUtils.addLastWithExecutorCheck("serverInboundByteCounter",
               inByteCounter, handlerToUseExecutionGroup, executionGroup, ch);
+          if (conf.doCompression()) {
+            PipelineUtils.addLastWithExecutorCheck("compressionDecoder",
+                conf.getNettyCompressionDecoder(),
+                handlerToUseExecutionGroup, executionGroup, ch);
+          }
           PipelineUtils.addLastWithExecutorCheck("serverOutboundByteCounter",
               outByteCounter, handlerToUseExecutionGroup, executionGroup, ch);
+          if (conf.doCompression()) {
+            PipelineUtils.addLastWithExecutorCheck("compressionEncoder",
+                conf.getNettyCompressionEncoder(),
+                handlerToUseExecutionGroup, executionGroup, ch);
+          }
           PipelineUtils.addLastWithExecutorCheck("requestFrameDecoder",
               new LengthFieldBasedFrameDecoder(1024 * 1024 * 1024, 0, 4, 0, 4),
               handlerToUseExecutionGroup, executionGroup, ch);
@@ -280,8 +290,18 @@ public class NettyServer {
               });
           PipelineUtils.addLastWithExecutorCheck("serverInboundByteCounter",
               inByteCounter, handlerToUseExecutionGroup, executionGroup, ch);
+          if (conf.doCompression()) {
+            PipelineUtils.addLastWithExecutorCheck("compressionDecoder",
+                conf.getNettyCompressionDecoder(),
+                handlerToUseExecutionGroup, executionGroup, ch);
+          }
           PipelineUtils.addLastWithExecutorCheck("serverOutboundByteCounter",
               outByteCounter, handlerToUseExecutionGroup, executionGroup, ch);
+          if (conf.doCompression()) {
+            PipelineUtils.addLastWithExecutorCheck("compressionEncoder",
+                conf.getNettyCompressionEncoder(),
+                handlerToUseExecutionGroup, executionGroup, ch);
+          }
           PipelineUtils.addLastWithExecutorCheck("requestFrameDecoder",
               new LengthFieldBasedFrameDecoder(1024 * 1024 * 1024, 0, 4, 0, 4),
               handlerToUseExecutionGroup, executionGroup, ch);

http://git-wip-us.apache.org/repos/asf/giraph/blob/4223ccc0/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 7d7ceb2..1879a25 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
@@ -613,6 +613,11 @@ public interface GiraphConstants {
       new BooleanConfOption("giraph.nettySimulateFirstResponseFailed", false,
           "Netty simulate a first response failed");
 
+  /** Netty - set which compression to use */
+  StrConfOption NETTY_COMPRESSION_ALGORITHM =
+      new StrConfOption("giraph.nettyCompressionAlgorithm", "",
+          "Which compression algorithm to use in netty");
+
   /** Max resolve address attempts */
   IntConfOption MAX_RESOLVE_ADDRESS_ATTEMPTS =
       new IntConfOption("giraph.maxResolveAddressAttempts", 5,

http://git-wip-us.apache.org/repos/asf/giraph/blob/4223ccc0/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 3d7b3db..3121fa8 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
@@ -19,6 +19,12 @@
 package org.apache.giraph.conf;
 
 import com.google.common.base.Preconditions;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.codec.MessageToByteEncoder;
+import io.netty.handler.codec.compression.JdkZlibDecoder;
+import io.netty.handler.codec.compression.JdkZlibEncoder;
+import io.netty.handler.codec.compression.SnappyFramedDecoder;
+import io.netty.handler.codec.compression.SnappyFramedEncoder;
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.edge.Edge;
@@ -1220,4 +1226,52 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
     }
     classes.setMessageCombiner(superstepClasses.getMessageCombinerClass());
   }
+
+  /**
+   * Has the user enabled compression in netty client & server
+   *
+   * @return true if ok to do compression of netty requests
+   */
+  public boolean doCompression() {
+    switch (GiraphConstants.NETTY_COMPRESSION_ALGORITHM.get(this)) {
+    case "SNAPPY":
+      return true;
+    case "INFLATE":
+      return true;
+    default:
+      return false;
+    }
+  }
+
+  /**
+   * Get encoder for message compression in netty
+   *
+   * @return message to byte encoder
+   */
+  public MessageToByteEncoder getNettyCompressionEncoder() {
+    switch (GiraphConstants.NETTY_COMPRESSION_ALGORITHM.get(this)) {
+    case "SNAPPY":
+      return new SnappyFramedEncoder();
+    case "INFLATE":
+      return new JdkZlibEncoder();
+    default:
+      return null;
+    }
+  }
+
+  /**
+   * Get decoder for message decompression in netty
+   *
+   * @return byte to message decoder
+   */
+  public ByteToMessageDecoder getNettyCompressionDecoder() {
+    switch (GiraphConstants.NETTY_COMPRESSION_ALGORITHM.get(this)) {
+    case "SNAPPY":
+      return new SnappyFramedDecoder(true);
+    case "INFLATE":
+      return new JdkZlibDecoder();
+    default:
+      return null;
+    }
+  }
 }


[38/47] Reduce/broadcast API

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendReducedToMasterRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendReducedToMasterRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendReducedToMasterRequest.java
new file mode 100644
index 0000000..7171f04
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendReducedToMasterRequest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.comm.requests;
+
+import java.io.IOException;
+
+import org.apache.giraph.master.MasterAggregatorHandler;
+
+/**
+ * Request to send final aggregated values from worker which owns
+ * aggregators to the master
+ */
+public class SendReducedToMasterRequest extends ByteArrayRequest
+    implements MasterRequest {
+
+  /**
+   * Constructor
+   *
+   * @param data Serialized aggregator data
+   */
+  public SendReducedToMasterRequest(byte[] data) {
+    super(data);
+  }
+
+  /**
+   * Constructor used for reflection only
+   */
+  public SendReducedToMasterRequest() {
+  }
+
+  @Override
+  public void doRequest(MasterAggregatorHandler aggregatorHandler) {
+    try {
+      aggregatorHandler.acceptReducedValues(getDataInput());
+    } catch (IOException e) {
+      throw new IllegalStateException("doRequest: " +
+          "IOException occurred while processing request", e);
+    }
+  }
+
+  @Override
+  public RequestType getType() {
+    return RequestType.SEND_AGGREGATORS_TO_MASTER_REQUEST;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerAggregatorsRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerAggregatorsRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerAggregatorsRequest.java
index 00a0c26..2f76e6e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerAggregatorsRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerAggregatorsRequest.java
@@ -18,15 +18,15 @@
 
 package org.apache.giraph.comm.requests;
 
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.ServerData;
-import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.comm.aggregators.OwnerAggregatorServerData;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 
-import java.io.DataInput;
-import java.io.IOException;
-
 /**
  * Request to send partial aggregated values for current superstep (values
  * which were computed by one worker's vertices)
@@ -56,20 +56,23 @@ public class SendWorkerAggregatorsRequest extends
     OwnerAggregatorServerData aggregatorData =
         serverData.getOwnerAggregatorData();
     try {
-      int numAggregators = input.readInt();
-      for (int i = 0; i < numAggregators; i++) {
-        String aggregatorName = input.readUTF();
-        if (aggregatorName.equals(
-            AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)) {
-          LongWritable count = new LongWritable(0);
-          count.readFields(input);
-          aggregatorData.receivedRequestCountFromWorker(count.get(),
+      int num = input.readInt();
+      for (int i = 0; i < num; i++) {
+        String name = input.readUTF();
+        GlobalCommType type = GlobalCommType.values()[input.readByte()];
+        if (type == GlobalCommType.SPECIAL_COUNT) {
+          LongWritable value = new LongWritable();
+          value.readFields(input);
+          aggregatorData.receivedRequestCountFromWorker(
+              value.get(),
               getSenderTaskId());
+        } else if (type == GlobalCommType.REDUCED_VALUE) {
+          Writable value = aggregatorData.createInitialValue(name);
+          value.readFields(input);
+          aggregatorData.reduce(name, value);
         } else {
-          Writable aggregatedValue =
-              aggregatorData.createAggregatorInitialValue(aggregatorName);
-          aggregatedValue.readFields(input);
-          aggregatorData.aggregate(aggregatorName, aggregatedValue);
+          throw new IllegalStateException(
+              "SendWorkerAggregatorsRequest received " + type);
         }
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/graph/AbstractComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/AbstractComputation.java b/giraph-core/src/main/java/org/apache/giraph/graph/AbstractComputation.java
index e7c3084..1ea6603 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/AbstractComputation.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/AbstractComputation.java
@@ -18,20 +18,20 @@
 
 package org.apache.giraph.graph;
 
+import java.io.IOException;
+import java.util.Iterator;
+
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.OutEdges;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerAggregatorDelegator;
 import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
 
-import java.io.IOException;
-import java.util.Iterator;
-
 /**
  * See {@link Computation} for explanation of the interface.
  *
@@ -52,7 +52,7 @@ import java.util.Iterator;
 public abstract class AbstractComputation<I extends WritableComparable,
     V extends Writable, E extends Writable, M1 extends Writable,
     M2 extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
+    extends WorkerAggregatorDelegator<I, V, E>
     implements Computation<I, V, E, M1, M2> {
   /** Logger */
   private static final Logger LOG = Logger.getLogger(AbstractComputation.class);
@@ -63,8 +63,6 @@ public abstract class AbstractComputation<I extends WritableComparable,
   private WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor;
   /** Graph-wide BSP Mapper for this Computation */
   private GraphTaskManager<I, V, E> graphTaskManager;
-  /** Worker aggregator usage */
-  private WorkerAggregatorUsage workerAggregatorUsage;
   /** Worker context */
   private WorkerContext workerContext;
 
@@ -76,6 +74,7 @@ public abstract class AbstractComputation<I extends WritableComparable,
    *                 superstep.  Each message is only guaranteed to have
    *                 a life expectancy as long as next() is not called.
    */
+  @Override
   public abstract void compute(Vertex<I, V, E> vertex,
       Iterable<M1> messages) throws IOException;
 
@@ -103,7 +102,7 @@ public abstract class AbstractComputation<I extends WritableComparable,
    * @param graphState Graph state
    * @param workerClientRequestProcessor Processor for handling requests
    * @param graphTaskManager Graph-wide BSP Mapper for this Vertex
-   * @param workerAggregatorUsage Worker aggregator usage
+   * @param workerGlobalCommUsage Worker global communication usage
    * @param workerContext Worker context
    */
   @Override
@@ -111,12 +110,12 @@ public abstract class AbstractComputation<I extends WritableComparable,
       GraphState graphState,
       WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor,
       GraphTaskManager<I, V, E> graphTaskManager,
-      WorkerAggregatorUsage workerAggregatorUsage,
+      WorkerGlobalCommUsage workerGlobalCommUsage,
       WorkerContext workerContext) {
     this.graphState = graphState;
     this.workerClientRequestProcessor = workerClientRequestProcessor;
     this.graphTaskManager = graphTaskManager;
-    this.workerAggregatorUsage = workerAggregatorUsage;
+    this.setWorkerGlobalCommUsage(workerGlobalCommUsage);
     this.workerContext = workerContext;
   }
 
@@ -274,14 +273,4 @@ public abstract class AbstractComputation<I extends WritableComparable,
   public <W extends WorkerContext> W getWorkerContext() {
     return (W) workerContext;
   }
-
-  @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    workerAggregatorUsage.aggregate(name, value);
-  }
-
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return workerAggregatorUsage.<A>getAggregatedValue(name);
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java b/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
index 7a7b40f..d310da9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
@@ -17,6 +17,9 @@
  */
 package org.apache.giraph.graph;
 
+import java.io.IOException;
+import java.util.Iterator;
+
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.TypesHolder;
@@ -24,13 +27,11 @@ import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.worker.WorkerAggregatorUsage;
 import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 
-import java.io.IOException;
-import java.util.Iterator;
-
 /**
  * Interface for an application for computation.
  *
@@ -55,7 +56,7 @@ public interface Computation<I extends WritableComparable,
     M2 extends Writable>
     extends TypesHolder<I, V, E, M1, M2>,
     ImmutableClassesGiraphConfigurable<I, V, E>,
-    WorkerAggregatorUsage {
+    WorkerGlobalCommUsage, WorkerAggregatorUsage {
   /**
    * Must be defined by user to do computation on a single Vertex.
    *
@@ -87,13 +88,13 @@ public interface Computation<I extends WritableComparable,
    * @param graphState Graph state
    * @param workerClientRequestProcessor Processor for handling requests
    * @param graphTaskManager Graph-wide BSP Mapper for this Vertex
-   * @param workerAggregatorUsage Worker aggregator usage
+   * @param workerGlobalCommUsage Worker global communication usage
    * @param workerContext Worker context
    */
   void initialize(GraphState graphState,
       WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor,
       GraphTaskManager<I, V, E> graphTaskManager,
-      WorkerAggregatorUsage workerAggregatorUsage, WorkerContext workerContext);
+      WorkerGlobalCommUsage workerGlobalCommUsage, WorkerContext workerContext);
 
   /**
    * Retrieves the current superstep.

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java b/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
index d9c4302..33f2255 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
@@ -36,7 +36,7 @@ import org.apache.giraph.utils.TimedLogger;
 import org.apache.giraph.utils.Trimmable;
 import org.apache.giraph.worker.WorkerContext;
 import org.apache.giraph.worker.WorkerProgress;
-import org.apache.giraph.worker.WorkerThreadAggregatorUsage;
+import org.apache.giraph.worker.WorkerThreadGlobalCommUsage;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -135,7 +135,7 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
     WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor =
         new NettyWorkerClientRequestProcessor<I, V, E>(
             context, configuration, serviceWorker);
-    WorkerThreadAggregatorUsage aggregatorUsage =
+    WorkerThreadGlobalCommUsage aggregatorUsage =
         serviceWorker.getAggregatorHandler().newThreadAggregatorUsage();
     WorkerContext workerContext = serviceWorker.getWorkerContext();
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 ba5d2fa..eb9fad3 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
@@ -18,6 +18,19 @@
 
 package org.apache.giraph.graph;
 
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
@@ -26,9 +39,7 @@ import org.apache.giraph.comm.messages.MessageStore;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.job.JobProgressTracker;
-import org.apache.giraph.scripting.ScriptLoader;
 import org.apache.giraph.master.BspServiceMaster;
-import org.apache.giraph.master.MasterAggregatorUsage;
 import org.apache.giraph.master.MasterThread;
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.metrics.GiraphMetricsRegistry;
@@ -40,6 +51,7 @@ import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionOwner;
 import org.apache.giraph.partition.PartitionStats;
 import org.apache.giraph.partition.PartitionStore;
+import org.apache.giraph.scripting.ScriptLoader;
 import org.apache.giraph.utils.CallableFactory;
 import org.apache.giraph.utils.MemoryUtils;
 import org.apache.giraph.utils.ProgressableUtils;
@@ -60,19 +72,6 @@ import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.log4j.PatternLayout;
 
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-
 /**
  * The Giraph-specific business logic for a single BSP
  * compute node in whatever underlying type of cluster
@@ -149,7 +148,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   /** Timer for WorkerContext#preSuperstep() */
   private GiraphTimer wcPreSuperstepTimer;
   /** The Hadoop Mapper#Context for this job */
-  private Mapper<?, ?, ?, ?>.Context context;
+  private final Mapper<?, ?, ?, ?>.Context context;
   /** is this GraphTaskManager the master? */
   private boolean isMaster;
 
@@ -497,15 +496,6 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     return graphFunctions;
   }
 
-  /**
-   * Get master aggregator usage, a subset of the functionality
-   *
-   * @return Master aggregator usage interface
-   */
-  public final MasterAggregatorUsage getMasterAggregatorUsage() {
-    return serviceMaster.getAggregatorHandler();
-  }
-
   public final WorkerContext getWorkerContext() {
     return serviceWorker.getWorkerContext();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java b/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
index 1bc48e3..83a0369 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
@@ -19,9 +19,9 @@
 package org.apache.giraph.io;
 
 import java.io.IOException;
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+
 import org.apache.giraph.edge.Edge;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerAggregatorDelegator;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -36,11 +36,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  */
 @SuppressWarnings("rawtypes")
 public abstract class EdgeReader<I extends WritableComparable,
-    E extends Writable> extends DefaultImmutableClassesGiraphConfigurable<
-        I, Writable, E> implements WorkerAggregatorUsage {
-
-  /** Aggregator usage for edge reader */
-  private WorkerAggregatorUsage workerAggregatorUsage;
+    E extends Writable> extends WorkerAggregatorDelegator<
+        I, Writable, E> {
 
   /**
    * Use the input split and context to setup reading the edges.
@@ -56,21 +53,6 @@ public abstract class EdgeReader<I extends WritableComparable,
     throws IOException, InterruptedException;
 
   /**
-   * Set aggregator usage. It provides the functionality
-   * of aggregation operation in reading an edge.
-   * It is invoked just after initialization.
-   * E.g.,
-   * edgeReader.initialize(inputSplit, context);
-   * edgeReader.setAggregator(aggregatorUsage);
-   * This method is only for use by the infrastructure.
-   *
-   * @param agg aggregator usage for edge reader
-   */
-  public void setWorkerAggregatorUse(WorkerAggregatorUsage agg) {
-    workerAggregatorUsage = agg;
-  }
-
-  /**
    * Read the next edge.
    *
    * @return false iff there are no more edges
@@ -117,14 +99,4 @@ public abstract class EdgeReader<I extends WritableComparable,
    * @throws InterruptedException
    */
   public abstract float getProgress() throws IOException, InterruptedException;
-
-  @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    workerAggregatorUsage.aggregate(name, value);
-  }
-
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return workerAggregatorUsage.<A>getAggregatedValue(name);
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/io/MappingReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/MappingReader.java b/giraph-core/src/main/java/org/apache/giraph/io/MappingReader.java
index b7ce97c..7c71585 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/MappingReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/MappingReader.java
@@ -18,16 +18,15 @@
 
 package org.apache.giraph.io;
 
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import java.io.IOException;
+
 import org.apache.giraph.mapping.MappingEntry;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerAggregatorDelegator;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import java.io.IOException;
-
 /**
  * Will read the mapping from an input split.
  *
@@ -38,12 +37,7 @@ import java.io.IOException;
  */
 public abstract class MappingReader<I extends WritableComparable,
     V extends Writable, E extends Writable, B extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
-    implements WorkerAggregatorUsage {
-
-  /** Aggregator usage for vertex reader */
-  private WorkerAggregatorUsage workerAggregatorUsage;
-
+    extends WorkerAggregatorDelegator<I, V, E> {
   /**
    * Use the input split and context to setup reading the vertices.
    * Guaranteed to be called prior to any other function.
@@ -57,22 +51,6 @@ public abstract class MappingReader<I extends WritableComparable,
     TaskAttemptContext context)
     throws IOException, InterruptedException;
 
-
-  /**
-   * Set aggregator usage. It provides the functionality
-   * of aggregation operation in reading a vertex.
-   * It is invoked just after initialization.
-   * E.g.,
-   * vertexReader.initialize(inputSplit, context);
-   * vertexReader.setAggregator(aggregatorUsage);
-   * This method is only for use by the infrastructure.
-   *
-   * @param agg aggregator usage for vertex reader
-   */
-  public void setWorkerAggregatorUse(WorkerAggregatorUsage agg) {
-    workerAggregatorUsage = agg;
-  }
-
   /**
    *
    * @return false iff there are no more vertices
@@ -111,14 +89,4 @@ public abstract class MappingReader<I extends WritableComparable,
    * @throws InterruptedException
    */
   public abstract float getProgress() throws IOException, InterruptedException;
-
-  @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    workerAggregatorUsage.aggregate(name, value);
-  }
-
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return workerAggregatorUsage.getAggregatedValue(name);
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
index 94a4083..64ec800 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
@@ -18,16 +18,15 @@
 
 package org.apache.giraph.io;
 
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import java.io.IOException;
+
 import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerAggregatorDelegator;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import java.io.IOException;
-
 /**
  * Analogous to Hadoop's RecordReader for vertices.  Will read the
  * vertices from an input split.
@@ -39,11 +38,7 @@ import java.io.IOException;
 @SuppressWarnings("rawtypes")
 public abstract class VertexReader<I extends WritableComparable,
     V extends Writable, E extends Writable> extends
-    DefaultImmutableClassesGiraphConfigurable<I, V, E>
-    implements WorkerAggregatorUsage {
-  /** Aggregator usage for vertex reader */
-  private WorkerAggregatorUsage workerAggregatorUsage;
-
+    WorkerAggregatorDelegator<I, V, E> {
   /**
    * Use the input split and context to setup reading the vertices.
    * Guaranteed to be called prior to any other function.
@@ -58,21 +53,6 @@ public abstract class VertexReader<I extends WritableComparable,
     throws IOException, InterruptedException;
 
   /**
-   * Set aggregator usage. It provides the functionality
-   * of aggregation operation in reading a vertex.
-   * It is invoked just after initialization.
-   * E.g.,
-   * vertexReader.initialize(inputSplit, context);
-   * vertexReader.setAggregator(aggregatorUsage);
-   * This method is only for use by the infrastructure.
-   *
-   * @param agg aggregator usage for vertex reader
-   */
-  public void setWorkerAggregatorUse(WorkerAggregatorUsage agg) {
-    workerAggregatorUsage = agg;
-  }
-
-  /**
    *
    * @return false iff there are no more vertices
    * @throws IOException
@@ -108,14 +88,4 @@ public abstract class VertexReader<I extends WritableComparable,
    * @throws InterruptedException
    */
   public abstract float getProgress() throws IOException, InterruptedException;
-
-  @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    workerAggregatorUsage.aggregate(name, value);
-  }
-
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return workerAggregatorUsage.<A>getAggregatedValue(name);
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
index 9b5e8c6..05dd5bc 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
@@ -18,18 +18,18 @@
 
 package org.apache.giraph.io.internal;
 
+import java.io.IOException;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.io.EdgeReader;
 import org.apache.giraph.job.HadoopUtils;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import java.io.IOException;
-
 /**
  * For internal use only.
  *
@@ -72,9 +72,10 @@ public class WrappedEdgeReader<I extends WritableComparable,
   }
 
   @Override
-  public void setWorkerAggregatorUse(WorkerAggregatorUsage agg) {
-    // Set aggregator usage for edge reader
-    baseEdgeReader.setWorkerAggregatorUse(agg);
+  public void setWorkerGlobalCommUsage(WorkerGlobalCommUsage usage) {
+    super.setWorkerGlobalCommUsage(usage);
+    // Set global communication usage for edge reader
+    baseEdgeReader.setWorkerGlobalCommUsage(usage);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedMappingReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedMappingReader.java b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedMappingReader.java
index 7d1c4c9..659776b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedMappingReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedMappingReader.java
@@ -18,18 +18,18 @@
 
 package org.apache.giraph.io.internal;
 
+import java.io.IOException;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.io.MappingReader;
 import org.apache.giraph.job.HadoopUtils;
 import org.apache.giraph.mapping.MappingEntry;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import java.io.IOException;
-
 /**
  * For internal use only.
  *
@@ -74,11 +74,11 @@ public class WrappedMappingReader<I extends WritableComparable,
         HadoopUtils.makeTaskAttemptContext(getConf(), context));
   }
 
-
   @Override
-  public void setWorkerAggregatorUse(WorkerAggregatorUsage agg) {
-    // Set aggregator usage for vertex reader
-    baseMappingReader.setWorkerAggregatorUse(agg);
+  public void setWorkerGlobalCommUsage(WorkerGlobalCommUsage usage) {
+    super.setWorkerGlobalCommUsage(usage);
+    // Set global communication usage for edge reader
+    baseMappingReader.setWorkerGlobalCommUsage(usage);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
index 8e25602..8c23cba 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
@@ -18,18 +18,18 @@
 
 package org.apache.giraph.io.internal;
 
+import java.io.IOException;
+
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.job.HadoopUtils;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import java.io.IOException;
-
 /**
  * For internal use only.
  *
@@ -73,9 +73,10 @@ public class WrappedVertexReader<I extends WritableComparable,
   }
 
   @Override
-  public void setWorkerAggregatorUse(WorkerAggregatorUsage agg) {
+  public void setWorkerGlobalCommUsage(WorkerGlobalCommUsage usage) {
+    super.setWorkerGlobalCommUsage(usage);
     // Set aggregator usage for vertex reader
-    baseVertexReader.setWorkerAggregatorUse(agg);
+    baseVertexReader.setWorkerGlobalCommUsage(usage);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
new file mode 100644
index 0000000..1673f6d
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
@@ -0,0 +1,92 @@
+/*
+ * 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.master;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.reducers.OnSameReduceOperation;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Translates aggregation operation to reduce operations.
+ *
+ * @param <A> Aggregation object type
+ */
+public class AggregatorReduceOperation<A extends Writable>
+    extends OnSameReduceOperation<A> {
+  /** Aggregator factory */
+  private WritableFactory<? extends Aggregator<A>> aggregatorFactory;
+  /** Aggregator */
+  private Aggregator<A> aggregator;
+
+  /** Constructor */
+  public AggregatorReduceOperation() {
+  }
+
+  /**
+   * Constructor
+   * @param aggregatorFactory Aggregator factory
+   */
+  public AggregatorReduceOperation(
+      WritableFactory<? extends Aggregator<A>> aggregatorFactory) {
+    this.aggregatorFactory = aggregatorFactory;
+    this.aggregator = aggregatorFactory.create();
+    this.aggregator.setAggregatedValue(null);
+  }
+
+  @Override
+  public A createInitialValue() {
+    return aggregator.createInitialValue();
+  }
+
+  /**
+   * Creates copy of this object
+   * @return copy
+   */
+  public AggregatorReduceOperation<A> createCopy() {
+    return new AggregatorReduceOperation<>(aggregatorFactory);
+  }
+
+  @Override
+  public synchronized void reduceSingle(A curValue, A valueToReduce) {
+    aggregator.setAggregatedValue(curValue);
+    aggregator.aggregate(valueToReduce);
+    if (curValue != aggregator.getAggregatedValue()) {
+      throw new IllegalStateException(
+          "Aggregator " + aggregator + " aggregates by creating new value");
+    }
+    aggregator.setAggregatedValue(null);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeWritableObject(aggregatorFactory, out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    aggregatorFactory = WritableUtils.readWritableObject(in, null);
+    aggregator = aggregatorFactory.create();
+    this.aggregator.setAggregatedValue(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
new file mode 100644
index 0000000..7492fc7
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
@@ -0,0 +1,240 @@
+/*
+ * 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.master;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map.Entry;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.aggregators.ClassAggregatorFactory;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.utils.WritableFactory;
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Class that translates aggregator handling on the master to
+ * reduce and broadcast operations supported by the MasterAggregatorHandler.
+ */
+public class AggregatorToGlobalCommTranslation
+    extends DefaultImmutableClassesGiraphConfigurable
+    implements MasterAggregatorUsage, Writable {
+  /** Class providing reduce and broadcast interface to use */
+  private final MasterGlobalCommUsage globalComm;
+  /** List of registered aggregators */
+  private final HashMap<String, AggregatorWrapper<Writable>>
+  registeredAggregators = new HashMap<>();
+
+  /**
+   * Constructor
+   * @param globalComm Global communication interface
+   */
+  public AggregatorToGlobalCommTranslation(MasterGlobalCommUsage globalComm) {
+    this.globalComm = globalComm;
+  }
+
+  @Override
+  public <A extends Writable> A getAggregatedValue(String name) {
+    return globalComm.getReduced(name);
+  }
+
+  @Override
+  public <A extends Writable> void setAggregatedValue(String name, A value) {
+    AggregatorWrapper<Writable> aggregator = registeredAggregators.get(name);
+    aggregator.setCurrentValue(value);
+  }
+
+  /**
+   * Called after master compute, to do aggregator->reduce/broadcast
+   * translation
+   */
+  public void postMasterCompute() {
+    // broadcast what master set, or if it didn't broadcast reduced value
+    // register reduce with the same value
+    for (Entry<String, AggregatorWrapper<Writable>> entry :
+        registeredAggregators.entrySet()) {
+      Writable value = entry.getValue().currentValue != null ?
+          entry.getValue().getCurrentValue() :
+            globalComm.getReduced(entry.getKey());
+      if (value == null) {
+        value = entry.getValue().getReduceOp().createInitialValue();
+      }
+
+      globalComm.broadcast(entry.getKey(), value);
+      // Always register clean instance of reduceOp, not to conflict with
+      // reduceOp from previous superstep.
+      AggregatorReduceOperation<Writable> cleanReduceOp =
+          entry.getValue().createReduceOp();
+      if (entry.getValue().isPersistent()) {
+        globalComm.registerReduce(
+            entry.getKey(), cleanReduceOp, value);
+      } else {
+        globalComm.registerReduce(
+            entry.getKey(), cleanReduceOp);
+      }
+      entry.getValue().setCurrentValue(null);
+    }
+  }
+
+  @Override
+  public <A extends Writable> boolean registerAggregator(String name,
+      Class<? extends Aggregator<A>> aggregatorClass) throws
+      InstantiationException, IllegalAccessException {
+    ClassAggregatorFactory<A> aggregatorFactory =
+        new ClassAggregatorFactory<A>(aggregatorClass);
+    return registerAggregator(name, aggregatorFactory, false) != null;
+  }
+
+  @Override
+  public <A extends Writable> boolean registerAggregator(String name,
+      WritableFactory<? extends Aggregator<A>> aggregator) throws
+      InstantiationException, IllegalAccessException {
+    return registerAggregator(name, aggregator, false) != null;
+  }
+
+  @Override
+  public <A extends Writable> boolean registerPersistentAggregator(String name,
+      Class<? extends Aggregator<A>> aggregatorClass) throws
+      InstantiationException, IllegalAccessException {
+    ClassAggregatorFactory<A> aggregatorFactory =
+        new ClassAggregatorFactory<A>(aggregatorClass);
+    return registerAggregator(name, aggregatorFactory, true) != null;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(registeredAggregators.size());
+    for (Entry<String, AggregatorWrapper<Writable>> entry :
+        registeredAggregators.entrySet()) {
+      out.writeUTF(entry.getKey());
+      entry.getValue().write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    registeredAggregators.clear();
+    int numAggregators = in.readInt();
+    for (int i = 0; i < numAggregators; i++) {
+      String name = in.readUTF();
+      AggregatorWrapper<Writable> agg = new AggregatorWrapper<>();
+      agg.readFields(in);
+      registeredAggregators.put(name, agg);
+    }
+  }
+
+  /**
+   * Helper function for registering aggregators.
+   *
+   * @param name              Name of the aggregator
+   * @param aggregatorFactory Aggregator factory
+   * @param persistent        Whether aggregator is persistent or not
+   * @param <A>               Aggregated value type
+   * @return Newly registered aggregator or aggregator which was previously
+   *         created with selected name, if any
+   */
+  private <A extends Writable> AggregatorWrapper<A> registerAggregator
+  (String name, WritableFactory<? extends Aggregator<A>> aggregatorFactory,
+      boolean persistent) throws InstantiationException,
+      IllegalAccessException {
+    AggregatorWrapper<A> aggregatorWrapper =
+        (AggregatorWrapper<A>) registeredAggregators.get(name);
+    if (aggregatorWrapper == null) {
+      aggregatorWrapper =
+          new AggregatorWrapper<A>(aggregatorFactory, persistent);
+      registeredAggregators.put(
+          name, (AggregatorWrapper<Writable>) aggregatorWrapper);
+    }
+    return aggregatorWrapper;
+  }
+
+  /**
+   * Object holding all needed data related to single Aggregator
+   * @param <A> Aggregated value type
+   */
+  private static class AggregatorWrapper<A extends Writable>
+      implements Writable {
+    /** False iff aggregator should be reset at the end of each super step */
+    private boolean persistent;
+    /** Translation of aggregator to reduce operations */
+    private AggregatorReduceOperation<A> reduceOp;
+    /** Current value, set by master manually */
+    private A currentValue;
+
+    /** Constructor */
+    public AggregatorWrapper() {
+    }
+
+    /**
+     * Constructor
+     * @param aggregatorFactory Aggregator factory
+     * @param persistent Is persistent
+     */
+    public AggregatorWrapper(
+        WritableFactory<? extends Aggregator<A>> aggregatorFactory,
+        boolean persistent) {
+      this.persistent = persistent;
+      this.reduceOp = new AggregatorReduceOperation<>(aggregatorFactory);
+    }
+
+    public AggregatorReduceOperation<A> getReduceOp() {
+      return reduceOp;
+    }
+
+    /**
+     * Create a fresh instance of AggregatorReduceOperation
+     * @return fresh instance of AggregatorReduceOperation
+     */
+    public AggregatorReduceOperation<A> createReduceOp() {
+      return reduceOp.createCopy();
+    }
+
+    public A getCurrentValue() {
+      return currentValue;
+    }
+
+    public void setCurrentValue(A currentValue) {
+      this.currentValue = currentValue;
+    }
+
+    public boolean isPersistent() {
+      return persistent;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeBoolean(persistent);
+      reduceOp.write(out);
+
+      Preconditions.checkState(currentValue == null, "AggregatorWrapper " +
+          "shouldn't have value at the end of the superstep");
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      persistent = in.readBoolean();
+      reduceOp = new AggregatorReduceOperation<>();
+      reduceOp.readFields(in);
+      currentValue = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index efa5b87..ab1289d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -18,11 +18,39 @@
 
 package org.apache.giraph.master;
 
+import static org.apache.giraph.conf.GiraphConstants.INPUT_SPLIT_SAMPLE_PERCENT;
+import static org.apache.giraph.conf.GiraphConstants.KEEP_ZOOKEEPER_DATA;
+import static org.apache.giraph.conf.GiraphConstants.PARTITION_LONG_TAIL_MIN_PRINT;
+import static org.apache.giraph.conf.GiraphConstants.USE_INPUT_SPLIT_LOCALITY;
 import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
 import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import net.iharder.Base64;
+
 import org.apache.commons.io.FilenameUtils;
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.BspInputFormat;
+import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.bsp.CheckpointStatus;
 import org.apache.giraph.bsp.SuperstepState;
@@ -33,23 +61,17 @@ import org.apache.giraph.comm.netty.NettyMasterServer;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.counters.GiraphStats;
-import org.apache.giraph.graph.InputSplitPaths;
-import org.apache.giraph.graph.GlobalStats;
 import org.apache.giraph.graph.AddressesAndPartitionsWritable;
+import org.apache.giraph.graph.GlobalStats;
 import org.apache.giraph.graph.GraphFunctions;
-import org.apache.giraph.graph.InputSplitEvents;
-import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.graph.GraphState;
+import org.apache.giraph.graph.GraphTaskManager;
+import org.apache.giraph.graph.InputSplitEvents;
+import org.apache.giraph.graph.InputSplitPaths;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.GiraphInputFormat;
-import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.io.MappingInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
-import org.apache.giraph.partition.BasicPartitionOwner;
-import org.apache.giraph.partition.MasterGraphPartitioner;
-import org.apache.giraph.partition.PartitionOwner;
-import org.apache.giraph.partition.PartitionStats;
-import org.apache.giraph.partition.PartitionUtils;
 import org.apache.giraph.metrics.AggregatedMetrics;
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.metrics.GiraphTimer;
@@ -57,13 +79,18 @@ import org.apache.giraph.metrics.GiraphTimerContext;
 import org.apache.giraph.metrics.ResetSuperstepMetricsObserver;
 import org.apache.giraph.metrics.SuperstepMetricsRegistry;
 import org.apache.giraph.metrics.WorkerSuperstepMetrics;
-import org.apache.giraph.utils.CheckpointingUtils;
-import org.apache.giraph.utils.JMapHistoDumper;
-import org.apache.giraph.utils.ReactiveJMapHistoDumper;
-import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.giraph.partition.BasicPartitionOwner;
+import org.apache.giraph.partition.MasterGraphPartitioner;
+import org.apache.giraph.partition.PartitionOwner;
+import org.apache.giraph.partition.PartitionStats;
+import org.apache.giraph.partition.PartitionUtils;
 import org.apache.giraph.time.SystemTime;
 import org.apache.giraph.time.Time;
+import org.apache.giraph.utils.CheckpointingUtils;
+import org.apache.giraph.utils.JMapHistoDumper;
 import org.apache.giraph.utils.LogStacktraceCallable;
+import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.giraph.utils.ReactiveJMapHistoDumper;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.giraph.zk.BspEvent;
@@ -89,32 +116,6 @@ import org.json.JSONObject;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import net.iharder.Base64;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.giraph.conf.GiraphConstants.INPUT_SPLIT_SAMPLE_PERCENT;
-import static org.apache.giraph.conf.GiraphConstants.KEEP_ZOOKEEPER_DATA;
-import static org.apache.giraph.conf.GiraphConstants.PARTITION_LONG_TAIL_MIN_PRINT;
-import static org.apache.giraph.conf.GiraphConstants.USE_INPUT_SPLIT_LOCALITY;
 
 /**
  * ZooKeeper-based implementation of {@link CentralizedServiceMaster}.
@@ -167,8 +168,10 @@ public class BspServiceMaster<I extends WritableComparable,
   /** All the partition stats from the last superstep */
   private final List<PartitionStats> allPartitionStatsList =
       new ArrayList<PartitionStats>();
-  /** Handler for aggregators */
-  private MasterAggregatorHandler aggregatorHandler;
+  /** Handler for global communication */
+  private MasterAggregatorHandler globalCommHandler;
+  /** Handler for aggregators to reduce/broadcast translation */
+  private AggregatorToGlobalCommTranslation aggregatorTranslation;
   /** Master class */
   private MasterCompute masterCompute;
   /** IPC Client */
@@ -232,7 +235,7 @@ public class BspServiceMaster<I extends WritableComparable,
     this.checkpointStatus = CheckpointStatus.NONE;
 
     GiraphMetrics.get().addSuperstepResetObserver(this);
-    GiraphStats.init((Mapper.Context) context);
+    GiraphStats.init(context);
   }
 
   @Override
@@ -738,8 +741,13 @@ public class BspServiceMaster<I extends WritableComparable,
   }
 
   @Override
-  public MasterAggregatorHandler getAggregatorHandler() {
-    return aggregatorHandler;
+  public MasterAggregatorHandler getGlobalCommHandler() {
+    return globalCommHandler;
+  }
+
+  @Override
+  public AggregatorToGlobalCommTranslation getAggregatorTranslationHandler() {
+    return aggregatorTranslation;
   }
 
   @Override
@@ -811,7 +819,8 @@ public class BspServiceMaster<I extends WritableComparable,
     });
 
 
-    aggregatorHandler.readFields(finalizedStream);
+    globalCommHandler.readFields(finalizedStream);
+    aggregatorTranslation.readFields(finalizedStream);
     masterCompute.readFields(finalizedStream);
     finalizedStream.close();
 
@@ -883,9 +892,12 @@ public class BspServiceMaster<I extends WritableComparable,
         if (masterChildArr.get(0).equals(myBid)) {
           GiraphStats.getInstance().getCurrentMasterTaskPartition().
               setValue(getTaskPartition());
-          aggregatorHandler = new MasterAggregatorHandler(getConfiguration(),
-              getContext());
-          aggregatorHandler.initialize(this);
+          globalCommHandler = new MasterAggregatorHandler(
+              getConfiguration(), getContext());
+          aggregatorTranslation = new AggregatorToGlobalCommTranslation(
+              globalCommHandler);
+
+          globalCommHandler.initialize(this);
           masterCompute = getConfiguration().createMasterCompute();
           masterCompute.setMasterService(this);
 
@@ -1097,7 +1109,8 @@ public class BspServiceMaster<I extends WritableComparable,
     for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) {
       finalizedOutputStream.writeInt(chosenWorkerInfo.getTaskId());
     }
-    aggregatorHandler.write(finalizedOutputStream);
+    globalCommHandler.write(finalizedOutputStream);
+    aggregatorTranslation.write(finalizedOutputStream);
     masterCompute.write(finalizedOutputStream);
     finalizedOutputStream.close();
     lastCheckpointedSuperstep = superstep;
@@ -1502,7 +1515,8 @@ public class BspServiceMaster<I extends WritableComparable,
    */
   private void initializeAggregatorInputSuperstep()
     throws InterruptedException {
-    aggregatorHandler.prepareSuperstep(masterClient);
+    globalCommHandler.prepareSuperstep();
+
     prepareMasterCompute(getSuperstep());
     try {
       masterCompute.initialize();
@@ -1516,7 +1530,10 @@ public class BspServiceMaster<I extends WritableComparable,
       throw new RuntimeException(
         "initializeAggregatorInputSuperstep: Failed in access", e);
     }
-    aggregatorHandler.finishSuperstep(masterClient);
+    aggregatorTranslation.postMasterCompute();
+    globalCommHandler.finishSuperstep();
+
+    globalCommHandler.sendDataToOwners(masterClient);
   }
 
   /**
@@ -1579,18 +1596,18 @@ public class BspServiceMaster<I extends WritableComparable,
       }
     }
 
+    // We need to finalize aggregators from previous superstep
+    if (getSuperstep() >= 0) {
+      aggregatorTranslation.postMasterCompute();
+      globalCommHandler.finishSuperstep();
+    }
+
     masterClient.openConnections();
 
     GiraphStats.getInstance().
         getCurrentWorkers().setValue(chosenWorkerInfoList.size());
     assignPartitionOwners();
 
-    // We need to finalize aggregators from previous superstep (send them to
-    // worker owners) after new worker assignments
-    if (getSuperstep() >= 0) {
-      aggregatorHandler.finishSuperstep(masterClient);
-    }
-
     // Finalize the valid checkpoint file prefixes and possibly
     // the aggregators.
     if (checkpointStatus != CheckpointStatus.NONE) {
@@ -1616,6 +1633,11 @@ public class BspServiceMaster<I extends WritableComparable,
       }
     }
 
+    // We need to send aggregators to worker owners after new worker assignments
+    if (getSuperstep() >= 0) {
+      globalCommHandler.sendDataToOwners(masterClient);
+    }
+
     if (getSuperstep() == INPUT_SUPERSTEP) {
       // Initialize aggregators before coordinating
       initializeAggregatorInputSuperstep();
@@ -1645,7 +1667,7 @@ public class BspServiceMaster<I extends WritableComparable,
 
     // Collect aggregator values, then run the master.compute() and
     // finally save the aggregator values
-    aggregatorHandler.prepareSuperstep(masterClient);
+    globalCommHandler.prepareSuperstep();
     SuperstepClasses superstepClasses =
       prepareMasterCompute(getSuperstep() + 1);
     doMasterCompute();
@@ -1710,7 +1732,7 @@ public class BspServiceMaster<I extends WritableComparable,
     } else {
       superstepState = SuperstepState.THIS_SUPERSTEP_DONE;
     }
-    aggregatorHandler.writeAggregators(getSuperstep(), superstepState);
+    globalCommHandler.writeAggregators(getSuperstep(), superstepState);
 
     return superstepState;
   }
@@ -1935,7 +1957,7 @@ public class BspServiceMaster<I extends WritableComparable,
         failJob(new Exception("Checkpoint and halt requested. " +
             "Killing this job."));
       }
-      aggregatorHandler.close();
+      globalCommHandler.close();
       masterClient.closeConnections();
       masterServer.close();
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
index 2b0cdd6..5f7bd73 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
@@ -15,263 +15,224 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.giraph.master;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.AbstractMap;
 import java.util.Map;
+import java.util.Map.Entry;
 
-import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.aggregators.AggregatorWrapper;
 import org.apache.giraph.aggregators.AggregatorWriter;
-import org.apache.giraph.aggregators.ClassAggregatorFactory;
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.SuperstepState;
+import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.MasterClient;
-import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.utils.MasterLoggingAggregator;
-import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.reducers.Reducer;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
-/** Handler for aggregators on master */
-public class MasterAggregatorHandler implements MasterAggregatorUsage,
-    Writable {
+/** Handler for reduce/broadcast on the master */
+public class MasterAggregatorHandler
+    implements MasterGlobalCommUsage, Writable {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(MasterAggregatorHandler.class);
-  /**
-   * Map of aggregators.
-   * This map is used to store final aggregated values received from worker
-   * owners, and also to read and write values provided during master.compute.
-   */
-  private final Map<String, AggregatorWrapper<Writable>> aggregatorMap =
+
+  /** Map of reducers registered for the next worker computation */
+  private final Map<String, Reducer<Object, Writable>> reducerMap =
+      Maps.newHashMap();
+  /** Map of values to be sent to workers for next computation */
+  private final Map<String, Writable> broadcastMap =
       Maps.newHashMap();
-  /** Aggregator writer */
+  /** Values reduced from previous computation */
+  private final Map<String, Writable> reducedMap =
+      Maps.newHashMap();
+
+  /** Aggregator writer - for writing reduced values */
   private final AggregatorWriter aggregatorWriter;
   /** Progressable used to report progress */
   private final Progressable progressable;
-  /** Giraph configuration */
-  private final ImmutableClassesGiraphConfiguration<?, ?, ?> conf;
 
   /**
    * Constructor
    *
-   * @param conf         Giraph configuration
-   * @param progressable Progressable used for reporting progress
+   * @param conf Configuration
+   * @param progressable Progress reporter
    */
   public MasterAggregatorHandler(
       ImmutableClassesGiraphConfiguration<?, ?, ?> conf,
       Progressable progressable) {
-    this.conf = conf;
     this.progressable = progressable;
     aggregatorWriter = conf.createAggregatorWriter();
-    MasterLoggingAggregator.registerAggregator(this, conf);
   }
 
   @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    AggregatorWrapper<? extends Writable> aggregator = aggregatorMap.get(name);
-    if (aggregator == null) {
-      LOG.warn("getAggregatedValue: " +
-          AggregatorUtils.getUnregisteredAggregatorMessage(name,
-              aggregatorMap.size() != 0, conf));
-      return null;
-    } else {
-      return (A) aggregator.getPreviousAggregatedValue();
-    }
+  public final <S, R extends Writable> void registerReduce(
+      String name, ReduceOperation<S, R> reduceOp) {
+    registerReduce(name, reduceOp, reduceOp.createInitialValue());
   }
 
   @Override
-  public <A extends Writable> void setAggregatedValue(String name, A value) {
-    AggregatorWrapper<? extends Writable> aggregator = aggregatorMap.get(name);
-    if (aggregator == null) {
-      throw new IllegalStateException(
-          "setAggregatedValue: " +
-              AggregatorUtils.getUnregisteredAggregatorMessage(name,
-                  aggregatorMap.size() != 0, conf));
+  public <S, R extends Writable> void registerReduce(
+      String name, ReduceOperation<S, R> reduceOp,
+      R globalInitialValue) {
+    if (reducerMap.containsKey(name)) {
+      throw new IllegalArgumentException(
+          "Reducer with name " + name + " was already registered");
+    }
+    if (reduceOp == null) {
+      throw new IllegalArgumentException("null reduce cannot be registered");
     }
-    ((AggregatorWrapper<A>) aggregator).setCurrentAggregatedValue(value);
-  }
 
-  @Override
-  public <A extends Writable> boolean registerAggregator(String name,
-      Class<? extends Aggregator<A>> aggregatorClass) throws
-      InstantiationException, IllegalAccessException {
-    checkAggregatorName(name);
-    ClassAggregatorFactory<A> aggregatorFactory =
-        new ClassAggregatorFactory<A>(aggregatorClass, conf);
-    return registerAggregator(name, aggregatorFactory, false) != null;
+    Reducer<S, R> reducer = new Reducer<>(reduceOp, globalInitialValue);
+    reducerMap.put(name, (Reducer<Object, Writable>) reducer);
   }
 
   @Override
-  public <A extends Writable> boolean registerAggregator(String name,
-      WritableFactory<? extends Aggregator<A>> aggregator) throws
-      InstantiationException, IllegalAccessException {
-    checkAggregatorName(name);
-    return registerAggregator(name, aggregator, false) != null;
+  public <T extends Writable> T getReduced(String name) {
+    return (T) reducedMap.get(name);
   }
 
   @Override
-  public <A extends Writable> boolean registerPersistentAggregator(String name,
-      Class<? extends Aggregator<A>> aggregatorClass) throws
-      InstantiationException, IllegalAccessException {
-    checkAggregatorName(name);
-    ClassAggregatorFactory<A> aggregatorFactory =
-        new ClassAggregatorFactory<A>(aggregatorClass, conf);
-    return registerAggregator(name, aggregatorFactory, true) != null;
-  }
-
-  /**
-   * Make sure user doesn't use AggregatorUtils.SPECIAL_COUNT_AGGREGATOR as
-   * the name of aggregator. Throw an exception if he tries to use it.
-   *
-   * @param name Name of the aggregator to check.
-   */
-  private void checkAggregatorName(String name) {
-    if (name.equals(AggregatorUtils.SPECIAL_COUNT_AGGREGATOR)) {
-      throw new IllegalStateException("checkAggregatorName: " +
-          AggregatorUtils.SPECIAL_COUNT_AGGREGATOR +
-          " is not allowed for the name of aggregator");
+  public void broadcast(String name, Writable object) {
+    if (broadcastMap.containsKey(name)) {
+      throw new IllegalArgumentException(
+          "Value already broadcasted for name " + name);
     }
-  }
-
-  /**
-   * Helper function for registering aggregators.
-   *
-   * @param name              Name of the aggregator
-   * @param aggregatorFactory Aggregator factory
-   * @param persistent        Whether aggregator is persistent or not
-   * @param <A>               Aggregated value type
-   * @return Newly registered aggregator or aggregator which was previously
-   *         created with selected name, if any
-   */
-  private <A extends Writable> AggregatorWrapper<A> registerAggregator
-  (String name, WritableFactory<? extends Aggregator<A>> aggregatorFactory,
-      boolean persistent) throws InstantiationException,
-      IllegalAccessException {
-    AggregatorWrapper<A> aggregatorWrapper =
-        (AggregatorWrapper<A>) aggregatorMap.get(name);
-    if (aggregatorWrapper == null) {
-      aggregatorWrapper =
-          new AggregatorWrapper<A>(aggregatorFactory, persistent, conf);
-      aggregatorMap.put(name, (AggregatorWrapper<Writable>) aggregatorWrapper);
+    if (object == null) {
+      throw new IllegalArgumentException("null cannot be broadcasted");
     }
-    return aggregatorWrapper;
+
+    broadcastMap.put(name, object);
   }
 
-  /**
-   * Prepare aggregators for current superstep
-   *
-   * @param masterClient IPC client on master
-   */
-  public void prepareSuperstep(MasterClient masterClient) {
+  /** Prepare reduced values for current superstep's master compute */
+  public void prepareSuperstep() {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("prepareSuperstep: Start preparing aggregators");
+      LOG.debug("prepareSuperstep: Start preparing reducers");
     }
-    // prepare aggregators for master compute
-    for (AggregatorWrapper<Writable> aggregator : aggregatorMap.values()) {
-      if (aggregator.isPersistent()) {
-        aggregator.aggregateCurrent(aggregator.getPreviousAggregatedValue());
+
+    Preconditions.checkState(reducedMap.isEmpty(),
+        "reducedMap must be empty before start of the superstep");
+    Preconditions.checkState(broadcastMap.isEmpty(),
+        "broadcastMap must be empty before start of the superstep");
+
+    for (Entry<String, Reducer<Object, Writable>> entry :
+        reducerMap.entrySet()) {
+      Writable value = entry.getValue().getCurrentValue();
+      if (value == null) {
+        value = entry.getValue().createInitialValue();
       }
-      aggregator.setPreviousAggregatedValue(
-          aggregator.getCurrentAggregatedValue());
-      aggregator.resetCurrentAggregator();
-      progressable.progress();
+
+      reducedMap.put(entry.getKey(), value);
     }
-    MasterLoggingAggregator.logAggregatedValue(this, conf);
+
+    reducerMap.clear();
+
     if (LOG.isDebugEnabled()) {
       LOG.debug("prepareSuperstep: Aggregators prepared");
     }
   }
 
-  /**
-   * Finalize aggregators for current superstep and share them with workers
-   *
-   * @param masterClient IPC client on master
-   */
-  public void finishSuperstep(MasterClient masterClient) {
+  /** Finalize aggregators for current superstep */
+  public void finishSuperstep() {
     if (LOG.isDebugEnabled()) {
       LOG.debug("finishSuperstep: Start finishing aggregators");
     }
-    for (AggregatorWrapper<Writable> aggregator : aggregatorMap.values()) {
-      if (aggregator.isChanged()) {
-        // if master compute changed the value, use the one he chose
-        aggregator.setPreviousAggregatedValue(
-            aggregator.getCurrentAggregatedValue());
-        // reset aggregator for the next superstep
-        aggregator.resetCurrentAggregator();
-      }
-      progressable.progress();
+
+    reducedMap.clear();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("finishSuperstep: Aggregators finished");
     }
+  }
 
-    // send aggregators to their owners
-    // TODO: if aggregator owner and it's value didn't change,
-    //       we don't need to resend it
+  /**
+   * Send data to workers (through owner workers)
+   *
+   * @param masterClient IPC client on master
+   */
+  public void sendDataToOwners(MasterClient masterClient) {
+    // send broadcast values and reduceOperations to their owners
     try {
-      for (Map.Entry<String, AggregatorWrapper<Writable>> entry :
-          aggregatorMap.entrySet()) {
-        masterClient.sendAggregator(entry.getKey(),
-            entry.getValue().getAggregatorFactory(),
-            entry.getValue().getPreviousAggregatedValue());
+      for (Entry<String, Reducer<Object, Writable>> entry :
+          reducerMap.entrySet()) {
+        masterClient.sendToOwner(entry.getKey(),
+            GlobalCommType.REDUCE_OPERATIONS,
+            entry.getValue().getReduceOp());
         progressable.progress();
       }
-      masterClient.finishSendingAggregatedValues();
+
+      for (Entry<String, Writable> entry : broadcastMap.entrySet()) {
+        masterClient.sendToOwner(entry.getKey(),
+            GlobalCommType.BROADCAST,
+            entry.getValue());
+        progressable.progress();
+      }
+      masterClient.finishSendingValues();
+
+      broadcastMap.clear();
     } catch (IOException e) {
       throw new IllegalStateException("finishSuperstep: " +
           "IOException occurred while sending aggregators", e);
     }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("finishSuperstep: Aggregators finished");
-    }
   }
 
   /**
-   * Accept aggregated values sent by worker. Every aggregator will be sent
+   * Accept reduced values sent by worker. Every value will be sent
    * only once, by its owner.
    * We don't need to count the number of these requests because global
    * superstep barrier will happen after workers ensure all requests of this
    * type have been received and processed by master.
    *
-   * @param aggregatedValuesInput Input in which aggregated values are
+   * @param reducedValuesInput Input in which aggregated values are
    *                              written in the following format:
-   *                              number_of_aggregators
-   *                              name_1  value_1
-   *                              name_2  value_2
+   *                              numReducers
+   *                              name_1  REDUCED_VALUE  value_1
+   *                              name_2  REDUCED_VALUE  value_2
    *                              ...
    * @throws IOException
    */
-  public void acceptAggregatedValues(
-      DataInput aggregatedValuesInput) throws IOException {
-    int numAggregators = aggregatedValuesInput.readInt();
-    for (int i = 0; i < numAggregators; i++) {
-      String aggregatorName = aggregatedValuesInput.readUTF();
-      AggregatorWrapper<Writable> aggregator =
-          aggregatorMap.get(aggregatorName);
-      if (aggregator == null) {
+  public void acceptReducedValues(
+      DataInput reducedValuesInput) throws IOException {
+    int numReducers = reducedValuesInput.readInt();
+    for (int i = 0; i < numReducers; i++) {
+      String name = reducedValuesInput.readUTF();
+      GlobalCommType type =
+          GlobalCommType.values()[reducedValuesInput.readByte()];
+      if (type != GlobalCommType.REDUCED_VALUE) {
         throw new IllegalStateException(
-            "acceptAggregatedValues: " +
-                "Master received aggregator which isn't registered: " +
-                aggregatorName);
+            "SendReducedToMasterRequest received " + type);
+      }
+      Reducer<Object, Writable> reducer = reducerMap.get(name);
+      if (reducer == null) {
+        throw new IllegalStateException(
+            "acceptReducedValues: " +
+                "Master received reduced value which isn't registered: " +
+                name);
+      }
+
+      Writable valueToReduce = reducer.createInitialValue();
+      valueToReduce.readFields(reducedValuesInput);
+
+      if (reducer.getCurrentValue() != null) {
+        reducer.reducePartial(valueToReduce);
+      } else {
+        reducer.setCurrentValue(valueToReduce);
       }
-      Writable aggregatorValue = aggregator.createInitialValue();
-      aggregatorValue.readFields(aggregatedValuesInput);
-      aggregator.setCurrentAggregatedValue(aggregatorValue);
       progressable.progress();
     }
     if (LOG.isDebugEnabled()) {
-      LOG.debug("acceptAggregatedValues: Accepted one set with " +
-          numAggregators + " aggregated values");
+      LOG.debug("acceptReducedValues: Accepted one set with " +
+          numReducers + " aggregated values");
     }
   }
 
@@ -281,23 +242,10 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
    * @param superstep      Superstep which just finished
    * @param superstepState State of the superstep which just finished
    */
-  public void writeAggregators(long superstep, SuperstepState superstepState) {
+  public void writeAggregators(
+      long superstep, SuperstepState superstepState) {
     try {
-      Iterable<Map.Entry<String, Writable>> iter =
-          Iterables.transform(
-              aggregatorMap.entrySet(),
-              new Function<Map.Entry<String, AggregatorWrapper<Writable>>,
-                  Map.Entry<String, Writable>>() {
-                @Override
-                public Map.Entry<String, Writable> apply(
-                    Map.Entry<String, AggregatorWrapper<Writable>> entry) {
-                  progressable.progress();
-                  return new AbstractMap.SimpleEntry<String,
-                      Writable>(entry.getKey(),
-                      entry.getValue().getPreviousAggregatedValue());
-                }
-              });
-      aggregatorWriter.writeAggregator(iter,
+      aggregatorWriter.writeAggregator(reducedMap.entrySet(),
           (superstepState == SuperstepState.ALL_SUPERSTEPS_DONE) ?
               AggregatorWriter.LAST_SUPERSTEP : superstep);
     } catch (IOException e) {
@@ -333,43 +281,44 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
 
   @Override
   public void write(DataOutput out) throws IOException {
-    out.writeInt(aggregatorMap.size());
-    for (Map.Entry<String, AggregatorWrapper<Writable>> entry :
-        aggregatorMap.entrySet()) {
+    // At the end of superstep, only reduceOpMap can be non-empty
+    Preconditions.checkState(reducedMap.isEmpty(),
+        "reducedMap must be empty at the end of the superstep");
+
+    out.writeInt(reducerMap.size());
+    for (Entry<String, Reducer<Object, Writable>> entry :
+        reducerMap.entrySet()) {
       out.writeUTF(entry.getKey());
-      WritableUtils.writeWritableObject(
-          entry.getValue().getAggregatorFactory(), out);
-      out.writeBoolean(entry.getValue().isPersistent());
-      entry.getValue().getPreviousAggregatedValue().write(out);
+      entry.getValue().write(out);
       progressable.progress();
     }
+
+    out.writeInt(broadcastMap.size());
+    for (Entry<String, Writable> entry : broadcastMap.entrySet()) {
+      out.writeUTF(entry.getKey());
+      WritableUtils.writeWritableObject(entry.getValue(), out);
+    }
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    aggregatorMap.clear();
-    int numAggregators = in.readInt();
-    try {
-      for (int i = 0; i < numAggregators; i++) {
-        String aggregatorName = in.readUTF();
-        WritableFactory<Aggregator<Writable>> aggregatorFactory =
-            WritableUtils.readWritableObject(in, conf);
-        boolean isPersistent = in.readBoolean();
-        AggregatorWrapper<Writable> aggregatorWrapper = registerAggregator(
-            aggregatorName,
-            aggregatorFactory,
-            isPersistent);
-        Writable value = aggregatorWrapper.createInitialValue();
-        value.readFields(in);
-        aggregatorWrapper.setPreviousAggregatedValue(value);
-        progressable.progress();
-      }
-    } catch (InstantiationException e) {
-      throw new IllegalStateException("readFields: " +
-          "InstantiationException occurred", e);
-    } catch (IllegalAccessException e) {
-      throw new IllegalStateException("readFields: " +
-          "IllegalAccessException occurred", e);
+    reducedMap.clear();
+    broadcastMap.clear();
+    reducerMap.clear();
+
+    int numReducers = in.readInt();
+    for (int i = 0; i < numReducers; i++) {
+      String name = in.readUTF();
+      Reducer<Object, Writable> reducer = new Reducer<>();
+      reducer.readFields(in);
+      reducerMap.put(name, reducer);
+    }
+
+    int numBroadcast = in.readInt();
+    for (int i = 0; i < numBroadcast; i++) {
+      String name = in.readUTF();
+      Writable value = WritableUtils.readWritableObject(in, null);
+      broadcastMap.put(name, value);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
index 552cca9..72e4d0a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
@@ -24,6 +24,7 @@ import org.apache.giraph.combiner.MessageCombiner;
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.Computation;
 import org.apache.giraph.graph.GraphState;
+import org.apache.giraph.reducers.ReduceOperation;
 import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -43,7 +44,7 @@ import org.apache.hadoop.mapreduce.Mapper;
  */
 public abstract class MasterCompute
     extends DefaultImmutableClassesGiraphConfigurable
-    implements MasterAggregatorUsage, Writable {
+    implements MasterAggregatorUsage, MasterGlobalCommUsage, Writable {
   /** If true, do not do anymore computation on this vertex. */
   private boolean halt = false;
   /** Master aggregator usage */
@@ -190,10 +191,33 @@ public abstract class MasterCompute
   }
 
   @Override
+  public final <S, R extends Writable> void registerReduce(
+      String name, ReduceOperation<S, R> reduceOp) {
+    serviceMaster.getGlobalCommHandler().registerReduce(name, reduceOp);
+  }
+
+  @Override
+  public final <S, R extends Writable> void registerReduce(
+      String name, ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+    serviceMaster.getGlobalCommHandler().registerReduce(
+        name, reduceOp, globalInitialValue);
+  }
+
+  @Override
+  public final <T extends Writable> T getReduced(String name) {
+    return serviceMaster.getGlobalCommHandler().getReduced(name);
+  }
+
+  @Override
+  public final void broadcast(String name, Writable object) {
+    serviceMaster.getGlobalCommHandler().broadcast(name, object);
+  }
+
+  @Override
   public final <A extends Writable> boolean registerAggregator(
     String name, Class<? extends Aggregator<A>> aggregatorClass)
     throws InstantiationException, IllegalAccessException {
-    return serviceMaster.getAggregatorHandler().registerAggregator(
+    return serviceMaster.getAggregatorTranslationHandler().registerAggregator(
         name, aggregatorClass);
   }
 
@@ -201,7 +225,7 @@ public abstract class MasterCompute
   public final <A extends Writable> boolean registerAggregator(
     String name, WritableFactory<? extends Aggregator<A>> aggregator)
     throws InstantiationException, IllegalAccessException {
-    return serviceMaster.getAggregatorHandler().registerAggregator(
+    return serviceMaster.getAggregatorTranslationHandler().registerAggregator(
         name, aggregator);
   }
 
@@ -210,19 +234,21 @@ public abstract class MasterCompute
       String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
-    return serviceMaster.getAggregatorHandler().registerPersistentAggregator(
-        name, aggregatorClass);
+    return serviceMaster.getAggregatorTranslationHandler()
+        .registerPersistentAggregator(name, aggregatorClass);
   }
 
   @Override
   public final <A extends Writable> A getAggregatedValue(String name) {
-    return serviceMaster.getAggregatorHandler().<A>getAggregatedValue(name);
+    return serviceMaster.getAggregatorTranslationHandler()
+        .<A>getAggregatedValue(name);
   }
 
   @Override
   public final <A extends Writable> void setAggregatedValue(
       String name, A value) {
-    serviceMaster.getAggregatorHandler().setAggregatedValue(name, value);
+    serviceMaster.getAggregatorTranslationHandler()
+        .setAggregatedValue(name, value);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/master/MasterGlobalCommUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterGlobalCommUsage.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterGlobalCommUsage.java
new file mode 100644
index 0000000..c3ce0ea
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterGlobalCommUsage.java
@@ -0,0 +1,68 @@
+/*
+ * 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.master;
+
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Master compute can access reduce and broadcast methods
+ * through this interface, from masterCompute method.
+ */
+public interface MasterGlobalCommUsage {
+  /**
+   * Register reducer to be reduced in the next worker computation,
+   * using given name and operations.
+   * @param name Name of the reducer
+   * @param reduceOp Reduce operations
+   * @param <S> Single value type
+   * @param <R> Reduced value type
+   */
+  <S, R extends Writable> void registerReduce(
+      String name, ReduceOperation<S, R> reduceOp);
+
+  /**
+   * Register reducer to be reduced in the next worker computation, using
+   * given name and operations, starting globally from globalInitialValue.
+   * (globalInitialValue is reduced only once, each worker will still start
+   * from neutral initial value)
+   *
+   * @param name Name of the reducer
+   * @param reduceOp Reduce operations
+   * @param globalInitialValue Global initial value
+   * @param <S> Single value type
+   * @param <R> Reduced value type
+   */
+  <S, R extends Writable> void registerReduce(
+      String name, ReduceOperation<S, R> reduceOp, R globalInitialValue);
+
+  /**
+   * Get reduced value from previous worker computation.
+   * @param name Name of the reducer
+   * @return Reduced value
+   * @param <R> Reduced value type
+   */
+  <R extends Writable> R getReduced(String name);
+
+  /**
+   * Broadcast given value to all workers for next computation.
+   * @param name Name of the broadcast object
+   * @param value Value
+   */
+  void broadcast(String name, Writable value);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java b/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
new file mode 100644
index 0000000..a675f4d
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
@@ -0,0 +1,34 @@
+/*
+ * 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.reducers;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * ReduceOperation object when single object being reduced is of
+ * same type as reduced value.
+ *
+ * @param <R> Reduced object type.
+ */
+public abstract class OnSameReduceOperation<R extends Writable>
+    implements ReduceOperation<R, R> {
+  @Override
+  public final void reducePartial(R curValue, R valueToReduce) {
+    reduceSingle(curValue, valueToReduce);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/f43f4500/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java b/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
new file mode 100644
index 0000000..434e21a
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
@@ -0,0 +1,57 @@
+/*
+ * 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.reducers;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Reduce operations defining how to reduce single values
+ * passed on workers, into partial values on workers, and then
+ * into a single global reduced value.
+ *
+ * Object should be thread safe. Most frequently it should be
+ * immutable object, so that functions can execute concurrently.
+ * Rarely when object is mutable ({@link AggregatorReduceOperation}),
+ * i.e. stores reusable object inside, accesses should be synchronized.
+ *
+ * @param <S> Single value type, objects passed on workers
+ * @param <R> Reduced value type
+ */
+public interface ReduceOperation<S, R extends Writable> extends Writable {
+  /**
+   * Return new reduced value which is neutral to reduce operation.
+   *
+   * @return Neutral value
+   */
+  R createInitialValue();
+  /**
+   * Add a new value.
+   * Needs to be commutative and associative
+   *
+   * @param curValue Partial value into which to reduce and store the result
+   * @param valueToReduce Single value to be reduced
+   */
+  void reduceSingle(R curValue, S valueToReduce);
+  /**
+   * Add partially reduced value to current partially reduced value.
+   *
+   * @param curValue Partial value into which to reduce and store the result
+   * @param valueToReduce Partial value to be reduced
+   */
+  void reducePartial(R curValue, R valueToReduce);
+}


[21/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Fix GIRAPH-934


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/24309fea
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/24309fea
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/24309fea

Branch: refs/heads/release-1.1
Commit: 24309fea666a7c82b79e17395523ad1b8ee87528
Parents: 6d4af60
Author: Maja Kabiljo <ma...@fb.com>
Authored: Fri Aug 1 17:15:59 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Fri Aug 1 17:15:59 2014 -0700

----------------------------------------------------------------------
 .../org/apache/giraph/aggregators/TestAggregatorsHandling.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/24309fea/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java b/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
index e2b611b..eb3f686 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
@@ -129,8 +129,8 @@ public class TestAggregatorsHandling extends BspCase {
 
     AggregatorWrapper<Writable> regularAgg =
         getAggregatorMap(restartedHandler).get(regularAggName);
-    assertTrue(
-        regularAgg.getAggregatorClass().equals(LongSumAggregator.class));
+    assertTrue(regularAgg.getAggregatorFactory().create().getClass().equals(
+        LongSumAggregator.class));
     assertEquals(regularValue, regularAgg.getPreviousAggregatedValue());
     assertEquals(regularValue,
         restartedHandler.<LongWritable>getAggregatedValue(regularAggName));
@@ -138,7 +138,7 @@ public class TestAggregatorsHandling extends BspCase {
 
     AggregatorWrapper<Writable> persistentAgg =
         getAggregatorMap(restartedHandler).get(persistentAggName);
-    assertTrue(persistentAgg.getAggregatorClass().equals
+    assertTrue(persistentAgg.getAggregatorFactory().create().getClass().equals
         (DoubleOverwriteAggregator.class));
     assertEquals(persistentValue, persistentAgg.getPreviousAggregatedValue());
     assertEquals(persistentValue,


[28/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-933: Checkpointing improvements (edunov via majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/5adca63d
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/5adca63d
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/5adca63d

Branch: refs/heads/release-1.1
Commit: 5adca63deca25d84f4fdea053c35a85efc8bbb3d
Parents: bc9f823
Author: Maja Kabiljo <ma...@fb.com>
Authored: Fri Aug 15 15:03:19 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Fri Aug 15 15:03:19 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../java/org/apache/giraph/bsp/BspService.java  | 108 ++++++----
 .../apache/giraph/bsp/CentralizedService.java   |   9 -
 .../giraph/bsp/CentralizedServiceMaster.java    |   4 +-
 .../giraph/bsp/CentralizedServiceWorker.java    |   8 +
 .../org/apache/giraph/bsp/CheckpointStatus.java |  31 +++
 .../org/apache/giraph/bsp/SuperstepState.java   |  30 ++-
 .../java/org/apache/giraph/comm/ServerData.java |  10 +
 .../org/apache/giraph/conf/GiraphConstants.java |  10 +-
 .../giraph/graph/FinishedSuperstepStats.java    |  20 +-
 .../org/apache/giraph/graph/GlobalStats.java    |  27 ++-
 .../apache/giraph/graph/GraphTaskManager.java   |  44 ++--
 .../job/DefaultGiraphJobRetryChecker.java       |   5 +
 .../java/org/apache/giraph/job/GiraphJob.java   |  23 ++
 .../giraph/job/GiraphJobRetryChecker.java       |   6 +
 .../java/org/apache/giraph/job/HadoopUtils.java |  15 ++
 .../apache/giraph/master/BspServiceMaster.java  | 147 +++++++++----
 .../org/apache/giraph/master/MasterThread.java  |  10 +-
 .../apache/giraph/utils/CheckpointingUtils.java |  62 ++++++
 .../org/apache/giraph/utils/WritableUtils.java  |  63 ++++++
 .../apache/giraph/worker/BspServiceWorker.java  |  47 ++++-
 .../apache/giraph/utils/TestWritableUtils.java  |  70 +++++++
 .../org/apache/giraph/TestCheckpointing.java    | 208 +++++++++++++++----
 pom.xml                                         |   2 +-
 24 files changed, 787 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 66136b2..b64ce2c 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-933: Checkpointing improvements (edunov via majakabiljo)
+
   GIRAPH-943: Perf regression due to netty 4.0.21 (pavanka)
 
   GIRAPH-935: Loosen modifiers when needed (ikabiljo via majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
----------------------------------------------------------------------
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 02577b9..c418a89 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
@@ -24,13 +24,16 @@ import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.graph.InputSplitEvents;
 import org.apache.giraph.graph.InputSplitPaths;
 import org.apache.giraph.partition.GraphPartitionerFactory;
+import org.apache.giraph.utils.CheckpointingUtils;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.giraph.zk.BspEvent;
 import org.apache.giraph.zk.PredicateLock;
 import org.apache.giraph.zk.ZooKeeperExt;
 import org.apache.giraph.zk.ZooKeeperManager;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -50,10 +53,10 @@ import java.net.UnknownHostException;
 import java.nio.charset.Charset;
 import java.security.InvalidParameterException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import static org.apache.giraph.conf.GiraphConstants.CHECKPOINT_DIRECTORY;
 import static org.apache.giraph.conf.GiraphConstants.RESTART_JOB_ID;
 
 /**
@@ -162,6 +165,8 @@ public abstract class BspService<I extends WritableComparable,
   public static final String WORKER_PROGRESSES = "/_workerProgresses";
   /** Denotes that computation should be halted */
   public static final String HALT_COMPUTATION_NODE = "/_haltComputation";
+  /** User sets this flag to checkpoint and stop the job */
+  public static final String FORCE_CHECKPOINT_USER_FLAG = "/_checkpointAndStop";
   /** Denotes which workers have been cleaned up */
   public static final String CLEANED_UP_DIR = "/_cleanedUpDir";
   /** JSON partition stats key */
@@ -283,8 +288,6 @@ public abstract class BspService<I extends WritableComparable,
   private final GraphTaskManager<I, V, E> graphTaskManager;
   /** File system */
   private final FileSystem fs;
-  /** Checkpoint frequency */
-  private final int checkpointFrequency;
 
   /**
    * Constructor.
@@ -325,13 +328,6 @@ public abstract class BspService<I extends WritableComparable,
     this.taskPartition = conf.getTaskPartition();
     this.restartedSuperstep = conf.getLong(
         GiraphConstants.RESTART_SUPERSTEP, UNSET_SUPERSTEP);
-    this.cachedSuperstep = restartedSuperstep;
-    if ((restartedSuperstep != UNSET_SUPERSTEP) &&
-        (restartedSuperstep < 0)) {
-      throw new IllegalArgumentException(
-          "BspService: Invalid superstep to restart - " +
-              restartedSuperstep);
-    }
     try {
       this.hostname = conf.getLocalHostname();
     } catch (UnknownHostException e) {
@@ -340,8 +336,6 @@ public abstract class BspService<I extends WritableComparable,
     this.hostnamePartitionId = hostname + "_" + getTaskPartition();
     this.graphPartitionerFactory = conf.createGraphPartitioner();
 
-    this.checkpointFrequency = conf.getCheckpointFrequency();
-
     basePath = ZooKeeperManager.getBasePath(conf) + BASE_DIR + "/" + jobId;
     getContext().getCounter(GiraphConstants.ZOOKEEPER_BASE_PATH_COUNTER_GROUP,
         basePath);
@@ -360,13 +354,14 @@ public abstract class BspService<I extends WritableComparable,
     cleanedUpPath = basePath + CLEANED_UP_DIR;
 
     String restartJobId = RESTART_JOB_ID.get(conf);
+
     savedCheckpointBasePath =
-        CHECKPOINT_DIRECTORY.getWithDefault(getConfiguration(),
-            CHECKPOINT_DIRECTORY.getDefaultValue() + "/" +
-                (restartJobId == null ? getJobId() : restartJobId));
-    checkpointBasePath =
-        CHECKPOINT_DIRECTORY.getWithDefault(getConfiguration(),
-            CHECKPOINT_DIRECTORY.getDefaultValue() + "/" + getJobId());
+        CheckpointingUtils.getCheckpointBasePath(getConfiguration(),
+            restartJobId == null ? getJobId() : restartJobId);
+
+    checkpointBasePath = CheckpointingUtils.
+        getCheckpointBasePath(getConfiguration(), getJobId());
+
     masterElectionPath = basePath + MASTER_ELECTION_DIR;
     myProgressPath = basePath + WORKER_PROGRESSES + "/" + taskPartition;
     String serverPortList = conf.getZookeeperList();
@@ -392,6 +387,24 @@ public abstract class BspService<I extends WritableComparable,
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
+
+    //Trying to restart from the latest superstep
+    if (restartJobId != null &&
+        restartedSuperstep == UNSET_SUPERSTEP) {
+      try {
+        restartedSuperstep = getLastCheckpointedSuperstep();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    this.cachedSuperstep = restartedSuperstep;
+    if ((restartedSuperstep != UNSET_SUPERSTEP) &&
+        (restartedSuperstep < 0)) {
+      throw new IllegalArgumentException(
+          "BspService: Invalid superstep to restart - " +
+              restartedSuperstep);
+    }
+
   }
 
   /**
@@ -643,28 +656,6 @@ public abstract class BspService<I extends WritableComparable,
   }
 
   /**
-   * Should checkpoint on this superstep?  If checkpointing, always
-   * checkpoint the first user superstep.  If restarting, the first
-   * checkpoint is after the frequency has been met.
-   *
-   * @param superstep Decide if checkpointing no this superstep
-   * @return True if this superstep should be checkpointed, false otherwise
-   */
-  public final boolean checkpointFrequencyMet(long superstep) {
-    if (checkpointFrequency == 0) {
-      return false;
-    }
-    long firstCheckpoint = INPUT_SUPERSTEP + 1;
-    if (getRestartedSuperstep() != UNSET_SUPERSTEP) {
-      firstCheckpoint = getRestartedSuperstep() + checkpointFrequency;
-    }
-    if (superstep < firstCheckpoint) {
-      return false;
-    }
-    return ((superstep - firstCheckpoint) % checkpointFrequency) == 0;
-  }
-
-  /**
    * Get the file system
    *
    * @return file system
@@ -1241,4 +1232,41 @@ public abstract class BspService<I extends WritableComparable,
     }
     return eventProcessed;
   }
+
+  /**
+   * Get the last saved superstep.
+   *
+   * @return Last good superstep number
+   * @throws IOException
+   */
+  protected long getLastCheckpointedSuperstep() throws IOException {
+    FileStatus[] fileStatusArray =
+        getFs().listStatus(new Path(savedCheckpointBasePath),
+            new FinalizedCheckpointPathFilter());
+    if (fileStatusArray == null) {
+      return -1;
+    }
+    Arrays.sort(fileStatusArray);
+    long lastCheckpointedSuperstep = getCheckpoint(
+        fileStatusArray[fileStatusArray.length - 1].getPath());
+    if (LOG.isInfoEnabled()) {
+      LOG.info("getLastGoodCheckpoint: Found last good checkpoint " +
+          lastCheckpointedSuperstep + " from " +
+          fileStatusArray[fileStatusArray.length - 1].
+              getPath().toString());
+    }
+    return lastCheckpointedSuperstep;
+  }
+
+  /**
+   * Only get the finalized checkpoint files
+   */
+  private static class FinalizedCheckpointPathFilter implements PathFilter {
+    @Override
+    public boolean accept(Path path) {
+      return path.getName().endsWith(BspService.CHECKPOINT_FINALIZED_POSTFIX);
+    }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
index ff3e427..560f1fb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
@@ -51,15 +51,6 @@ public interface CentralizedService<I extends WritableComparable,
   long getRestartedSuperstep();
 
   /**
-   * Given a superstep, should it be checkpointed based on the
-   * checkpoint frequency?
-   *
-   * @param superstep superstep to check against frequency
-   * @return true if checkpoint frequency met or superstep is 1.
-   */
-  boolean checkpointFrequencyMet(long superstep);
-
-  /**
    * Get list of workers
    *
    * @return List of workers

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
index e5b7cf3..9b4f9d6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
@@ -175,7 +175,9 @@ public interface CentralizedServiceMaster<I extends WritableComparable,
    *
    * @throws IOException
    * @throws InterruptedException
+   * @param superstepState what was the state
+   *                       of the last complete superstep?
    */
-  void cleanup()
+  void cleanup(SuperstepState superstepState)
     throws IOException, InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
index e5d0ae1..37aed45 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
@@ -21,6 +21,7 @@ package org.apache.giraph.bsp;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.graph.FinishedSuperstepStats;
+import org.apache.giraph.graph.GlobalStats;
 import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.superstep_output.SuperstepOutput;
@@ -237,4 +238,11 @@ public interface CentralizedServiceWorker<I extends WritableComparable,
    */
   void cleanup(FinishedSuperstepStats finishedSuperstepStats)
     throws IOException, InterruptedException;
+
+  /**
+   * Loads Global stats from zookeeper.
+   * @return global stats stored in zookeeper for
+   * previous superstep.
+   */
+  GlobalStats getGlobalStats();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/bsp/CheckpointStatus.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CheckpointStatus.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CheckpointStatus.java
new file mode 100644
index 0000000..74db490
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CheckpointStatus.java
@@ -0,0 +1,31 @@
+/*
+ * 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.bsp;
+
+/**
+ * Enum represents possible checkpoint state.
+ */
+public enum CheckpointStatus {
+  /** Do nothing, no checkpoint required */
+  NONE,
+  /** Regular checkpoint */
+  CHECKPOINT,
+  /** Do checkpoint and then halt further computation */
+  CHECKPOINT_AND_HALT
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/bsp/SuperstepState.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/SuperstepState.java b/giraph-core/src/main/java/org/apache/giraph/bsp/SuperstepState.java
index c384fbf..768278b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/SuperstepState.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/SuperstepState.java
@@ -23,11 +23,33 @@ package org.apache.giraph.bsp;
  */
 public enum SuperstepState {
   /** Nothing happened yet */
-  INITIAL,
+  INITIAL(false),
   /** A worker died during this superstep */
-  WORKER_FAILURE,
+  WORKER_FAILURE(false),
   /** This superstep completed correctly */
-  THIS_SUPERSTEP_DONE,
+  THIS_SUPERSTEP_DONE(false),
   /** All supersteps are complete */
-  ALL_SUPERSTEPS_DONE,
+  ALL_SUPERSTEPS_DONE(true),
+  /** Execution halted */
+  CHECKPOINT_AND_HALT(true);
+
+  /** Should we stop execution after this superstep? */
+  private boolean executionComplete;
+
+  /**
+   * Enum constructor
+   * @param executionComplete is final state?
+   */
+  SuperstepState(boolean executionComplete) {
+    this.executionComplete = executionComplete;
+  }
+
+  /**
+   * Returns true if execution has to be stopped after this
+   * superstep.
+   * @return whether execution is complete.
+   */
+  public boolean isExecutionComplete() {
+    return executionComplete;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
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 a92cd1c..1fd85e4 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
@@ -269,4 +269,14 @@ public class ServerData<I extends WritableComparable,
   public void addIncomingWorkerToWorkerMessage(Writable message) {
     incomingWorkerToWorkerMessages.add(message);
   }
+
+
+  /**
+   * Get worker to worker messages received in previous superstep.
+   * @return list of current worker to worker messages.
+   */
+  public List<Writable> getCurrentWorkerToWorkerMessages() {
+    return currentWorkerToWorkerMessages;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
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 0424a47..da0a8db 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
@@ -1137,12 +1137,16 @@ public interface GiraphConstants {
       new IntConfOption("giraph.checkpoint.io.threads", 8,
           "Number of threads for writing and reading checkpoints");
 
-  /** Compression algorithm to be used for checkpointing */
+  /**
+   * Compression algorithm to be used for checkpointing.
+   * Defined by extension for hadoop compatibility reasons.
+  */
   StrConfOption CHECKPOINT_COMPRESSION_CODEC =
       new StrConfOption("giraph.checkpoint.compression.codec",
-          "org.apache.hadoop.io.compress.DefaultCodec",
+          ".deflate",
           "Defines compression algorithm we will be using for " +
-              "storing checkpoint");
+              "storing checkpoint. Available options include but " +
+              "not restricted to: .deflate, .gz, .bz2, .lzo");
 
   /** Number of threads to use in async message store, 0 means
    * we should not use async message processing */

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/graph/FinishedSuperstepStats.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/FinishedSuperstepStats.java b/giraph-core/src/main/java/org/apache/giraph/graph/FinishedSuperstepStats.java
index c351778..f7895a9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/FinishedSuperstepStats.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/FinishedSuperstepStats.java
@@ -17,6 +17,8 @@
  */
 package org.apache.giraph.graph;
 
+import org.apache.giraph.bsp.CheckpointStatus;
+
 /**
  * Immutable graph stats after the completion of a superstep
  */
@@ -27,6 +29,11 @@ public class FinishedSuperstepStats extends VertexEdgeCount {
   private final boolean allVerticesHalted;
   /** Needs to load a checkpoint */
   private final boolean mustLoadCheckpoint;
+  /**
+   * Master decides when we need to checkpoint and what should
+   * we do next.
+   */
+  private final CheckpointStatus checkpointStatus;
 
   /**
    * Constructor.
@@ -36,16 +43,19 @@ public class FinishedSuperstepStats extends VertexEdgeCount {
    * @param numVertices Number of vertices
    * @param numEdges Number of edges
    * @param mustLoadCheckpoint Has to load a checkpoint?
+   * @param checkpointStatus Should we checkpoint after this superstep?
    */
   public FinishedSuperstepStats(long numLocalVertices,
                                 boolean allVerticesHalted,
                                 long numVertices,
                                 long numEdges,
-                                boolean mustLoadCheckpoint) {
+                                boolean mustLoadCheckpoint,
+                                CheckpointStatus checkpointStatus) {
     super(numVertices, numEdges);
     this.localVertexCount = numLocalVertices;
     this.allVerticesHalted = allVerticesHalted;
     this.mustLoadCheckpoint = mustLoadCheckpoint;
+    this.checkpointStatus = checkpointStatus;
   }
 
   public long getLocalVertexCount() {
@@ -69,4 +79,12 @@ public class FinishedSuperstepStats extends VertexEdgeCount {
   public boolean mustLoadCheckpoint() {
     return mustLoadCheckpoint;
   }
+
+  /**
+   * What master thinks about checkpointing after this superstep.
+   * @return CheckpointStatus that reflects master decision.
+   */
+  public CheckpointStatus getCheckpointStatus() {
+    return checkpointStatus;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/graph/GlobalStats.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GlobalStats.java b/giraph-core/src/main/java/org/apache/giraph/graph/GlobalStats.java
index bc56c9c..e11f02c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GlobalStats.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GlobalStats.java
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.giraph.bsp.CheckpointStatus;
 import org.apache.giraph.partition.PartitionStats;
 import org.apache.hadoop.io.Writable;
 
@@ -41,6 +42,12 @@ public class GlobalStats implements Writable {
   private long messageBytesCount = 0;
   /** Whether the computation should be halted */
   private boolean haltComputation = false;
+  /**
+   * Master's decision on whether we should checkpoint and
+   * what to do next.
+   */
+  private CheckpointStatus checkpointStatus =
+      CheckpointStatus.NONE;
 
   /**
    * Add the stats of a partition to the global stats.
@@ -81,6 +88,14 @@ public class GlobalStats implements Writable {
     haltComputation = value;
   }
 
+  public CheckpointStatus getCheckpointStatus() {
+    return checkpointStatus;
+  }
+
+  public void setCheckpointStatus(CheckpointStatus checkpointStatus) {
+    this.checkpointStatus = checkpointStatus;
+  }
+
   /**
    * Add messages to the global stats.
    *
@@ -107,6 +122,11 @@ public class GlobalStats implements Writable {
     messageCount = input.readLong();
     messageBytesCount = input.readLong();
     haltComputation = input.readBoolean();
+    if (input.readBoolean()) {
+      checkpointStatus = CheckpointStatus.values()[input.readInt()];
+    } else {
+      checkpointStatus = null;
+    }
   }
 
   @Override
@@ -117,6 +137,10 @@ public class GlobalStats implements Writable {
     output.writeLong(messageCount);
     output.writeLong(messageBytesCount);
     output.writeBoolean(haltComputation);
+    output.writeBoolean(checkpointStatus != null);
+    if (checkpointStatus != null) {
+      output.writeInt(checkpointStatus.ordinal());
+    }
   }
 
   @Override
@@ -124,6 +148,7 @@ public class GlobalStats implements Writable {
     return "(vtx=" + vertexCount + ",finVtx=" +
         finishedVertexCount + ",edges=" + edgeCount + ",msgCount=" +
         messageCount + ",msgBytesCount=" +
-          messageBytesCount + ",haltComputation=" + haltComputation + ")";
+          messageBytesCount + ",haltComputation=" + haltComputation +
+        ", checkpointStatus=" + checkpointStatus + ')';
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
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 6ebb002..8a97939 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
@@ -21,6 +21,7 @@ package org.apache.giraph.graph;
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.bsp.CheckpointStatus;
 import org.apache.giraph.comm.messages.MessageStore;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
@@ -120,7 +121,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   private GraphFunctions graphFunctions = GraphFunctions.UNKNOWN;
   /** Superstep stats */
   private FinishedSuperstepStats finishedSuperstepStats =
-      new FinishedSuperstepStats(0, false, 0, 0, false);
+      new FinishedSuperstepStats(0, false, 0, 0, false, CheckpointStatus.NONE);
 
   // Per-Job Metrics
   /** Timer for WorkerContext#preApplication() */
@@ -281,7 +282,18 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
       context.progress();
       serviceWorker.exchangeVertexPartitions(masterAssignedPartitionOwners);
       context.progress();
-      graphState = checkSuperstepRestarted(superstep, graphState);
+      boolean hasBeenRestarted = checkSuperstepRestarted(superstep);
+
+      GlobalStats globalStats = serviceWorker.getGlobalStats();
+
+      if (hasBeenRestarted) {
+        graphState = new GraphState(superstep,
+            finishedSuperstepStats.getVertexCount(),
+            finishedSuperstepStats.getEdgeCount(),
+            context);
+      } else if (storeCheckpoint(globalStats.getCheckpointStatus())) {
+        break;
+      }
       prepareForSuperstep(graphState);
       context.progress();
       MessageStore<I, Writable> messageStore =
@@ -735,11 +747,9 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   /**
    * Handle the event that this superstep is a restart of a failed one.
    * @param superstep current superstep
-   * @param graphState the BSP graph state
    * @return the graph state, updated if this is a restart superstep
    */
-  private GraphState checkSuperstepRestarted(long superstep,
-    GraphState graphState) throws IOException {
+  private boolean checkSuperstepRestarted(long superstep) throws IOException {
     // Might need to restart from another superstep
     // (manually or automatic), or store a checkpoint
     if (serviceWorker.getRestartedSuperstep() == superstep) {
@@ -750,15 +760,25 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
         serviceWorker.getRestartedSuperstep());
       finishedSuperstepStats = new FinishedSuperstepStats(0, false,
           vertexEdgeCount.getVertexCount(), vertexEdgeCount.getEdgeCount(),
-          false);
-      graphState = new GraphState(superstep,
-          finishedSuperstepStats.getVertexCount(),
-          finishedSuperstepStats.getEdgeCount(),
-          context);
-    } else if (serviceWorker.checkpointFrequencyMet(superstep)) {
+          false, CheckpointStatus.NONE);
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Check if it's time to checkpoint and actually does checkpointing
+   * if it is.
+   * @param checkpointStatus master's decision
+   * @return true if we need to stop computation after checkpoint
+   * @throws IOException
+   */
+  private boolean storeCheckpoint(CheckpointStatus checkpointStatus)
+    throws IOException {
+    if (checkpointStatus != CheckpointStatus.NONE) {
       serviceWorker.storeCheckpoint();
     }
-    return graphState;
+    return checkpointStatus == CheckpointStatus.CHECKPOINT_AND_HALT;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java b/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
index 0cab86c..edf6bce 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
@@ -30,4 +30,9 @@ public class DefaultGiraphJobRetryChecker implements GiraphJobRetryChecker {
     // By default, don't retry failed jobs
     return false;
   }
+
+  @Override
+  public boolean shouldRestartCheckpoint() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
index 4a1f02e..436126b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
@@ -24,9 +24,13 @@ import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GraphMapper;
+import org.apache.giraph.utils.CheckpointingUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobID;
 import org.apache.log4j.Logger;
 
 import java.io.IOException;
@@ -261,6 +265,25 @@ public class GiraphJob {
         jobProgressTracker.stop();
       }
       jobObserver.jobFinished(submittedJob, passed);
+
+      FileSystem fs = FileSystem.get(conf);
+      JobID jobID = HadoopUtils.getJobID(submittedJob);
+      if (jobID != null) {
+        Path checkpointMark =
+            CheckpointingUtils.getCheckpointMarkPath(conf, jobID.toString());
+
+        if (fs.exists(checkpointMark)) {
+          if (retryChecker.shouldRestartCheckpoint()) {
+            GiraphConstants.RESTART_JOB_ID.set(conf, jobID.toString());
+            continue;
+          }
+        }
+      } else {
+        LOG.warn("jobID is null, are you using hadoop 0.20.203? " +
+            "Please report this issue here " +
+            "https://issues.apache.org/jira/browse/GIRAPH-933");
+      }
+
       if (passed || !retryChecker.shouldRetry(submittedJob, tryCount)) {
         return passed;
       }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
index 53a800e..556b128 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
@@ -33,4 +33,10 @@ public interface GiraphJobRetryChecker {
    * @return True iff job should be retried
    */
   boolean shouldRetry(Job submittedJob, int tryCount);
+
+  /**
+   * The job has been checkpointed and halted. Should we now restart it?
+   * @return true if checkpointed job should be automatically restarted.
+   */
+  boolean shouldRestartCheckpoint();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/job/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/HadoopUtils.java b/giraph-core/src/main/java/org/apache/giraph/job/HadoopUtils.java
index 9530fd6..f2c673b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/HadoopUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/HadoopUtils.java
@@ -18,6 +18,7 @@
 package org.apache.giraph.job;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -102,6 +103,20 @@ public class HadoopUtils {
   }
 
   /**
+   * Get Job ID from job.
+   * May return null for hadoop 0.20.203
+   * @param job submitted job
+   * @return JobId for submitted job.
+   */
+  public static JobID getJobID(Job job) {
+    /*if[HADOOP_JOB_ID_AVAILABLE]
+    return job.getID();
+    else[HADOOP_JOB_ID_AVAILABLE]*/
+    return job.getJobID();
+    /*end[HADOOP_JOB_ID_AVAILABLE]*/
+  }
+
+  /**
    * Create a JobContext, supporting many Hadoops.
    *
    * @param conf Configuration

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index e129390..671df23 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -24,6 +24,7 @@ import org.apache.commons.io.FilenameUtils;
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.BspInputFormat;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
+import org.apache.giraph.bsp.CheckpointStatus;
 import org.apache.giraph.bsp.SuperstepState;
 import org.apache.giraph.comm.MasterClient;
 import org.apache.giraph.comm.MasterServer;
@@ -56,6 +57,7 @@ import org.apache.giraph.metrics.GiraphTimerContext;
 import org.apache.giraph.metrics.ResetSuperstepMetricsObserver;
 import org.apache.giraph.metrics.SuperstepMetricsRegistry;
 import org.apache.giraph.metrics.WorkerSuperstepMetrics;
+import org.apache.giraph.utils.CheckpointingUtils;
 import org.apache.giraph.utils.JMapHistoDumper;
 import org.apache.giraph.utils.ReactiveJMapHistoDumper;
 import org.apache.giraph.utils.ProgressableUtils;
@@ -67,10 +69,8 @@ import org.apache.giraph.worker.WorkerInfo;
 import org.apache.giraph.zk.BspEvent;
 import org.apache.giraph.zk.PredicateLock;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -99,7 +99,6 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -189,6 +188,11 @@ public class BspServiceMaster<I extends WritableComparable,
   /** MasterCompute time */
   private GiraphTimer masterComputeTimer;
 
+  /** Checkpoint frequency */
+  private final int checkpointFrequency;
+  /** Current checkpoint status */
+  private CheckpointStatus checkpointStatus;
+
   /**
    * Constructor for setting up the master.
    *
@@ -224,6 +228,9 @@ public class BspServiceMaster<I extends WritableComparable,
     }
     observers = conf.createMasterObservers();
 
+    this.checkpointFrequency = conf.getCheckpointFrequency();
+    this.checkpointStatus = CheckpointStatus.NONE;
+
     GiraphMetrics.get().addSuperstepResetObserver(this);
     GiraphStats.init((Mapper.Context) context);
   }
@@ -365,7 +372,11 @@ public class BspServiceMaster<I extends WritableComparable,
         @SuppressWarnings("deprecation")
         JobID jobId = JobID.forName(getJobId());
         RunningJob job = jobClient.getJob(jobId);
-        job.killJob();
+        if (job != null) {
+          job.killJob();
+        } else {
+          LOG.error("Jon not found for jobId=" + getJobId());
+        }
       }
     } catch (IOException ioe) {
       throw new RuntimeException(ioe);
@@ -1196,11 +1207,11 @@ public class BspServiceMaster<I extends WritableComparable,
    *
    * @param chosenWorkerInfoHealthPath Path to the healthy workers in ZooKeeper
    * @param chosenWorkerInfoList List of the healthy workers
-   * @return true if they are all alive, false otherwise.
+   * @return a list of dead workers. Empty list if all workers are alive.
    * @throws InterruptedException
    * @throws KeeperException
    */
-  private boolean superstepChosenWorkerAlive(
+  private Collection<WorkerInfo> superstepChosenWorkerAlive(
     String chosenWorkerInfoHealthPath,
     List<WorkerInfo> chosenWorkerInfoList)
     throws KeeperException, InterruptedException {
@@ -1208,16 +1219,13 @@ public class BspServiceMaster<I extends WritableComparable,
         getWorkerInfosFromPath(chosenWorkerInfoHealthPath, false);
     Set<WorkerInfo> chosenWorkerInfoHealthySet =
         new HashSet<WorkerInfo>(chosenWorkerInfoHealthyList);
-    boolean allChosenWorkersHealthy = true;
+    List<WorkerInfo> deadWorkers = new ArrayList<>();
     for (WorkerInfo chosenWorkerInfo : chosenWorkerInfoList) {
       if (!chosenWorkerInfoHealthySet.contains(chosenWorkerInfo)) {
-        allChosenWorkersHealthy = false;
-        LOG.error("superstepChosenWorkerAlive: Missing chosen " +
-            "worker " + chosenWorkerInfo +
-            " on superstep " + getSuperstep());
+        deadWorkers.add(chosenWorkerInfo);
       }
     }
-    return allChosenWorkersHealthy;
+    return deadWorkers;
   }
 
   @Override
@@ -1257,37 +1265,13 @@ public class BspServiceMaster<I extends WritableComparable,
     }
   }
 
-  /**
-   * Only get the finalized checkpoint files
-   */
-  public static class FinalizedCheckpointPathFilter implements PathFilter {
-    @Override
-    public boolean accept(Path path) {
-      return path.getName().endsWith(BspService.CHECKPOINT_FINALIZED_POSTFIX);
-    }
-  }
-
   @Override
   public long getLastGoodCheckpoint() throws IOException {
     // Find the last good checkpoint if none have been written to the
     // knowledge of this master
     if (lastCheckpointedSuperstep == -1) {
       try {
-        FileStatus[] fileStatusArray =
-            getFs().listStatus(new Path(savedCheckpointBasePath),
-                new FinalizedCheckpointPathFilter());
-        if (fileStatusArray == null) {
-          return -1;
-        }
-        Arrays.sort(fileStatusArray);
-        lastCheckpointedSuperstep = getCheckpoint(
-            fileStatusArray[fileStatusArray.length - 1].getPath());
-        if (LOG.isInfoEnabled()) {
-          LOG.info("getLastGoodCheckpoint: Found last good checkpoint " +
-              lastCheckpointedSuperstep + " from " +
-              fileStatusArray[fileStatusArray.length - 1].
-                  getPath().toString());
-        }
+        lastCheckpointedSuperstep = getLastCheckpointedSuperstep();
       } catch (IOException e) {
         LOG.fatal("getLastGoodCheckpoint: No last good checkpoints can be " +
             "found, killing the job.", e);
@@ -1306,12 +1290,15 @@ public class BspServiceMaster<I extends WritableComparable,
    *        hostname and id
    * @param workerInfoList List of the workers to wait for
    * @param event Event to wait on for a chance to be done.
+   * @param ignoreDeath In case if worker died after making it through
+   *                    barrier, we will ignore death if set to true.
    * @return True if barrier was successful, false if there was a worker
    *         failure
    */
   private boolean barrierOnWorkerList(String finishedWorkerPath,
       List<WorkerInfo> workerInfoList,
-      BspEvent event) {
+      BspEvent event,
+      boolean ignoreDeath) {
     try {
       getZkExt().createOnceExt(finishedWorkerPath,
           null,
@@ -1339,6 +1326,7 @@ public class BspServiceMaster<I extends WritableComparable,
     final int defaultTaskTimeoutMsec = 10 * 60 * 1000;  // from TaskTracker
     final int taskTimeoutMsec = getContext().getConfiguration().getInt(
         "mapred.task.timeout", defaultTaskTimeoutMsec);
+    List<WorkerInfo> deadWorkers = new ArrayList<>();
     while (true) {
       try {
         finishedHostnameIdList =
@@ -1389,6 +1377,15 @@ public class BspServiceMaster<I extends WritableComparable,
         break;
       }
 
+      for (WorkerInfo deadWorker : deadWorkers) {
+        if (!finishedHostnameIdList.contains(deadWorker.getHostnameId())) {
+          LOG.error("barrierOnWorkerList: no results arived from " +
+              "worker that was pronounced dead: " + deadWorker +
+              " on superstep " + getSuperstep());
+          return false;
+        }
+      }
+
       // Wait for a signal or timeout
       event.waitMsecs(taskTimeoutMsec / 2);
       event.reset();
@@ -1396,9 +1393,13 @@ public class BspServiceMaster<I extends WritableComparable,
 
       // Did a worker die?
       try {
-        if (!superstepChosenWorkerAlive(
+        deadWorkers.addAll(superstepChosenWorkerAlive(
                 workerInfoHealthyPath,
-                workerInfoList)) {
+                workerInfoList));
+        if (!ignoreDeath && deadWorkers.size() > 0) {
+          LOG.error("barrierOnWorkerList: Missing chosen " +
+              "workers " + deadWorkers +
+              " on superstep " + getSuperstep());
           return false;
         }
       } catch (KeeperException e) {
@@ -1462,7 +1463,8 @@ public class BspServiceMaster<I extends WritableComparable,
     String logPrefix = "coordinate" + inputSplitsType + "InputSplits";
     if (!barrierOnWorkerList(inputSplitPaths.getDonePath(),
         chosenWorkerInfoList,
-        inputSplitEvents.getDoneStateChanged())) {
+        inputSplitEvents.getDoneStateChanged(),
+        false)) {
       throw new IllegalStateException(logPrefix + ": Worker failed during " +
           "input split (currently not supported)");
     }
@@ -1589,14 +1591,15 @@ public class BspServiceMaster<I extends WritableComparable,
 
     // Finalize the valid checkpoint file prefixes and possibly
     // the aggregators.
-    if (checkpointFrequencyMet(getSuperstep())) {
+    if (checkpointStatus != CheckpointStatus.NONE) {
       String workerWroteCheckpointPath =
           getWorkerWroteCheckpointPath(getApplicationAttempt(),
               getSuperstep());
       // first wait for all the workers to write their checkpoint data
       if (!barrierOnWorkerList(workerWroteCheckpointPath,
           chosenWorkerInfoList,
-          getWorkerWroteCheckpointEvent())) {
+          getWorkerWroteCheckpointEvent(),
+          checkpointStatus == CheckpointStatus.CHECKPOINT_AND_HALT)) {
         return SuperstepState.WORKER_FAILURE;
       }
       try {
@@ -1606,6 +1609,9 @@ public class BspServiceMaster<I extends WritableComparable,
             "coordinateSuperstep: IOException on finalizing checkpoint",
             e);
       }
+      if (checkpointStatus == CheckpointStatus.CHECKPOINT_AND_HALT) {
+        return SuperstepState.CHECKPOINT_AND_HALT;
+      }
     }
 
     if (getSuperstep() == INPUT_SUPERSTEP) {
@@ -1630,7 +1636,8 @@ public class BspServiceMaster<I extends WritableComparable,
         getWorkerFinishedPath(getApplicationAttempt(), getSuperstep());
     if (!barrierOnWorkerList(finishedWorkerPath,
         chosenWorkerInfoList,
-        getSuperstepStateChangedEvent())) {
+        getSuperstepStateChangedEvent(),
+        false)) {
       return SuperstepState.WORKER_FAILURE;
     }
 
@@ -1677,10 +1684,14 @@ public class BspServiceMaster<I extends WritableComparable,
     }
     getConfiguration().updateSuperstepClasses(superstepClasses);
 
+    //Signal workers that we want to checkpoint
+    checkpointStatus = getCheckpointStatus(getSuperstep() + 1);
+    globalStats.setCheckpointStatus(checkpointStatus);
     // Let everyone know the aggregated application state through the
     // superstep finishing znode.
     String superstepFinishedNode =
         getSuperstepFinishedPath(getApplicationAttempt(), getSuperstep());
+
     WritableUtils.writeToZnode(
         getZkExt(), superstepFinishedNode, -1, globalStats, superstepClasses);
     updateCounters(globalStats);
@@ -1703,6 +1714,43 @@ public class BspServiceMaster<I extends WritableComparable,
   }
 
   /**
+   * Should checkpoint on this superstep?  If checkpointing, always
+   * checkpoint the first user superstep.  If restarting, the first
+   * checkpoint is after the frequency has been met.
+   *
+   * @param superstep Decide if checkpointing no this superstep
+   * @return True if this superstep should be checkpointed, false otherwise
+   */
+  private CheckpointStatus getCheckpointStatus(long superstep) {
+    try {
+      if (getZkExt().
+          exists(basePath + FORCE_CHECKPOINT_USER_FLAG, false) != null) {
+        return CheckpointStatus.CHECKPOINT_AND_HALT;
+      }
+    } catch (KeeperException e) {
+      throw new IllegalStateException(
+          "cleanupZooKeeper: Got KeeperException", e);
+    } catch (InterruptedException e) {
+      throw new IllegalStateException(
+          "cleanupZooKeeper: Got IllegalStateException", e);
+    }
+    if (checkpointFrequency == 0) {
+      return CheckpointStatus.NONE;
+    }
+    long firstCheckpoint = INPUT_SUPERSTEP + 1;
+    if (getRestartedSuperstep() != UNSET_SUPERSTEP) {
+      firstCheckpoint = getRestartedSuperstep() + checkpointFrequency;
+    }
+    if (superstep < firstCheckpoint) {
+      return CheckpointStatus.NONE;
+    }
+    if (((superstep - firstCheckpoint) % checkpointFrequency) == 0) {
+      return CheckpointStatus.CHECKPOINT;
+    }
+    return CheckpointStatus.NONE;
+  }
+
+  /**
    * This doMasterCompute is only called
    * after masterCompute is initialized
    */
@@ -1837,7 +1885,7 @@ public class BspServiceMaster<I extends WritableComparable,
   }
 
   @Override
-  public void cleanup() throws IOException {
+  public void cleanup(SuperstepState superstepState) throws IOException {
     ImmutableClassesGiraphConfiguration conf = getConfiguration();
 
     // All master processes should denote they are done by adding special
@@ -1872,7 +1920,8 @@ public class BspServiceMaster<I extends WritableComparable,
       getGraphTaskManager().setIsMaster(true);
       cleanUpZooKeeper();
       // If desired, cleanup the checkpoint directory
-      if (GiraphConstants.CLEANUP_CHECKPOINTS_AFTER_SUCCESS.get(conf)) {
+      if (superstepState == SuperstepState.ALL_SUPERSTEPS_DONE &&
+          GiraphConstants.CLEANUP_CHECKPOINTS_AFTER_SUCCESS.get(conf)) {
         boolean success =
             getFs().delete(new Path(checkpointBasePath), true);
         if (LOG.isInfoEnabled()) {
@@ -1882,6 +1931,12 @@ public class BspServiceMaster<I extends WritableComparable,
               " succeeded ");
         }
       }
+      if (superstepState == SuperstepState.CHECKPOINT_AND_HALT) {
+        getFs().create(CheckpointingUtils.getCheckpointMarkPath(conf,
+            getJobId()), true);
+        failJob(new Exception("Checkpoint and halt requested. " +
+            "Killing this job."));
+      }
       aggregatorHandler.close();
       masterClient.closeConnections();
       masterServer.close();

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
index 0635210..8e4e0b8 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
@@ -96,6 +96,8 @@ public class MasterThread<I extends WritableComparable, V extends Writable,
       long initializeMillis = 0;
       long endMillis = 0;
       bspServiceMaster.setup();
+      SuperstepState superstepState = SuperstepState.INITIAL;
+
       if (bspServiceMaster.becomeMaster()) {
         // First call to checkWorkers waits for all pending resources.
         // If these resources are still available at subsequent calls it just
@@ -113,11 +115,9 @@ public class MasterThread<I extends WritableComparable, V extends Writable,
           long setupMillis = System.currentTimeMillis() - initializeMillis;
           GiraphTimers.getInstance().getSetupMs().increment(setupMillis);
           setupSecs = setupMillis / 1000.0d;
-          SuperstepState superstepState = SuperstepState.INITIAL;
-          long cachedSuperstep = BspService.UNSET_SUPERSTEP;
-          while (superstepState != SuperstepState.ALL_SUPERSTEPS_DONE) {
+          while (!superstepState.isExecutionComplete()) {
             long startSuperstepMillis = System.currentTimeMillis();
-            cachedSuperstep = bspServiceMaster.getSuperstep();
+            long cachedSuperstep = bspServiceMaster.getSuperstep();
             GiraphMetrics.get().resetSuperstepMetrics(cachedSuperstep);
             Class<? extends Computation> computationClass =
                 bspServiceMaster.getMasterCompute().getComputation();
@@ -153,7 +153,7 @@ public class MasterThread<I extends WritableComparable, V extends Writable,
           bspServiceMaster.setJobState(ApplicationState.FINISHED, -1, -1);
         }
       }
-      bspServiceMaster.cleanup();
+      bspServiceMaster.cleanup(superstepState);
       if (!superstepSecsMap.isEmpty()) {
         GiraphTimers.getInstance().getShutdownMs().
           increment(System.currentTimeMillis() - endMillis);

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
new file mode 100644
index 0000000..11d5e4f
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
@@ -0,0 +1,62 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.giraph.conf.GiraphConstants.CHECKPOINT_DIRECTORY;
+
+/**
+ * Holds useful functions to get checkpoint paths
+ * in hdfs.
+ */
+public class CheckpointingUtils {
+
+  /**
+   * Do not call constructor.
+   */
+  private CheckpointingUtils() {
+  }
+
+  /**
+   * Path to the checkpoint's root (including job id)
+   * @param conf Immutable configuration of the job
+   * @param jobId job ID
+   * @return checkpoint's root
+   */
+  public static String getCheckpointBasePath(Configuration conf,
+                                             String jobId) {
+    return CHECKPOINT_DIRECTORY.getWithDefault(conf,
+        CHECKPOINT_DIRECTORY.getDefaultValue() + "/" + jobId);
+  }
+
+  /**
+   * Path to checkpoint&halt node in hdfs.
+   * It is set to let client know that master has
+   * successfully finished checkpointing and job can be restarted.
+   * @param conf Immutable configuration of the job
+   * @param jobId job ID
+   * @return path to checkpoint&halt node in hdfs.
+   */
+  public static Path getCheckpointMarkPath(Configuration conf,
+                                           String jobId) {
+    return new Path(getCheckpointBasePath(conf, jobId), "halt");
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 763f59d..3c5cbad 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -676,4 +676,67 @@ public class WritableUtils {
       return null;
     }
   }
+
+  /**
+   * Writes a list of Writable objects into output stream.
+   * This method is trying to optimize space occupied by class information only
+   * storing class object if it is different from the previous one
+   * as in most cases arrays tend to have objects of the same type inside.
+   * @param list serialized object
+   * @param output the output stream
+   * @throws IOException
+   */
+  public static void writeList(List<Writable> list, DataOutput output)
+    throws IOException {
+    output.writeInt(list.size());
+    Class<? extends Writable> clazz = null;
+    for (Writable element : list) {
+      output.writeBoolean(element == null);
+      if (element != null) {
+        if (element.getClass() != clazz) {
+          clazz = element.getClass();
+          output.writeBoolean(true);
+          writeClass(clazz, output);
+        } else {
+          output.writeBoolean(false);
+        }
+        element.write(output);
+      }
+    }
+  }
+
+  /**
+   * Reads list of Writable objects from data input stream.
+   * Input stream should have class information along with object data.
+   * @param input input stream
+   * @return deserialized list
+   * @throws IOException
+   */
+  public static List<Writable> readList(DataInput input) throws IOException {
+    try {
+
+      int size = input.readInt();
+      List<Writable> res = new ArrayList<>(size);
+      Class<? extends Writable> clazz = null;
+      for (int i = 0; i < size; i++) {
+        boolean isNull = input.readBoolean();
+        if (isNull) {
+          res.add(null);
+        } else {
+          boolean hasClassInfo = input.readBoolean();
+          if (hasClassInfo) {
+            clazz = readClass(input);
+          }
+          Writable element = clazz.newInstance();
+          element.readFields(input);
+          res.add(element);
+        }
+      }
+      return res;
+
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalStateException("unable to instantiate object", e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index d2d24ee..447bb6f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -21,6 +21,7 @@ package org.apache.giraph.worker;
 import org.apache.giraph.bsp.ApplicationState;
 import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.bsp.CheckpointStatus;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
@@ -540,7 +541,8 @@ public class BspServiceWorker<I extends WritableComparable,
     // 6. Wait for superstep INPUT_SUPERSTEP to complete.
     if (getRestartedSuperstep() != UNSET_SUPERSTEP) {
       setCachedSuperstep(getRestartedSuperstep());
-      return new FinishedSuperstepStats(0, false, 0, 0, true);
+      return new FinishedSuperstepStats(0, false, 0, 0, true,
+          CheckpointStatus.NONE);
     }
 
     JSONObject jobState = getJobState();
@@ -557,7 +559,8 @@ public class BspServiceWorker<I extends WritableComparable,
                 getApplicationAttempt());
           }
           setRestartedSuperstep(getSuperstep());
-          return new FinishedSuperstepStats(0, false, 0, 0, true);
+          return new FinishedSuperstepStats(0, false, 0, 0, true,
+              CheckpointStatus.NONE);
         }
       } catch (JSONException e) {
         throw new RuntimeException(
@@ -946,7 +949,8 @@ public class BspServiceWorker<I extends WritableComparable,
         globalStats.getHaltComputation(),
         globalStats.getVertexCount(),
         globalStats.getEdgeCount(),
-        false);
+        false,
+        globalStats.getCheckpointStatus());
   }
 
   /**
@@ -1314,8 +1318,11 @@ public class BspServiceWorker<I extends WritableComparable,
     throws IOException, InterruptedException {
     workerClient.closeConnections();
     setCachedSuperstep(getSuperstep() - 1);
-    saveVertices(finishedSuperstepStats.getLocalVertexCount());
-    saveEdges();
+    if (finishedSuperstepStats.getCheckpointStatus() !=
+        CheckpointStatus.CHECKPOINT_AND_HALT) {
+      saveVertices(finishedSuperstepStats.getLocalVertexCount());
+      saveEdges();
+    }
     WorkerProgress.get().finishStoring();
     if (workerProgressWriter != null) {
       workerProgressWriter.stop();
@@ -1414,6 +1421,10 @@ public class BspServiceWorker<I extends WritableComparable,
 
     }
 
+    List<Writable> w2wMessages =
+        getServerData().getCurrentWorkerToWorkerMessages();
+    WritableUtils.writeList(w2wMessages, checkpointOutputStream);
+
     checkpointOutputStream.close();
 
     getFs().createNewFile(validFilePath);
@@ -1488,9 +1499,9 @@ public class BspServiceWorker<I extends WritableComparable,
 
     final CompressionCodec codec =
         new CompressionCodecFactory(getConfiguration())
-            .getCodecByClassName(
+            .getCodec(new Path(
                 GiraphConstants.CHECKPOINT_COMPRESSION_CODEC
-                    .get(getConfiguration()));
+                    .get(getConfiguration())));
 
     long t0 = System.currentTimeMillis();
 
@@ -1559,9 +1570,9 @@ public class BspServiceWorker<I extends WritableComparable,
 
     final CompressionCodec codec =
         new CompressionCodecFactory(getConfiguration())
-            .getCodecByClassName(
+            .getCodec(new Path(
                 GiraphConstants.CHECKPOINT_COMPRESSION_CODEC
-                    .get(getConfiguration()));
+                    .get(getConfiguration())));
 
     long t0 = System.currentTimeMillis();
 
@@ -1660,6 +1671,10 @@ public class BspServiceWorker<I extends WritableComparable,
         getServerData().getCurrentMessageStore().readFieldsForPartition(
             checkpointStream, partitionId);
       }
+
+      List<Writable> w2wMessages = WritableUtils.readList(checkpointStream);
+      getServerData().getCurrentWorkerToWorkerMessages().addAll(w2wMessages);
+
       checkpointStream.close();
 
       if (LOG.isInfoEnabled()) {
@@ -1920,4 +1935,18 @@ else[HADOOP_NON_SECURE]*/
   public SuperstepOutput<I, V, E> getSuperstepOutput() {
     return superstepOutput;
   }
+
+  @Override
+  public GlobalStats getGlobalStats() {
+    GlobalStats globalStats = new GlobalStats();
+    if (getSuperstep() > Math.max(INPUT_SUPERSTEP, getRestartedSuperstep())) {
+      String superstepFinishedNode =
+          getSuperstepFinishedPath(getApplicationAttempt(),
+              getSuperstep() - 1);
+      WritableUtils.readFieldsFromZnode(
+          getZkExt(), superstepFinishedNode, false, null,
+          globalStats);
+    }
+    return globalStats;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java b/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
new file mode 100644
index 0000000..c712b5a
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
@@ -0,0 +1,70 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test case for WritableUtils.
+ */
+public class TestWritableUtils {
+
+  /**
+   * Tests readList and writeList functions in writable utils.
+   * @throws IOException
+   */
+  @Test
+  public void testListSerialization() throws IOException {
+    List<Writable> list = new ArrayList<>();
+    list.add(new LongWritable(1));
+    list.add(new LongWritable(2));
+    list.add(null);
+    list.add(new FloatWritable(3));
+    list.add(new FloatWritable(4));
+    list.add(new LongWritable(5));
+    list.add(new LongWritable(6));
+
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(bos);
+    WritableUtils.writeList(list, dos);
+    dos.close();
+
+    byte[] data = bos.toByteArray();
+
+    DataInputStream input =
+        new DataInputStream(new ByteArrayInputStream(data));
+
+    List<Writable> result = WritableUtils.readList(input);
+
+    Assert.assertEquals(list, result);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java b/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
index 2939af7..9502557 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java
@@ -19,8 +19,10 @@
 package org.apache.giraph;
 
 import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.examples.SimpleSuperstepComputation;
@@ -29,17 +31,25 @@ import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.worker.DefaultWorkerContext;
+import org.apache.giraph.zk.ZooKeeperExt;
+import org.apache.giraph.zk.ZooKeeperManager;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -56,13 +66,8 @@ public class TestCheckpointing extends BspCase {
       Logger.getLogger(TestCheckpointing.class);
   /** ID to be used with test job */
   public static final String TEST_JOB_ID = "test_job";
-  /**
-   * Compute will double check that we don't run supersteps
-   * lesser than specified by this key. That way we ensure that
-   * computation actually restarted and not recalculated from the
-   * beginning.
-   */
-  public static final String KEY_MIN_SUPERSTEP = "minimum.superstep";
+
+  private static SuperstepCallback SUPERSTEP_CALLBACK;
 
   /**
    * Create the test case
@@ -84,49 +89,45 @@ public class TestCheckpointing extends BspCase {
   public void testBspCheckpoint(boolean useAsyncMessageStore)
       throws IOException, InterruptedException, ClassNotFoundException {
     Path checkpointsDir = getTempPath("checkpointing");
-    Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setComputationClass(
-        CheckpointComputation.class);
-    conf.setWorkerContextClass(
-        CheckpointVertexWorkerContext.class);
-    conf.setMasterComputeClass(
-        CheckpointVertexMasterCompute.class);
-    conf.setVertexInputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat.class);
-    conf.setVertexOutputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat.class);
-    conf.set("mapred.job.id", TEST_JOB_ID);
-    conf.set(KEY_MIN_SUPERSTEP, "0");
     if (useAsyncMessageStore) {
       GiraphConstants.ASYNC_MESSAGE_STORE_THREADS_COUNT.set(conf, 2);
     }
-    GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
 
-    GiraphConfiguration configuration = job.getConfiguration();
-    GiraphConstants.CHECKPOINT_DIRECTORY.set(configuration, checkpointsDir.toString());
-    GiraphConstants.CLEANUP_CHECKPOINTS_AFTER_SUCCESS.set(configuration, false);
-    configuration.setCheckpointFrequency(2);
+    SUPERSTEP_CALLBACK = null;
 
-    assertTrue(job.run(true));
+    GiraphConstants.CLEANUP_CHECKPOINTS_AFTER_SUCCESS.set(conf, false);
+    conf.setCheckpointFrequency(2);
 
-    long idSum = 0;
-    if (!runningInDistributedMode()) {
-      FileStatus fileStatus = getSinglePartFileStatus(job.getConfiguration(),
-          outputPath);
-      idSum = CheckpointVertexWorkerContext
-          .getFinalSum();
-      LOG.info("testBspCheckpoint: idSum = " + idSum +
-          " fileLen = " + fileStatus.getLen());
-    }
+    long idSum = runOriginalJob(checkpointsDir, conf);
+    assertEquals(10, idSum);
+
+    SUPERSTEP_CALLBACK = new SuperstepCallback() {
+      @Override
+      public void superstep(long superstep,
+                            ImmutableClassesGiraphConfiguration<LongWritable, IntWritable, FloatWritable> conf) {
+        if (superstep < 2) {
+          Assert.fail("Restarted JOB should not be executed on superstep " + superstep);
+        }
+      }
+    };
+
+    runRestartedJob(checkpointsDir, conf, idSum, 2);
+
+
+  }
 
-    // Restart the test from superstep 2
-    LOG.info("testBspCheckpoint: Restarting from superstep 2" +
-        " with checkpoint path = " + checkpointsDir);
+  private void runRestartedJob(Path checkpointsDir, GiraphConfiguration conf, long idSum, long restartFrom) throws IOException, InterruptedException, ClassNotFoundException {
+    Path outputPath;
+    LOG.info("testBspCheckpoint: Restarting from the latest superstep " +
+        "with checkpoint path = " + checkpointsDir);
     outputPath = getTempPath("checkpointing_restarted");
 
     GiraphConstants.RESTART_JOB_ID.set(conf, TEST_JOB_ID);
     conf.set("mapred.job.id", "restarted_test_job");
-    conf.set(GiraphConstants.RESTART_SUPERSTEP, "2");
-    conf.set(KEY_MIN_SUPERSTEP, "2");
+    if (restartFrom >= 0) {
+      conf.set(GiraphConstants.RESTART_SUPERSTEP, Long.toString(restartFrom));
+    }
 
     GiraphJob restartedJob = prepareJob(getCallingMethodName() + "Restarted",
         conf, outputPath);
@@ -135,6 +136,8 @@ public class TestCheckpointing extends BspCase {
         checkpointsDir.toString());
 
     assertTrue(restartedJob.run(true));
+
+
     if (!runningInDistributedMode()) {
       long idSumRestarted =
           CheckpointVertexWorkerContext
@@ -145,6 +148,36 @@ public class TestCheckpointing extends BspCase {
     }
   }
 
+  private long runOriginalJob(Path checkpointsDir,  GiraphConfiguration conf) throws IOException, InterruptedException, ClassNotFoundException {
+    Path outputPath = getTempPath("checkpointing_original");
+    conf.setComputationClass(
+        CheckpointComputation.class);
+    conf.setWorkerContextClass(
+        CheckpointVertexWorkerContext.class);
+    conf.setMasterComputeClass(
+        CheckpointVertexMasterCompute.class);
+    conf.setVertexInputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat.class);
+    conf.setVertexOutputFormatClass(SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat.class);
+    conf.set("mapred.job.id", TEST_JOB_ID);
+    GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
+
+    GiraphConfiguration configuration = job.getConfiguration();
+    GiraphConstants.CHECKPOINT_DIRECTORY.set(configuration, checkpointsDir.toString());
+
+    assertTrue(job.run(true));
+
+    long idSum = 0;
+    if (!runningInDistributedMode()) {
+      FileStatus fileStatus = getSinglePartFileStatus(job.getConfiguration(),
+          outputPath);
+      idSum = CheckpointVertexWorkerContext
+          .getFinalSum();
+      LOG.info("testBspCheckpoint: idSum = " + idSum +
+          " fileLen = " + fileStatus.getLen());
+    }
+    return idSum;
+  }
+
 
   /**
    * Actual computation.
@@ -159,10 +192,6 @@ public class TestCheckpointing extends BspCase {
       CheckpointVertexWorkerContext workerContext = getWorkerContext();
       assertEquals(getSuperstep() + 1, workerContext.testValue);
 
-      if (getSuperstep() < getConf().getInt(KEY_MIN_SUPERSTEP, Integer.MAX_VALUE)){
-        fail("Should not be running compute on superstep " + getSuperstep());
-      }
-
       if (getSuperstep() > 4) {
         vertex.voteToHalt();
         return;
@@ -186,10 +215,76 @@ public class TestCheckpointing extends BspCase {
             EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
         vertex.addEdge(newEdge);
         sendMessage(edge.getTargetVertexId(), newEdgeValue);
+
       }
     }
   }
 
+  @Test
+  public void testManualCheckpointAtTheBeginning()
+      throws InterruptedException, IOException, ClassNotFoundException {
+    testManualCheckpoint(0);
+  }
+
+  @Test
+  public void testManualCheckpoint()
+      throws InterruptedException, IOException, ClassNotFoundException {
+    testManualCheckpoint(2);
+  }
+
+
+  private void testManualCheckpoint(final int checkpointSuperstep)
+      throws IOException, InterruptedException, ClassNotFoundException {
+    Path checkpointsDir = getTempPath("checkpointing");
+    GiraphConfiguration conf = new GiraphConfiguration();
+
+    SUPERSTEP_CALLBACK = new SuperstepCallback() {
+
+      @Override
+      public void superstep(long superstep, ImmutableClassesGiraphConfiguration<LongWritable, IntWritable, FloatWritable> conf) {
+        if (superstep == checkpointSuperstep) {
+          try {
+            ZooKeeperExt zooKeeperExt = new ZooKeeperExt(conf.getZookeeperList(),
+                conf.getZooKeeperSessionTimeout(),
+                conf.getZookeeperOpsMaxAttempts(),
+                conf.getZookeeperOpsRetryWaitMsecs(),
+                TestCheckpointing.this);
+            String basePath = ZooKeeperManager.getBasePath(conf) + BspService.BASE_DIR + "/" + conf.get("mapred.job.id");
+            zooKeeperExt.createExt(
+                basePath + BspService.FORCE_CHECKPOINT_USER_FLAG,
+                null,
+                ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT,
+                true);
+          } catch (IOException | InterruptedException | KeeperException e) {
+            throw new RuntimeException(e);
+          }
+        } else if (superstep > checkpointSuperstep) {
+          Assert.fail("Job should be stopped by now " + superstep);
+        }
+      }
+    };
+
+    try {
+      runOriginalJob(checkpointsDir, conf);
+      fail("Original job should fail after checkpointing");
+    } catch (Exception e) {
+      LOG.info("Original job failed, that's OK " + e);
+    }
+
+    SUPERSTEP_CALLBACK = new SuperstepCallback() {
+      @Override
+      public void superstep(long superstep,
+                            ImmutableClassesGiraphConfiguration<LongWritable, IntWritable, FloatWritable> conf) {
+        if (superstep < checkpointSuperstep) {
+          Assert.fail("Restarted JOB should not be executed on superstep " + superstep);
+        }
+      }
+    };
+
+    runRestartedJob(checkpointsDir, conf, 10, -1);
+  }
+
   /**
    * Worker context associated.
    */
@@ -205,6 +300,21 @@ public class TestCheckpointing extends BspCase {
     }
 
     @Override
+    public void postSuperstep() {
+      super.postSuperstep();
+      sendMessageToMyself(new LongWritable(getSuperstep()));
+    }
+
+    /**
+     * Send message to all workers (except this worker)
+     *
+     * @param message Message to send
+     */
+    private void sendMessageToMyself(Writable message) {
+      sendMessageToWorker(message, getMyWorkerIndex());
+    }
+
+    @Override
     public void postApplication() {
       setFinalSum(this.<LongWritable>getAggregatedValue(
           LongSumAggregator.class.getName()).get());
@@ -223,6 +333,11 @@ public class TestCheckpointing extends BspCase {
     @Override
     public void preSuperstep() {
       assertEquals(getSuperstep(), testValue++);
+      if (getSuperstep() > 0) {
+        List<Writable> messages = getAndClearMessagesFromOtherWorkers();
+        assertEquals(1, messages.size());
+        assertEquals(getSuperstep() - 1, ((LongWritable)(messages.get(0))).get());
+      }
     }
 
     @Override
@@ -249,6 +364,9 @@ public class TestCheckpointing extends BspCase {
     @Override
     public void compute() {
       long superstep = getSuperstep();
+      if (SUPERSTEP_CALLBACK != null) {
+        SUPERSTEP_CALLBACK.superstep(getSuperstep(), getConf());
+      }
       assertEquals(superstep, testValue++);
     }
 
@@ -272,6 +390,12 @@ public class TestCheckpointing extends BspCase {
     }
   }
 
+  private static interface SuperstepCallback {
 
+    public void superstep(long superstep,
+                          ImmutableClassesGiraphConfiguration<LongWritable,
+                              IntWritable, FloatWritable> conf);
+
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/5adca63d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 672ec44..b4d78ae 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1059,7 +1059,7 @@ under the License.
       </modules>
       <properties>
         <hadoop.version>0.20.0</hadoop.version>
-        <munge.symbols>HADOOP_NON_SECURE,HADOOP_NON_JOBCONTEXT_IS_INTERFACE</munge.symbols>
+        <munge.symbols>HADOOP_NON_SECURE,HADOOP_NON_JOBCONTEXT_IS_INTERFACE,HADOOP_JOB_ID_AVAILABLE</munge.symbols>
       </properties>
       <dependencies>
         <!-- sorted lexicographically -->


[44/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Fix using aggregators before aggregation

Summary:
If we register aggregator and immediatelly ask for aggregated value,
previous code was returning initial value, so we have to do the same.

Additionally - cleaning up errors/exceptions to be more understandable
(vs NullPointerEx for example)

Test Plan:
mvn install
AggregatorsBenchmark and ReducersBenchmark

Reviewers: majakabiljo, pavanka, sergey.edunov, maja.kabiljo

Differential Revision: https://reviews.facebook.net/D24951


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/d32c429a
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/d32c429a
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/d32c429a

Branch: refs/heads/release-1.1
Commit: d32c429a1d475b322b3fe44738f0cc8f30a97b48
Parents: 7c61dcf
Author: Igor Kabiljo <ik...@fb.com>
Authored: Mon Oct 20 09:50:55 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Mon Oct 20 09:54:37 2014 -0700

----------------------------------------------------------------------
 .../aggregators/ArrayAggregatorFactory.java     | 128 -------------------
 .../aggregators/ClassAggregatorFactory.java     |  72 -----------
 .../giraph/benchmark/ReducersBenchmark.java     |   3 +-
 .../comm/aggregators/AggregatorUtils.java       |  51 ++++++++
 .../requests/SendAggregatorsToOwnerRequest.java |   2 +-
 .../giraph/master/AggregatorBroadcast.java      |  75 +++++++++++
 .../master/AggregatorReduceOperation.java       |  66 +++++++---
 .../AggregatorToGlobalCommTranslation.java      | 113 +++++++++++-----
 .../apache/giraph/master/BspServiceMaster.java  |   2 +-
 .../giraph/master/MasterAggregatorHandler.java  |  29 ++++-
 .../giraph/master/MasterAggregatorUsage.java    |  16 ---
 .../org/apache/giraph/master/MasterCompute.java |   9 --
 .../giraph/reducers/OnSameReduceOperation.java  |   4 +-
 .../apache/giraph/reducers/ReduceOperation.java |  10 +-
 .../org/apache/giraph/reducers/Reducer.java     |  41 ++++--
 .../apache/giraph/utils/WritableFactory.java    |  28 ----
 .../org/apache/giraph/utils/WritableUtils.java  |   6 +-
 .../worker/WorkerAggregatorDelegator.java       |   4 +-
 .../giraph/worker/WorkerAggregatorHandler.java  |   8 +-
 .../giraph/worker/WorkerBroadcastUsage.java     |  33 +++++
 .../giraph/worker/WorkerGlobalCommUsage.java    |  17 +--
 .../apache/giraph/worker/WorkerReduceUsage.java |  30 +++++
 .../giraph/aggregators/TestArrayAggregator.java |  50 --------
 23 files changed, 396 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java
deleted file mode 100644
index c977c57..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/ArrayAggregatorFactory.java
+++ /dev/null
@@ -1,128 +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.aggregators;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.lang.reflect.Array;
-
-import org.apache.giraph.utils.ArrayWritable;
-import org.apache.giraph.utils.WritableFactory;
-import org.apache.giraph.utils.WritableUtils;
-import org.apache.hadoop.io.Writable;
-
-/**
- * Generic array aggregator factory, used to aggregate elements
- * of ArrayWritable via passed element aggregator.
- *
- * @param <A> Type of individual element
- */
-public class ArrayAggregatorFactory<A extends Writable>
-    implements WritableFactory<Aggregator<ArrayWritable<A>>> {
-  /** number of elements in array */
-  private int n;
-  /** element aggregator class */
-  private WritableFactory<? extends Aggregator<A>> elementAggregatorFactory;
-
-  /**
-   * Constructor
-   * @param n Number of elements in array
-   * @param elementAggregatorClass Type of element aggregator
-   */
-  public ArrayAggregatorFactory(
-      int n, Class<? extends Aggregator<A>> elementAggregatorClass) {
-    this(n, new ClassAggregatorFactory<>(elementAggregatorClass));
-  }
-
-  /**
-   * Constructor
-   * @param n Number of elements in array
-   * @param elementAggregatorFactory Element aggregator factory
-   */
-  public ArrayAggregatorFactory(int n,
-      WritableFactory<? extends Aggregator<A>> elementAggregatorFactory) {
-    this.n = n;
-    this.elementAggregatorFactory = elementAggregatorFactory;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    n = in.readInt();
-    elementAggregatorFactory = WritableUtils.readWritableObject(in, null);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(n);
-    WritableUtils.writeWritableObject(elementAggregatorFactory, out);
-  }
-
-  @Override
-  public Aggregator<ArrayWritable<A>> create() {
-    return new ArrayAggregator<>(
-        n, elementAggregatorFactory.create());
-  }
-
-  /**
-   * Stateful aggregator that aggregates ArrayWritable by
-   * aggregating individual elements
-   *
-   * @param <A> Type of individual element
-   */
-  public static class ArrayAggregator<A extends Writable>
-      extends BasicAggregator<ArrayWritable<A>> {
-    /** number of elements in array */
-    private final int n;
-    /** element aggregator */
-    private final Aggregator<A> elementAggregator;
-
-    /**
-     * Constructor
-     * @param n Number of elements in array
-     * @param elementAggregator Element aggregator
-     */
-    public ArrayAggregator(int n, Aggregator<A> elementAggregator) {
-      super(null);
-      this.n = n;
-      this.elementAggregator = elementAggregator;
-      reset();
-    }
-
-    @Override
-    public void aggregate(ArrayWritable<A> other) {
-      A[] array = getAggregatedValue().get();
-      for (int i = 0; i < n; i++) {
-        elementAggregator.setAggregatedValue(array[i]);
-        elementAggregator.aggregate(other.get()[i]);
-        array[i] = elementAggregator.getAggregatedValue();
-      }
-    }
-
-    @Override
-    public ArrayWritable<A> createInitialValue() {
-      Class<A> elementClass =
-          (Class) elementAggregator.createInitialValue().getClass();
-      A[] array = (A[]) Array.newInstance(elementClass, n);
-      for (int i = 0; i < n; i++) {
-        array[i] = elementAggregator.createInitialValue();
-      }
-      return new ArrayWritable<>(elementClass, array);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java b/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
deleted file mode 100644
index a022480..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/aggregators/ClassAggregatorFactory.java
+++ /dev/null
@@ -1,72 +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.aggregators;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.giraph.utils.ReflectionUtils;
-import org.apache.giraph.utils.WritableFactory;
-import org.apache.giraph.utils.WritableUtils;
-import org.apache.hadoop.io.Writable;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Aggregator factory based on aggregatorClass.
- *
- * @param <T> Aggregated value type
- */
-public class ClassAggregatorFactory<T extends Writable>
-    implements WritableFactory<Aggregator<T>> {
-  /** Aggregator class */
-  private Class<? extends Aggregator<T>> aggregatorClass;
-
-  /** Constructor */
-  public ClassAggregatorFactory() {
-  }
-
-  /**
-   * Constructor
-   * @param aggregatorClass Aggregator class
-   */
-  public ClassAggregatorFactory(
-      Class<? extends Aggregator<T>> aggregatorClass) {
-    Preconditions.checkNotNull(aggregatorClass,
-        "aggregatorClass cannot be null in ClassAggregatorFactory");
-    this.aggregatorClass = aggregatorClass;
-  }
-
-  @Override
-  public Aggregator<T> create() {
-    return ReflectionUtils.newInstance(aggregatorClass, null);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    aggregatorClass = WritableUtils.readClass(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Preconditions.checkNotNull(aggregatorClass,
-        "aggregatorClass cannot be null in ClassAggregatorFactory");
-    WritableUtils.writeClass(aggregatorClass, out);
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
index ce5c96e..263274d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/ReducersBenchmark.java
@@ -64,9 +64,10 @@ public class ReducersBenchmark extends GiraphBenchmark {
     }
 
     @Override
-    public void reduceSingle(
+    public LongWritable reduceSingle(
         LongWritable curValue, LongWritable valueToReduce) {
       curValue.set(curValue.get() + valueToReduce.get());
+      return curValue;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
index ecb3a6b..dc0ceed 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
@@ -99,4 +99,55 @@ public class AggregatorUtils {
     }
     return message;
   }
+
+  /**
+   * Get the warning message about usage of unregistered reducer to be
+   * printed to user. If user didn't register any reducers also provide
+   * the explanation on how to do so.
+   *
+   * @param reducerName The name of the aggregator which user tried to
+   *                       access
+   * @param hasRegisteredReducers True iff user registered some aggregators
+   * @param conf Giraph configuration
+   * @return Warning message
+   */
+  public static String getUnregisteredReducerMessage(
+      String reducerName, boolean hasRegisteredReducers,
+      ImmutableClassesGiraphConfiguration conf) {
+    String message = "Tried to access reducer which wasn't registered " +
+        reducerName;
+    if (!hasRegisteredReducers) {
+      message = message + "; Aggregators can be registered from " +
+          "MasterCompute by calling registerReducer function. " +
+          "Also be sure that you are correctly setting MasterCompute class, " +
+          "currently using " + conf.getMasterComputeClass().getName();
+    }
+    return message;
+  }
+
+  /**
+   * Get the warning message when user tries to access broadcast, without
+   * previously setting it, to be printed to user.
+   * If user didn't broadcast any value also provide
+   * the explanation on how to do so.
+   *
+   * @param broadcastName The name of the broadcast which user tried to
+   *                       access
+   * @param hasBroadcasted True iff user has broadcasted value before
+   * @param conf Giraph configuration
+   * @return Warning message
+   */
+  public static String getUnregisteredBroadcastMessage(
+      String broadcastName, boolean hasBroadcasted,
+      ImmutableClassesGiraphConfiguration conf) {
+    String message = "Tried to access broadcast which wasn't set before " +
+        broadcastName;
+    if (!hasBroadcasted) {
+      message = message + "; Values can be broadcasted from " +
+          "MasterCompute by calling broadcast function. " +
+          "Also be sure that you are correctly setting MasterCompute class, " +
+          "currently using " + conf.getMasterComputeClass().getName();
+    }
+    return message;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/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 2d5cc51..8f168a2 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
@@ -77,7 +77,7 @@ public class SendAggregatorsToOwnerRequest
           if (type == GlobalCommType.REDUCE_OPERATIONS) {
             ReduceOperation<Object, Writable> reduceOpCopy =
                 (ReduceOperation<Object, Writable>)
-                WritableUtils.createCopy(reusedOut, reusedIn, value);
+                WritableUtils.createCopy(reusedOut, reusedIn, value, conf);
 
             serverData.getOwnerAggregatorData().registerReducer(
                 name, reduceOpCopy);

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorBroadcast.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorBroadcast.java b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorBroadcast.java
new file mode 100644
index 0000000..81ea654
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorBroadcast.java
@@ -0,0 +1,75 @@
+/*
+ * 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.master;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Writable representation of aggregated value
+ *
+ * @param <A> Aggregation object type
+ */
+public class AggregatorBroadcast<A extends Writable>
+  extends DefaultImmutableClassesGiraphConfigurable
+  implements Writable {
+  /** Aggregator class */
+  private Class<? extends Aggregator<A>> aggregatorClass;
+  /** Aggregated value */
+  private A value;
+
+  /** Constructor */
+  public AggregatorBroadcast() {
+  }
+
+  /**
+   * Constructor
+   * @param aggregatorClass Aggregator class
+   * @param value Aggregated value
+   */
+  public AggregatorBroadcast(
+      Class<? extends Aggregator<A>> aggregatorClass, A value) {
+    this.aggregatorClass = aggregatorClass;
+    this.value = value;
+  }
+
+  public A getValue() {
+    return value;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeClass(aggregatorClass, out);
+    value.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    aggregatorClass = WritableUtils.readClass(in);
+    value = ReflectionUtils.newInstance(aggregatorClass, getConf())
+        .createInitialValue();
+    value.readFields(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
index 1673f6d..54d421b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorReduceOperation.java
@@ -22,8 +22,10 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.conf.GiraphConfigurationSettable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.reducers.OnSameReduceOperation;
-import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 
@@ -33,11 +35,13 @@ import org.apache.hadoop.io.Writable;
  * @param <A> Aggregation object type
  */
 public class AggregatorReduceOperation<A extends Writable>
-    extends OnSameReduceOperation<A> {
-  /** Aggregator factory */
-  private WritableFactory<? extends Aggregator<A>> aggregatorFactory;
+    extends OnSameReduceOperation<A> implements GiraphConfigurationSettable {
+  /** Aggregator class */
+  private Class<? extends Aggregator<A>> aggregatorClass;
   /** Aggregator */
   private Aggregator<A> aggregator;
+  /** Configuration */
+  private ImmutableClassesGiraphConfiguration<?, ?, ?> conf;
 
   /** Constructor */
   public AggregatorReduceOperation() {
@@ -45,18 +49,32 @@ public class AggregatorReduceOperation<A extends Writable>
 
   /**
    * Constructor
-   * @param aggregatorFactory Aggregator factory
+   * @param aggregatorClass Aggregator class
+   * @param conf Configuration
    */
   public AggregatorReduceOperation(
-      WritableFactory<? extends Aggregator<A>> aggregatorFactory) {
-    this.aggregatorFactory = aggregatorFactory;
-    this.aggregator = aggregatorFactory.create();
-    this.aggregator.setAggregatedValue(null);
+      Class<? extends Aggregator<A>> aggregatorClass,
+      ImmutableClassesGiraphConfiguration<?, ?, ?> conf) {
+    this.aggregatorClass = aggregatorClass;
+    this.conf = conf;
+    initAggregator();
+  }
+
+  /** Initialize aggregator */
+  private void initAggregator() {
+    aggregator = ReflectionUtils.newInstance(aggregatorClass, conf);
+    aggregator.setAggregatedValue(null);
   }
 
   @Override
   public A createInitialValue() {
-    return aggregator.createInitialValue();
+    A agg = aggregator.createInitialValue();
+    if (agg == null) {
+      throw new IllegalStateException(
+          "Aggregators initial value must not be null, but is for " +
+          aggregator);
+    }
+    return agg;
   }
 
   /**
@@ -64,29 +82,37 @@ public class AggregatorReduceOperation<A extends Writable>
    * @return copy
    */
   public AggregatorReduceOperation<A> createCopy() {
-    return new AggregatorReduceOperation<>(aggregatorFactory);
+    return new AggregatorReduceOperation<>(aggregatorClass, conf);
+  }
+
+  public Class<? extends Aggregator<A>> getAggregatorClass() {
+    return aggregatorClass;
   }
 
   @Override
-  public synchronized void reduceSingle(A curValue, A valueToReduce) {
+  public synchronized A reduceSingle(A curValue, A valueToReduce) {
     aggregator.setAggregatedValue(curValue);
     aggregator.aggregate(valueToReduce);
-    if (curValue != aggregator.getAggregatedValue()) {
-      throw new IllegalStateException(
-          "Aggregator " + aggregator + " aggregates by creating new value");
-    }
+    A aggregated = aggregator.getAggregatedValue();
     aggregator.setAggregatedValue(null);
+    return aggregated;
+  }
+
+  @Override
+  public void setConf(ImmutableClassesGiraphConfiguration conf) {
+    this.conf = conf;
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
-    WritableUtils.writeWritableObject(aggregatorFactory, out);
+    WritableUtils.writeClass(aggregatorClass, out);
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    aggregatorFactory = WritableUtils.readWritableObject(in, null);
-    aggregator = aggregatorFactory.create();
-    this.aggregator.setAggregatedValue(null);
+    aggregatorClass = WritableUtils.readClass(in);
+    initAggregator();
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
index 7492fc7..36a4553 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/AggregatorToGlobalCommTranslation.java
@@ -24,10 +24,10 @@ import java.util.HashMap;
 import java.util.Map.Entry;
 
 import org.apache.giraph.aggregators.Aggregator;
-import org.apache.giraph.aggregators.ClassAggregatorFactory;
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
-import org.apache.giraph.utils.WritableFactory;
+import org.apache.giraph.comm.aggregators.AggregatorUtils;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
 
 import com.google.common.base.Preconditions;
 
@@ -36,8 +36,11 @@ import com.google.common.base.Preconditions;
  * reduce and broadcast operations supported by the MasterAggregatorHandler.
  */
 public class AggregatorToGlobalCommTranslation
-    extends DefaultImmutableClassesGiraphConfigurable
     implements MasterAggregatorUsage, Writable {
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(AggregatorToGlobalCommTranslation.class);
+
   /** Class providing reduce and broadcast interface to use */
   private final MasterGlobalCommUsage globalComm;
   /** List of registered aggregators */
@@ -45,21 +48,64 @@ public class AggregatorToGlobalCommTranslation
   registeredAggregators = new HashMap<>();
 
   /**
+   * List of init aggregator values, in case someone tries to
+   * access aggregator immediatelly after registering it.
+   *
+   * Instead of simply returning value, we need to store it during
+   * that superstep, so consecutive calls will return identical object,
+   * which they can modify.
+   */
+  private final HashMap<String, Writable>
+  initAggregatorValues = new HashMap<>();
+
+  /** Conf */
+  private final ImmutableClassesGiraphConfiguration<?, ?, ?> conf;
+
+  /**
    * Constructor
+   * @param conf Configuration
    * @param globalComm Global communication interface
    */
-  public AggregatorToGlobalCommTranslation(MasterGlobalCommUsage globalComm) {
+  public AggregatorToGlobalCommTranslation(
+      ImmutableClassesGiraphConfiguration<?, ?, ?> conf,
+      MasterGlobalCommUsage globalComm) {
+    this.conf = conf;
     this.globalComm = globalComm;
   }
 
   @Override
   public <A extends Writable> A getAggregatedValue(String name) {
-    return globalComm.getReduced(name);
+    AggregatorWrapper<Writable> agg = registeredAggregators.get(name);
+    if (agg == null) {
+      LOG.warn("getAggregatedValue: " +
+        AggregatorUtils.getUnregisteredAggregatorMessage(name,
+            registeredAggregators.size() != 0, conf));
+      // to make sure we are not accessing reducer of the same name.
+      return null;
+    }
+
+    A value = globalComm.getReduced(name);
+    if (value == null) {
+      value = (A) initAggregatorValues.get(name);
+    }
+
+    if (value == null) {
+      value = (A) agg.getReduceOp().createInitialValue();
+      initAggregatorValues.put(name, value);
+    }
+
+    Preconditions.checkState(value != null);
+    return value;
   }
 
   @Override
   public <A extends Writable> void setAggregatedValue(String name, A value) {
     AggregatorWrapper<Writable> aggregator = registeredAggregators.get(name);
+    if (aggregator == null) {
+      throw new IllegalArgumentException("setAggregatedValue: "  +
+          AggregatorUtils.getUnregisteredAggregatorMessage(name,
+              registeredAggregators.size() != 0, conf));
+    }
     aggregator.setCurrentValue(value);
   }
 
@@ -72,14 +118,15 @@ public class AggregatorToGlobalCommTranslation
     // register reduce with the same value
     for (Entry<String, AggregatorWrapper<Writable>> entry :
         registeredAggregators.entrySet()) {
-      Writable value = entry.getValue().currentValue != null ?
-          entry.getValue().getCurrentValue() :
-            globalComm.getReduced(entry.getKey());
+      Writable value = entry.getValue().getCurrentValue();
       if (value == null) {
-        value = entry.getValue().getReduceOp().createInitialValue();
+        value = globalComm.getReduced(entry.getKey());
       }
+      Preconditions.checkState(value != null);
+
+      globalComm.broadcast(entry.getKey(), new AggregatorBroadcast<>(
+          entry.getValue().getReduceOp().getAggregatorClass(), value));
 
-      globalComm.broadcast(entry.getKey(), value);
       // Always register clean instance of reduceOp, not to conflict with
       // reduceOp from previous superstep.
       AggregatorReduceOperation<Writable> cleanReduceOp =
@@ -93,31 +140,21 @@ public class AggregatorToGlobalCommTranslation
       }
       entry.getValue().setCurrentValue(null);
     }
+    initAggregatorValues.clear();
   }
 
   @Override
   public <A extends Writable> boolean registerAggregator(String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
-    ClassAggregatorFactory<A> aggregatorFactory =
-        new ClassAggregatorFactory<A>(aggregatorClass);
-    return registerAggregator(name, aggregatorFactory, false) != null;
-  }
-
-  @Override
-  public <A extends Writable> boolean registerAggregator(String name,
-      WritableFactory<? extends Aggregator<A>> aggregator) throws
-      InstantiationException, IllegalAccessException {
-    return registerAggregator(name, aggregator, false) != null;
+    return registerAggregator(name, aggregatorClass, false) != null;
   }
 
   @Override
   public <A extends Writable> boolean registerPersistentAggregator(String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
-    ClassAggregatorFactory<A> aggregatorFactory =
-        new ClassAggregatorFactory<A>(aggregatorClass);
-    return registerAggregator(name, aggregatorFactory, true) != null;
+    return registerAggregator(name, aggregatorClass, true) != null;
   }
 
   @Override
@@ -140,27 +177,35 @@ public class AggregatorToGlobalCommTranslation
       agg.readFields(in);
       registeredAggregators.put(name, agg);
     }
+    initAggregatorValues.clear();
   }
 
   /**
    * Helper function for registering aggregators.
    *
-   * @param name              Name of the aggregator
-   * @param aggregatorFactory Aggregator factory
-   * @param persistent        Whether aggregator is persistent or not
-   * @param <A>               Aggregated value type
+   * @param name            Name of the aggregator
+   * @param aggregatorClass Aggregator class
+   * @param persistent      Whether aggregator is persistent or not
+   * @param <A>             Aggregated value type
    * @return Newly registered aggregator or aggregator which was previously
    *         created with selected name, if any
    */
   private <A extends Writable> AggregatorWrapper<A> registerAggregator
-  (String name, WritableFactory<? extends Aggregator<A>> aggregatorFactory,
+  (String name, Class<? extends Aggregator<A>> aggregatorClass,
       boolean persistent) throws InstantiationException,
       IllegalAccessException {
     AggregatorWrapper<A> aggregatorWrapper =
         (AggregatorWrapper<A>) registeredAggregators.get(name);
     if (aggregatorWrapper == null) {
       aggregatorWrapper =
-          new AggregatorWrapper<A>(aggregatorFactory, persistent);
+          new AggregatorWrapper<A>(aggregatorClass, persistent);
+      // postMasterCompute uses previously reduced value to broadcast,
+      // unless current value is set. After aggregator is registered,
+      // there was no previously reduced value, so set current value
+      // to default to avoid calling getReduced() on unregistered reducer.
+      // (which logs unnecessary warnings)
+      aggregatorWrapper.setCurrentValue(
+          aggregatorWrapper.getReduceOp().createInitialValue());
       registeredAggregators.put(
           name, (AggregatorWrapper<Writable>) aggregatorWrapper);
     }
@@ -171,7 +216,7 @@ public class AggregatorToGlobalCommTranslation
    * Object holding all needed data related to single Aggregator
    * @param <A> Aggregated value type
    */
-  private static class AggregatorWrapper<A extends Writable>
+  private class AggregatorWrapper<A extends Writable>
       implements Writable {
     /** False iff aggregator should be reset at the end of each super step */
     private boolean persistent;
@@ -186,14 +231,14 @@ public class AggregatorToGlobalCommTranslation
 
     /**
      * Constructor
-     * @param aggregatorFactory Aggregator factory
+     * @param aggregatorClass Aggregator class
      * @param persistent Is persistent
      */
     public AggregatorWrapper(
-        WritableFactory<? extends Aggregator<A>> aggregatorFactory,
+        Class<? extends Aggregator<A>> aggregatorClass,
         boolean persistent) {
       this.persistent = persistent;
-      this.reduceOp = new AggregatorReduceOperation<>(aggregatorFactory);
+      this.reduceOp = new AggregatorReduceOperation<>(aggregatorClass, conf);
     }
 
     public AggregatorReduceOperation<A> getReduceOp() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index ab1289d..af7e5fd 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -895,7 +895,7 @@ public class BspServiceMaster<I extends WritableComparable,
           globalCommHandler = new MasterAggregatorHandler(
               getConfiguration(), getContext());
           aggregatorTranslation = new AggregatorToGlobalCommTranslation(
-              globalCommHandler);
+              getConfiguration(), globalCommHandler);
 
           globalCommHandler.initialize(this);
           masterCompute = getConfiguration().createMasterCompute();

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
index 5f7bd73..ccee656 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
@@ -28,6 +28,7 @@ import org.apache.giraph.bsp.BspService;
 import org.apache.giraph.bsp.SuperstepState;
 import org.apache.giraph.comm.GlobalCommType;
 import org.apache.giraph.comm.MasterClient;
+import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.reducers.ReduceOperation;
 import org.apache.giraph.reducers.Reducer;
@@ -61,6 +62,9 @@ public class MasterAggregatorHandler
   /** Progressable used to report progress */
   private final Progressable progressable;
 
+  /** Conf */
+  private final ImmutableClassesGiraphConfiguration<?, ?, ?> conf;
+
   /**
    * Constructor
    *
@@ -71,6 +75,7 @@ public class MasterAggregatorHandler
       ImmutableClassesGiraphConfiguration<?, ?, ?> conf,
       Progressable progressable) {
     this.progressable = progressable;
+    this.conf = conf;
     aggregatorWriter = conf.createAggregatorWriter();
   }
 
@@ -86,10 +91,18 @@ public class MasterAggregatorHandler
       R globalInitialValue) {
     if (reducerMap.containsKey(name)) {
       throw new IllegalArgumentException(
-          "Reducer with name " + name + " was already registered");
+          "Reducer with name " + name + " was already registered, " +
+          " and is " + reducerMap.get(name) + ", and we are trying to " +
+          " register " + reduceOp);
     }
     if (reduceOp == null) {
-      throw new IllegalArgumentException("null reduce cannot be registered");
+      throw new IllegalArgumentException(
+          "null reducer cannot be registered, with name " + name);
+    }
+    if (globalInitialValue == null) {
+      throw new IllegalArgumentException(
+          "global initial value for reducer cannot be null, but is for " +
+          reduceOp + " with naem" + name);
     }
 
     Reducer<S, R> reducer = new Reducer<>(reduceOp, globalInitialValue);
@@ -98,7 +111,13 @@ public class MasterAggregatorHandler
 
   @Override
   public <T extends Writable> T getReduced(String name) {
-    return (T) reducedMap.get(name);
+    T value = (T) reducedMap.get(name);
+    if (value == null) {
+      LOG.warn("getReduced: " +
+        AggregatorUtils.getUnregisteredReducerMessage(name,
+            reducedMap.size() != 0, conf));
+    }
+    return value;
   }
 
   @Override
@@ -310,14 +329,14 @@ public class MasterAggregatorHandler
     for (int i = 0; i < numReducers; i++) {
       String name = in.readUTF();
       Reducer<Object, Writable> reducer = new Reducer<>();
-      reducer.readFields(in);
+      reducer.readFields(in, conf);
       reducerMap.put(name, reducer);
     }
 
     int numBroadcast = in.readInt();
     for (int i = 0; i < numBroadcast; i++) {
       String name = in.readUTF();
-      Writable value = WritableUtils.readWritableObject(in, null);
+      Writable value = WritableUtils.readWritableObject(in, conf);
       broadcastMap.put(name, value);
     }
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
index 91f5d24..cadae67 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorUsage.java
@@ -20,7 +20,6 @@ package org.apache.giraph.master;
 
 import org.apache.giraph.aggregators.Aggregator;
 import org.apache.giraph.aggregators.AggregatorUsage;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -41,21 +40,6 @@ public interface MasterAggregatorUsage extends AggregatorUsage {
       InstantiationException, IllegalAccessException;
 
   /**
-   * Register an aggregator in preSuperstep() and/or preApplication(). This
-   * aggregator will have its value reset at the end of each super step.
-   *
-   * Aggregator should either implement Writable, or have no-arg constructor.
-   *
-   * @param name of aggregator
-   * @param aggregatorFactory aggregator factory
-   * @param <A> Aggregator type
-   * @return True iff aggregator wasn't already registered
-   */
-  <A extends Writable> boolean registerAggregator(String name,
-      WritableFactory<? extends Aggregator<A>> aggregatorFactory) throws
-      InstantiationException, IllegalAccessException;
-
-  /**
    * Register persistent aggregator in preSuperstep() and/or
    * preApplication(). This aggregator will not reset value at the end of
    * super step.

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
index 72e4d0a..68eb416 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
@@ -25,7 +25,6 @@ import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.Computation;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.reducers.ReduceOperation;
-import org.apache.giraph.utils.WritableFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
 
@@ -222,14 +221,6 @@ public abstract class MasterCompute
   }
 
   @Override
-  public final <A extends Writable> boolean registerAggregator(
-    String name, WritableFactory<? extends Aggregator<A>> aggregator)
-    throws InstantiationException, IllegalAccessException {
-    return serviceMaster.getAggregatorTranslationHandler().registerAggregator(
-        name, aggregator);
-  }
-
-  @Override
   public final <A extends Writable> boolean registerPersistentAggregator(
       String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java b/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
index a675f4d..cb9f6e0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/OnSameReduceOperation.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.io.Writable;
 public abstract class OnSameReduceOperation<R extends Writable>
     implements ReduceOperation<R, R> {
   @Override
-  public final void reducePartial(R curValue, R valueToReduce) {
-    reduceSingle(curValue, valueToReduce);
+  public final R reducePartial(R curValue, R valueToReduce) {
+    return reduceSingle(curValue, valueToReduce);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java b/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
index 434e21a..adbc4d8 100644
--- a/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/ReduceOperation.java
@@ -43,15 +43,21 @@ public interface ReduceOperation<S, R extends Writable> extends Writable {
    * Add a new value.
    * Needs to be commutative and associative
    *
+   * Commonly, returned value should be same as curValue argument.
+   *
    * @param curValue Partial value into which to reduce and store the result
    * @param valueToReduce Single value to be reduced
+   * @return reduced value
    */
-  void reduceSingle(R curValue, S valueToReduce);
+  R reduceSingle(R curValue, S valueToReduce);
   /**
    * Add partially reduced value to current partially reduced value.
    *
+   * Commonly, returned value should be same as curValue argument.
+   *
    * @param curValue Partial value into which to reduce and store the result
    * @param valueToReduce Partial value to be reduced
+   * @return reduced value
    */
-  void reducePartial(R curValue, R valueToReduce);
+  R reducePartial(R curValue, R valueToReduce);
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java b/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
index 9f821b4..6759276 100644
--- a/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/reducers/Reducer.java
@@ -21,6 +21,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 
@@ -32,7 +33,7 @@ import org.apache.hadoop.io.Writable;
  * @param <S> Single value type, objects passed on workers
  * @param <R> Reduced value type
  */
-public class Reducer<S, R extends Writable> implements Writable {
+public class Reducer<S, R extends Writable> {
   /** Reduce operations */
   private ReduceOperation<S, R> reduceOp;
   /** Current (partially) reduced value*/
@@ -49,7 +50,7 @@ public class Reducer<S, R extends Writable> implements Writable {
    */
   public Reducer(ReduceOperation<S, R> reduceOp) {
     this.reduceOp = reduceOp;
-    this.currentValue = reduceOp.createInitialValue();
+    this.currentValue = createInitialValue();
   }
   /**
    * Constructor
@@ -66,21 +67,26 @@ public class Reducer<S, R extends Writable> implements Writable {
    * @param valueToReduce Single value to reduce
    */
   public void reduceSingle(S valueToReduce) {
-    reduceOp.reduceSingle(currentValue, valueToReduce);
+    currentValue = reduceOp.reduceSingle(currentValue, valueToReduce);
   }
   /**
    * Reduce given partially reduced value into current reduced value.
    * @param valueToReduce Partial value to reduce
    */
   public void reducePartial(R valueToReduce) {
-    reduceOp.reducePartial(currentValue, valueToReduce);
+    currentValue = reduceOp.reducePartial(currentValue, valueToReduce);
   }
   /**
    * Return new initial reduced value.
    * @return New initial reduced value
    */
   public R createInitialValue() {
-    return reduceOp.createInitialValue();
+    R value = reduceOp.createInitialValue();
+    if (value == null) {
+      throw new IllegalStateException(
+          "Initial value for reducer cannot be null, but is for " + reduceOp);
+    }
+    return value;
   }
 
   public ReduceOperation<S, R> getReduceOp() {
@@ -95,16 +101,31 @@ public class Reducer<S, R extends Writable> implements Writable {
     this.currentValue = currentValue;
   }
 
-  @Override
+  /**
+   * Serialize the fields of this object to <code>out</code>.
+   *
+   * @param out <code>DataOuput</code> to serialize this object into.
+   * @throws IOException
+   */
   public void write(DataOutput out) throws IOException {
     WritableUtils.writeWritableObject(reduceOp, out);
     currentValue.write(out);
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    reduceOp = WritableUtils.readWritableObject(in, null);
-    currentValue = reduceOp.createInitialValue();
+  /**
+   * Deserialize the fields of this object from <code>in</code>.
+   *
+   * <p>For efficiency, implementations should attempt to re-use storage in the
+   * existing object where possible.</p>
+   *
+   * @param in <code>DataInput</code> to deseriablize this object from.
+   * @param conf Configuration
+   * @throws IOException
+   */
+  public void readFields(DataInput in,
+      ImmutableClassesGiraphConfiguration conf) throws IOException {
+    reduceOp = WritableUtils.readWritableObject(in, conf);
+    currentValue = createInitialValue();
     currentValue.readFields(in);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java
deleted file mode 100644
index 43bed7e..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableFactory.java
+++ /dev/null
@@ -1,28 +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.utils;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * Factory that can be serialized.
- * @param <T> Type of object factory creates
- */
-public interface WritableFactory<T> extends Writable, Factory<T> {
-
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 923d369..8c24697 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -745,13 +745,15 @@ public class WritableUtils {
    * @param reusableOut Reusable output stream to serialize into
    * @param reusableIn Reusable input stream to deserialize out of
    * @param original Original value of which to make a copy
+   * @param conf Configuration
    * @param <T> Type of the object
    * @return Copy of the original value
    */
   public static <T extends Writable> T createCopy(
       UnsafeByteArrayOutputStream reusableOut,
-      UnsafeReusableByteArrayInput reusableIn, T original) {
-    T copy = (T) createWritable(original.getClass(), null);
+      UnsafeReusableByteArrayInput reusableIn, T original,
+      ImmutableClassesGiraphConfiguration conf) {
+    T copy = (T) createWritable(original.getClass(), conf);
 
     try {
       reusableOut.reset();

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
index 5238a07..916e7a0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorDelegator.java
@@ -18,6 +18,7 @@
 package org.apache.giraph.worker;
 
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.master.AggregatorBroadcast;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -64,6 +65,7 @@ public abstract class WorkerAggregatorDelegator<I extends WritableComparable,
 
   @Override
   public final <A extends Writable> A getAggregatedValue(String name) {
-    return this.<A>getBroadcast(name);
+    AggregatorBroadcast<A> broadcast = workerGlobalCommUsage.getBroadcast(name);
+    return broadcast.getValue();
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
index 05a13a7..ee47542 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
@@ -87,7 +87,7 @@ public class WorkerAggregatorHandler implements WorkerThreadGlobalCommUsage {
     B value = (B) broadcastedMap.get(name);
     if (value == null) {
       LOG.warn("getBroadcast: " +
-          AggregatorUtils.getUnregisteredAggregatorMessage(name,
+          AggregatorUtils.getUnregisteredBroadcastMessage(name,
               broadcastedMap.size() != 0, conf));
     }
     return value;
@@ -103,7 +103,7 @@ public class WorkerAggregatorHandler implements WorkerThreadGlobalCommUsage {
       }
     } else {
       throw new IllegalStateException("reduce: " +
-          AggregatorUtils.getUnregisteredAggregatorMessage(name,
+          AggregatorUtils.getUnregisteredReducerMessage(name,
               reducerMap.size() != 0, conf));
     }
   }
@@ -122,7 +122,7 @@ public class WorkerAggregatorHandler implements WorkerThreadGlobalCommUsage {
       }
     } else {
       throw new IllegalStateException("reduce: " +
-          AggregatorUtils.getUnregisteredAggregatorMessage(name,
+          AggregatorUtils.getUnregisteredReducerMessage(name,
               reducerMap.size() != 0, conf));
     }
   }
@@ -309,7 +309,7 @@ public class WorkerAggregatorHandler implements WorkerThreadGlobalCommUsage {
             entry.getValue().getReduceOp();
 
         ReduceOperation<Object, Writable> threadLocalCopy =
-            WritableUtils.createCopy(out, in, globalReduceOp);
+            WritableUtils.createCopy(out, in, globalReduceOp, conf);
 
         threadReducerMap.put(entry.getKey(), new Reducer<>(threadLocalCopy));
       }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerBroadcastUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerBroadcastUsage.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerBroadcastUsage.java
new file mode 100644
index 0000000..9b4e160
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerBroadcastUsage.java
@@ -0,0 +1,33 @@
+/*
+ * 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.worker;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Methods on worker can access broadcasted values through this interface
+ */
+public interface WorkerBroadcastUsage {
+  /**
+   * Get value broadcasted from master
+   * @param name Name of the broadcasted value
+   * @return Broadcasted value
+   * @param <B> Broadcast value type
+   */
+  <B extends Writable> B getBroadcast(String name);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
index 39566f5..fa31bc2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerGlobalCommUsage.java
@@ -17,24 +17,11 @@
  */
 package org.apache.giraph.worker;
 
-import org.apache.hadoop.io.Writable;
 
 /**
  * Methods on worker can access broadcasted values and provide
  * values to reduce through this interface
  */
-public interface WorkerGlobalCommUsage {
-  /**
-   * Reduce given value.
-   * @param name Name of the reducer
-   * @param value Single value to reduce
-   */
-  void reduce(String name, Object value);
-  /**
-   * Get value broadcasted from master
-   * @param name Name of the broadcasted value
-   * @return Broadcasted value
-   * @param <B> Broadcast value type
-   */
-  <B extends Writable> B getBroadcast(String name);
+public interface WorkerGlobalCommUsage
+    extends WorkerBroadcastUsage, WorkerReduceUsage {
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerReduceUsage.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerReduceUsage.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerReduceUsage.java
new file mode 100644
index 0000000..9c2e90d
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerReduceUsage.java
@@ -0,0 +1,30 @@
+/*
+ * 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.worker;
+
+/**
+ * Methods on worker can provide values to reduce through this interface
+ */
+public interface WorkerReduceUsage {
+  /**
+   * Reduce given value.
+   * @param name Name of the reducer
+   * @param value Single value to reduce
+   */
+  void reduce(String name, Object value);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/d32c429a/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java b/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java
deleted file mode 100644
index 2898647..0000000
--- a/giraph-core/src/test/java/org/apache/giraph/aggregators/TestArrayAggregator.java
+++ /dev/null
@@ -1,50 +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.aggregators;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.giraph.utils.ArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.junit.Test;
-
-public class TestArrayAggregator {
-  @Test
-  public void testMaxAggregator() {
-    Aggregator<ArrayWritable<LongWritable>> max = new ArrayAggregatorFactory<>(2, LongMaxAggregator.class).create();
-
-    ArrayWritable<LongWritable> tmp = max.createInitialValue();
-
-    tmp.get()[0].set(2);
-    max.aggregate(tmp);
-
-    tmp.get()[0].set(3);
-    tmp.get()[1].set(1);
-    max.aggregate(tmp);
-
-    assertEquals(3L, max.getAggregatedValue().get()[0].get());
-    assertEquals(1L, max.getAggregatedValue().get()[1].get());
-
-    tmp.get()[0].set(-1);
-    tmp.get()[1].set(-1);
-    max.setAggregatedValue(tmp);
-
-    assertEquals(-1L, max.getAggregatedValue().get()[0].get());
-    assertEquals(-1L, max.getAggregatedValue().get()[1].get());
-  }
-}


[33/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-938: Allow fast working with primitives generically (ikabiljo via pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/18520570
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/18520570
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/18520570

Branch: refs/heads/release-1.1
Commit: 185205703ee4cd886598f5393395f19fc367f65f
Parents: f6845a3
Author: Pavan Kumar <pa...@fb.com>
Authored: Wed Sep 17 14:02:40 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Wed Sep 17 14:02:40 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../messages/InMemoryMessageStoreFactory.java   |  34 +-
 .../primitives/IdByteArrayMessageStore.java     | 246 ++++++++
 .../primitives/IdOneMessagePerVertexStore.java  | 226 +++++++
 .../apache/giraph/edge/IdAndNullArrayEdges.java | 184 ++++++
 .../giraph/edge/IdAndValueArrayEdges.java       | 248 ++++++++
 .../apache/giraph/types/ops/BooleanTypeOps.java |  53 ++
 .../apache/giraph/types/ops/ByteTypeOps.java    |  52 ++
 .../apache/giraph/types/ops/DoubleTypeOps.java  |  52 ++
 .../apache/giraph/types/ops/FloatTypeOps.java   |  52 ++
 .../org/apache/giraph/types/ops/IntTypeOps.java |  68 ++
 .../apache/giraph/types/ops/LongTypeOps.java    |  68 ++
 .../org/apache/giraph/types/ops/MapTypeOps.java |  47 ++
 .../giraph/types/ops/PrimitiveIdTypeOps.java    |  55 ++
 .../giraph/types/ops/PrimitiveTypeOps.java      |  42 ++
 .../apache/giraph/types/ops/TextTypeOps.java    |  46 ++
 .../org/apache/giraph/types/ops/TypeOps.java    |  51 ++
 .../apache/giraph/types/ops/TypeOpsUtils.java   | 149 +++++
 .../types/ops/collections/Basic2ObjectMap.java  | 322 ++++++++++
 .../types/ops/collections/BasicArrayList.java   | 632 +++++++++++++++++++
 .../giraph/types/ops/collections/BasicSet.java  | 206 ++++++
 .../ops/collections/ResettableIterator.java     |  32 +
 .../types/ops/collections/WritableWriter.java   |  47 ++
 .../types/ops/collections/package-info.java     |  21 +
 .../apache/giraph/types/ops/package-info.java   |  21 +
 25 files changed, 2949 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 34db15c..d9398e7 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-938: Allow fast working with primitives generically (ikabiljo via pavanka)
+
   GIRAPH-945: Always use job Configuration to create Configuration (majakabiljo)
 
   GIRAPH-931: Provide a Strongly Connected Components algorithm (gianluca via majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
index 02ea7b2..ae86c56 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
@@ -20,15 +20,19 @@ package org.apache.giraph.comm.messages;
 
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.primitives.IdByteArrayMessageStore;
+import org.apache.giraph.comm.messages.primitives.IdOneMessagePerVertexStore;
 import org.apache.giraph.comm.messages.primitives.IntByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.IntFloatMessageStore;
-import org.apache.giraph.comm.messages.primitives.long_id.LongByteArrayMessageStore;
 import org.apache.giraph.comm.messages.primitives.LongDoubleMessageStore;
+import org.apache.giraph.comm.messages.primitives.long_id.LongByteArrayMessageStore;
+import org.apache.giraph.comm.messages.primitives.long_id.LongPointerListMessageStore;
 import org.apache.giraph.comm.messages.queue.AsyncMessageStoreWrapper;
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.comm.messages.primitives.long_id.LongPointerListMessageStore;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
@@ -89,8 +93,17 @@ public class InMemoryMessageStoreFactory<I extends WritableComparable,
           (MessageCombiner<LongWritable, DoubleWritable>)
               conf.<DoubleWritable>createMessageCombiner());
     } else {
-      messageStore = new OneMessagePerVertexStore(messageValueFactory,
-          service, conf.<M>createMessageCombiner(), conf);
+      PrimitiveIdTypeOps<I> idTypeOps =
+          TypeOpsUtils.getPrimitiveIdTypeOpsOrNull(vertexIdClass);
+      if (idTypeOps != null) {
+        messageStore = new IdOneMessagePerVertexStore<>(
+            messageValueFactory, service, conf.<M>createMessageCombiner(),
+            conf);
+      } else {
+        messageStore =
+            new OneMessagePerVertexStore<I, M>(messageValueFactory, service,
+                conf.<M>createMessageCombiner(), conf);
+      }
     }
     return messageStore;
   }
@@ -127,11 +140,18 @@ public class InMemoryMessageStoreFactory<I extends WritableComparable,
           MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION) ||
           encodeAndStore.equals(
               MessageEncodeAndStoreType.EXTRACT_BYTEARRAY_PER_PARTITION)) {
-        messageStore = new ByteArrayMessagesPerVertexStore<>(
-            messageValueFactory, service, conf);
+        PrimitiveIdTypeOps<I> idTypeOps =
+            TypeOpsUtils.getPrimitiveIdTypeOpsOrNull(vertexIdClass);
+        if (idTypeOps != null) {
+          messageStore = new IdByteArrayMessageStore<>(
+              messageValueFactory, service, conf);
+        } else {
+          messageStore = new ByteArrayMessagesPerVertexStore<>(
+              messageValueFactory, service, conf);
+        }
       } else if (encodeAndStore.equals(
           MessageEncodeAndStoreType.POINTER_LIST_PER_VERTEX)) {
-        messageStore = new PointerListPerVertexStore(messageValueFactory,
+        messageStore = new PointerListPerVertexStore<>(messageValueFactory,
             service, conf);
       }
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdByteArrayMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdByteArrayMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdByteArrayMessageStore.java
new file mode 100644
index 0000000..efe6199
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdByteArrayMessageStore.java
@@ -0,0 +1,246 @@
+/*
+ * 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.comm.messages.primitives;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.comm.messages.MessagesIterable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap;
+import org.apache.giraph.types.ops.collections.WritableWriter;
+import org.apache.giraph.utils.EmptyIterable;
+import org.apache.giraph.utils.VerboseByteStructMessageWrite;
+import org.apache.giraph.utils.VertexIdMessageBytesIterator;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.giraph.utils.io.DataInputOutput;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Special message store to be used when IDs are primitive and no combiner is
+ * used.
+ * Data is backed by primitive maps in order to decrease number of objects and
+ * get better performance.
+ *
+ * @param <I> Vertex id type
+ * @param <M> Message type
+ */
+public class IdByteArrayMessageStore<I extends WritableComparable,
+    M extends Writable> implements MessageStore<I, M> {
+  /** Message value factory */
+  protected final MessageValueFactory<M> messageValueFactory;
+  /** Map from partition id to map from vertex id to message */
+  private final Int2ObjectOpenHashMap<Basic2ObjectMap<I, DataInputOutput>> map;
+  /** Service worker */
+  private final CentralizedServiceWorker<I, ?, ?> service;
+  /** Giraph configuration */
+  private final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
+  /** Vertex id TypeOps */
+  private final PrimitiveIdTypeOps<I> idTypeOps;
+  /** WritableWriter for values in this message store */
+  private final WritableWriter<DataInputOutput>
+  dataInputOutputWriter = new WritableWriter<DataInputOutput>() {
+    @Override
+    public DataInputOutput readFields(DataInput in) throws IOException {
+      DataInputOutput dataInputOutput = config.createMessagesInputOutput();
+      dataInputOutput.readFields(in);
+      return dataInputOutput;
+    }
+
+    @Override
+    public void write(DataOutput out, DataInputOutput value)
+      throws IOException {
+      value.write(out);
+    }
+  };
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Factory for creating message values
+   * @param service Service worker
+   * @param config Hadoop configuration
+   */
+  public IdByteArrayMessageStore(MessageValueFactory<M> messageValueFactory,
+      CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    this.messageValueFactory = messageValueFactory;
+    this.service = service;
+    this.config = config;
+
+    idTypeOps = TypeOpsUtils.getPrimitiveIdTypeOps(config.getVertexIdClass());
+
+    map = new Int2ObjectOpenHashMap<Basic2ObjectMap<I, DataInputOutput>>();
+    for (int partitionId : service.getPartitionStore().getPartitionIds()) {
+      Partition<I, ?, ?> partition =
+          service.getPartitionStore().getOrCreatePartition(partitionId);
+      Basic2ObjectMap<I, DataInputOutput> partitionMap =
+          idTypeOps.create2ObjectOpenHashMap(
+              Math.max(10, (int) partition.getVertexCount()));
+
+      map.put(partitionId, partitionMap);
+      service.getPartitionStore().putPartition((Partition) partition);
+    }
+  }
+
+  /**
+   * Get map which holds messages for partition which vertex belongs to.
+   *
+   * @param vertexId Id of the vertex
+   * @return Map which holds messages for partition which vertex belongs to.
+   */
+  private Basic2ObjectMap<I, DataInputOutput> getPartitionMap(I vertexId) {
+    return map.get(service.getPartitionId(vertexId));
+  }
+
+  /**
+   * Get the DataInputOutput for a vertex id, creating if necessary.
+   *
+   * @param partitionMap Partition map to look in
+   * @param vertexId Id of the vertex
+   * @return DataInputOutput for this vertex id (created if necessary)
+   */
+  private DataInputOutput getDataInputOutput(
+      Basic2ObjectMap<I, DataInputOutput> partitionMap,
+      I vertexId) {
+    DataInputOutput dataInputOutput = partitionMap.get(vertexId);
+    if (dataInputOutput == null) {
+      dataInputOutput = config.createMessagesInputOutput();
+      partitionMap.put(vertexId, dataInputOutput);
+    }
+    return dataInputOutput;
+  }
+
+  @Override
+  public void addPartitionMessages(int partitionId,
+      VertexIdMessages<I, M> messages) throws IOException {
+    Basic2ObjectMap<I, DataInputOutput> partitionMap = map.get(partitionId);
+    synchronized (partitionMap) {
+      VertexIdMessageBytesIterator<I, M> vertexIdMessageBytesIterator =
+          messages.getVertexIdMessageBytesIterator();
+      // Try to copy the message buffer over rather than
+      // doing a deserialization of a message just to know its size. This
+      // should be more efficient for complex objects where serialization is
+      // expensive. If this type of iterator is not available, fall back to
+      // deserializing/serializing the messages
+      if (vertexIdMessageBytesIterator != null) {
+        while (vertexIdMessageBytesIterator.hasNext()) {
+          vertexIdMessageBytesIterator.next();
+          DataInputOutput dataInputOutput = getDataInputOutput(
+              partitionMap, vertexIdMessageBytesIterator.getCurrentVertexId());
+          vertexIdMessageBytesIterator.writeCurrentMessageBytes(
+              dataInputOutput.getDataOutput());
+        }
+      } else {
+        VertexIdMessageIterator<I, M> iterator =
+            messages.getVertexIdMessageIterator();
+        while (iterator.hasNext()) {
+          iterator.next();
+          DataInputOutput dataInputOutput =
+              getDataInputOutput(partitionMap, iterator.getCurrentVertexId());
+
+          VerboseByteStructMessageWrite.verboseWriteCurrentMessage(iterator,
+              dataInputOutput.getDataOutput());
+        }
+      }
+    }
+  }
+
+  @Override
+  public void clearPartition(int partitionId) throws IOException {
+    map.get(partitionId).clear();
+  }
+
+  @Override
+  public boolean hasMessagesForVertex(I vertexId) {
+    return getPartitionMap(vertexId).containsKey(vertexId);
+  }
+
+  @Override
+  public Iterable<M> getVertexMessages(I vertexId) throws IOException {
+    DataInputOutput dataInputOutput = getPartitionMap(vertexId).get(vertexId);
+    if (dataInputOutput == null) {
+      return EmptyIterable.get();
+    } else {
+      return new MessagesIterable<M>(dataInputOutput, messageValueFactory);
+    }
+  }
+
+  @Override
+  public void clearVertexMessages(I vertexId) throws IOException {
+    getPartitionMap(vertexId).remove(vertexId);
+  }
+
+  @Override
+  public void clearAll() throws IOException {
+    map.clear();
+  }
+
+  @Override
+  public Iterable<I> getPartitionDestinationVertices(int partitionId) {
+    Basic2ObjectMap<I, DataInputOutput> partitionMap = map.get(partitionId);
+    List<I> vertices = Lists.newArrayListWithCapacity(partitionMap.size());
+    Iterator<I> iterator = partitionMap.fastKeyIterator();
+    while (iterator.hasNext()) {
+      vertices.add(idTypeOps.createCopy(iterator.next()));
+    }
+    return vertices;
+  }
+
+  @Override
+  public void writePartition(DataOutput out, int partitionId)
+    throws IOException {
+    Basic2ObjectMap<I, DataInputOutput> partitionMap = map.get(partitionId);
+    partitionMap.write(out, dataInputOutputWriter);
+  }
+
+  @Override
+  public void readFieldsForPartition(DataInput in, int partitionId)
+    throws IOException {
+    Basic2ObjectMap<I, DataInputOutput> partitionMap =
+        idTypeOps.create2ObjectOpenHashMap(10);
+    partitionMap.readFields(in, dataInputOutputWriter);
+    synchronized (map) {
+      map.put(partitionId, partitionMap);
+    }
+  }
+
+  @Override
+  public void finalizeStore() {
+  }
+
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdOneMessagePerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdOneMessagePerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdOneMessagePerVertexStore.java
new file mode 100644
index 0000000..c72bedf
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/primitives/IdOneMessagePerVertexStore.java
@@ -0,0 +1,226 @@
+/*
+ * 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.comm.messages.primitives;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.MessageStore;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.partition.Partition;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap;
+import org.apache.giraph.types.ops.collections.WritableWriter;
+import org.apache.giraph.utils.EmptyIterable;
+import org.apache.giraph.utils.VertexIdMessageIterator;
+import org.apache.giraph.utils.VertexIdMessages;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Special message store to be used when IDs are primitive and message doesn't
+ * need to be, and message combiner is used.
+ * Data is backed by primitive keyed maps in order to decrease number of
+ * objects and get better performance.
+ * (keys are using primitives, values are using objects, even if they
+ * are primitive)
+ *
+ * @param <I> Vertex id type
+ * @param <M> Message type
+ */
+public class IdOneMessagePerVertexStore<I extends WritableComparable,
+    M extends Writable> implements MessageStore<I, M> {
+  /** Map from partition id to map from vertex id to message */
+  private final Int2ObjectOpenHashMap<Basic2ObjectMap<I, M>> map;
+  /** Message value factory */
+  private final MessageValueFactory<M> messageValueFactory;
+  /** Message messageCombiner */
+  private final MessageCombiner<I, M> messageCombiner;
+  /** Service worker */
+  private final CentralizedServiceWorker<I, ?, ?> service;
+  /** Giraph configuration */
+  private final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
+  /** Vertex id TypeOps */
+  private final PrimitiveIdTypeOps<I> idTypeOps;
+  /** WritableWriter for values in this message store */
+  private final WritableWriter<M> messageWriter = new WritableWriter<M>() {
+    @Override
+    public M readFields(DataInput in) throws IOException {
+      M message = messageValueFactory.newInstance();
+      message.readFields(in);
+      return message;
+    }
+
+    @Override
+    public void write(DataOutput out, M value) throws IOException {
+      value.write(out);
+    }
+  };
+
+  /**
+   * Constructor
+   *
+   * @param messageValueFactory Message value factory
+   * @param service Service worker
+   * @param messageCombiner Message messageCombiner
+   * @param config Config
+   */
+  public IdOneMessagePerVertexStore(
+      MessageValueFactory<M> messageValueFactory,
+      CentralizedServiceWorker<I, ?, ?> service,
+      MessageCombiner<I, M> messageCombiner,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    this.service = service;
+    this.config = config;
+    this.messageValueFactory = messageValueFactory;
+    this.messageCombiner = messageCombiner;
+
+    idTypeOps = TypeOpsUtils.getPrimitiveIdTypeOps(config.getVertexIdClass());
+
+    map = new Int2ObjectOpenHashMap<>();
+    for (int partitionId : service.getPartitionStore().getPartitionIds()) {
+      Partition<I, ?, ?> partition =
+          service.getPartitionStore().getOrCreatePartition(partitionId);
+      Basic2ObjectMap<I, M> partitionMap = idTypeOps.create2ObjectOpenHashMap(
+          (int) partition.getVertexCount());
+      map.put(partitionId, partitionMap);
+      service.getPartitionStore().putPartition((Partition) partition);
+    }
+  }
+
+  /**
+   * Get map which holds messages for partition which vertex belongs to.
+   *
+   * @param vertexId Id of the vertex
+   * @return Map which holds messages for partition which vertex belongs to.
+   */
+  private Basic2ObjectMap<I, M> getPartitionMap(I vertexId) {
+    return map.get(service.getPartitionId(vertexId));
+  }
+
+  @Override
+  public void addPartitionMessages(
+      int partitionId,
+      VertexIdMessages<I, M> messages) throws IOException {
+    Basic2ObjectMap<I, M> partitionMap = map.get(partitionId);
+    synchronized (partitionMap) {
+      VertexIdMessageIterator<I, M>
+          iterator = messages.getVertexIdMessageIterator();
+      // This loop is a little complicated as it is optimized to only create
+      // the minimal amount of vertex id and message objects as possible.
+      while (iterator.hasNext()) {
+        iterator.next();
+        I vertexId = iterator.getCurrentVertexId();
+        M currentMessage =
+            partitionMap.get(iterator.getCurrentVertexId());
+        if (currentMessage == null) {
+          M newMessage = messageCombiner.createInitialMessage();
+          currentMessage = partitionMap.put(
+              iterator.getCurrentVertexId(), newMessage);
+          if (currentMessage == null) {
+            currentMessage = newMessage;
+          }
+        }
+        messageCombiner.combine(vertexId, currentMessage,
+          iterator.getCurrentMessage());
+      }
+    }
+  }
+
+  @Override
+  public void clearPartition(int partitionId) throws IOException {
+    map.get(partitionId).clear();
+  }
+
+  @Override
+  public boolean hasMessagesForVertex(I vertexId) {
+    return getPartitionMap(vertexId).containsKey(vertexId);
+  }
+
+  @Override
+  public Iterable<M> getVertexMessages(
+      I vertexId) throws IOException {
+    Basic2ObjectMap<I, M> partitionMap = getPartitionMap(vertexId);
+    if (!partitionMap.containsKey(vertexId)) {
+      return EmptyIterable.get();
+    } else {
+      return Collections.singleton(partitionMap.get(vertexId));
+    }
+  }
+
+  @Override
+  public void clearVertexMessages(I vertexId) throws IOException {
+    getPartitionMap(vertexId).remove(vertexId);
+  }
+
+  @Override
+  public void clearAll() throws IOException {
+    map.clear();
+  }
+
+  @Override
+  public Iterable<I> getPartitionDestinationVertices(
+      int partitionId) {
+    Basic2ObjectMap<I, M> partitionMap = map.get(partitionId);
+    List<I> vertices =
+        Lists.newArrayListWithCapacity(partitionMap.size());
+    Iterator<I> iterator = partitionMap.fastKeyIterator();
+    while (iterator.hasNext()) {
+      vertices.add(idTypeOps.createCopy(iterator.next()));
+    }
+    return vertices;
+  }
+
+  @Override
+  public void writePartition(DataOutput out,
+      int partitionId) throws IOException {
+    Basic2ObjectMap<I, M> partitionMap = map.get(partitionId);
+    partitionMap.write(out, messageWriter);
+  }
+
+  @Override
+  public void readFieldsForPartition(DataInput in,
+      int partitionId) throws IOException {
+    Basic2ObjectMap<I, M> partitionMap = idTypeOps.create2ObjectOpenHashMap(10);
+    partitionMap.readFields(in, messageWriter);
+    synchronized (map) {
+      map.put(partitionId, partitionMap);
+    }
+  }
+
+  @Override
+  public void finalizeStore() {
+  }
+
+  @Override
+  public boolean isPointerListEncoding() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/edge/IdAndNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IdAndNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IdAndNullArrayEdges.java
new file mode 100644
index 0000000..7de5d2a
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/IdAndNullArrayEdges.java
@@ -0,0 +1,184 @@
+/*
+ * 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.edge;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
+import org.apache.giraph.types.ops.collections.BasicArrayList;
+import org.apache.giraph.utils.EdgeIterables;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Implementation of {@link OutEdges} with IDs and null edge values having
+ * their TypeOps.
+ * Backed by a dynamic primitive array. Parallel edges are allowed.
+ * Note: this implementation is optimized for space
+ * usage, but random access and edge removals are expensive.
+ *
+ * @param <I> Vertex id type
+ */
+public class IdAndNullArrayEdges<I extends WritableComparable>
+  implements ReuseObjectsOutEdges<I, NullWritable>,
+  MutableOutEdges<I, NullWritable>,
+  ImmutableClassesGiraphConfigurable<I, Writable, NullWritable> {
+
+  /** Array of target vertex ids. */
+  private BasicArrayList<I> neighbors;
+
+  @Override
+  public
+  ImmutableClassesGiraphConfiguration<I, Writable, NullWritable> getConf() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setConf(
+      ImmutableClassesGiraphConfiguration<I, Writable, NullWritable> conf) {
+    PrimitiveIdTypeOps<I> idTypeOps =
+        TypeOpsUtils.getPrimitiveIdTypeOps(conf.getVertexIdClass());
+    neighbors = idTypeOps.createArrayList(10);
+    if (!conf.getEdgeValueClass().equals(NullWritable.class)) {
+      throw new IllegalArgumentException(
+          "IdAndNullArrayEdges can be used only with NullWritable " +
+          "as edgeValueClass, not with " + conf.getEdgeValueClass());
+    }
+  }
+
+  @Override
+  public void initialize(Iterable<Edge<I, NullWritable>> edges) {
+    EdgeIterables.initialize(this, edges);
+  }
+
+  @Override
+  public void initialize(int capacity) {
+    neighbors.setCapacity(capacity);
+  }
+
+  @Override
+  public void initialize() {
+    initialize(10);
+  }
+
+  @Override
+  public void add(Edge<I, NullWritable> edge) {
+    neighbors.add(edge.getTargetVertexId());
+  }
+
+  /**
+   * If the backing array is more than four times as big as the number of
+   * elements, reduce to 2 times current size.
+   */
+  private void trim() {
+    if (neighbors.capacity() > 4 * neighbors.size()) {
+      neighbors.setCapacity(neighbors.size() * 2);
+    }
+  }
+
+  /**
+   * Remove edge at position i.
+   *
+   * @param i Position of edge to be removed
+   */
+  private void removeAt(int i) {
+    // The order of the edges is irrelevant, so we can simply replace
+    // the deleted edge with the rightmost element, thus achieving constant
+    // time.
+    I tmpValue = neighbors.getElementTypeOps().create();
+    neighbors.popInto(tmpValue);
+    if (i != neighbors.size()) {
+      neighbors.set(i, tmpValue);
+    }
+    // If needed after the removal, trim the array.
+    trim();
+  }
+
+  @Override
+  public void remove(I targetVertexId) {
+    // Thanks to the constant-time implementation of removeAt(int),
+    // we can remove all matching edges in linear time.
+    I tmpValue = neighbors.getElementTypeOps().create();
+    for (int i = neighbors.size() - 1; i >= 0; --i) {
+      neighbors.getInto(i, tmpValue);
+      if (tmpValue.equals(targetVertexId)) {
+        removeAt(i);
+      }
+    }
+  }
+
+  @Override
+  public int size() {
+    return neighbors.size();
+  }
+
+  @Override
+  public Iterator<Edge<I, NullWritable>> iterator() {
+    // Returns an iterator that reuses objects.
+    // The downcast is fine because all concrete Edge implementations are
+    // mutable, but we only expose the mutation functionality when appropriate.
+    return (Iterator) mutableIterator();
+  }
+
+  @Override
+  public Iterator<MutableEdge<I, NullWritable>> mutableIterator() {
+    return new Iterator<MutableEdge<I, NullWritable>>() {
+      /** Current position in the array. */
+      private int offset = 0;
+      /** Representative edge object. */
+      private final MutableEdge<I, NullWritable> representativeEdge =
+          EdgeFactory.createReusable(neighbors.getElementTypeOps().create());
+
+      @Override
+      public boolean hasNext() {
+        return offset < neighbors.size();
+      }
+
+      @Override
+      public MutableEdge<I, NullWritable> next() {
+        neighbors.getInto(offset++, representativeEdge.getTargetVertexId());
+        return representativeEdge;
+      }
+
+      @Override
+      public void remove() {
+        // Since removeAt() might replace the deleted edge with the last edge
+        // in the array, we need to decrease the offset so that the latter
+        // won't be skipped.
+        removeAt(--offset);
+      }
+    };
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    neighbors.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    neighbors.readFields(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/edge/IdAndValueArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IdAndValueArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IdAndValueArrayEdges.java
new file mode 100644
index 0000000..b99692f
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/IdAndValueArrayEdges.java
@@ -0,0 +1,248 @@
+/*
+ * 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.edge;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.PrimitiveTypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
+import org.apache.giraph.types.ops.collections.BasicArrayList;
+import org.apache.giraph.utils.EdgeIterables;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.UnmodifiableIterator;
+
+/**
+ * Implementation of {@link OutEdges} with IDs and Edge values having their
+ * TypeOps.
+ * Data is backed by a dynamic primitive array. Parallel edges are allowed.
+ * Note: this implementation is optimized for space usage, but random access
+ * and edge removals are expensive.
+ *
+ * @param <I> Vertex id type
+ * @param <E> Edge value type
+ */
+public class IdAndValueArrayEdges<I extends WritableComparable,
+    E extends Writable> implements ReuseObjectsOutEdges<I, E>,
+    MutableOutEdges<I, E>,
+    ImmutableClassesGiraphConfigurable<I, Writable, E> {
+
+  /** Array of target vertex ids. */
+  private BasicArrayList<I> neighborIds;
+  /** Array of edge values. */
+  private BasicArrayList<E> neighborEdgeValues;
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<I, Writable, E> getConf() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setConf(
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
+    PrimitiveIdTypeOps<I> idTypeOps =
+        TypeOpsUtils.getPrimitiveIdTypeOps(conf.getVertexIdClass());
+    neighborIds = idTypeOps.createArrayList(10);
+
+    PrimitiveTypeOps<E> edgeTypeOps =
+        TypeOpsUtils.getPrimitiveTypeOps(conf.getEdgeValueClass());
+    neighborEdgeValues = edgeTypeOps.createArrayList(10);
+  }
+
+  @Override
+  public void initialize(Iterable<Edge<I, E>> edges) {
+    EdgeIterables.initialize(this, edges);
+  }
+
+  @Override
+  public void initialize(int capacity) {
+    neighborIds.setCapacity(capacity);
+    neighborEdgeValues.setCapacity(capacity);
+  }
+
+  @Override
+  public void initialize() {
+    initialize(10);
+  }
+
+  @Override
+  public void add(Edge<I, E> edge) {
+    neighborIds.add(edge.getTargetVertexId());
+    neighborEdgeValues.add(edge.getValue());
+  }
+
+  /**
+   * If the backing array is more than four times as big as the number of
+   * elements, reduce to 2 times current size.
+   */
+  private void trim() {
+    if (neighborIds.capacity() > 4 * neighborIds.size()) {
+      neighborIds.setCapacity(neighborIds.size() * 2);
+      neighborEdgeValues.setCapacity(neighborIds.size() * 2);
+    }
+  }
+
+  /**
+   * Remove edge at position i.
+   *
+   * @param i Position of edge to be removed
+   */
+  private void removeAt(int i) {
+    // The order of the edges is irrelevant, so we can simply replace
+    // the deleted edge with the rightmost element, thus achieving constant
+    // time.
+    I tmpId = neighborIds.getElementTypeOps().create();
+    E tmpValue = neighborEdgeValues.getElementTypeOps().create();
+
+    neighborIds.popInto(tmpId);
+    neighborEdgeValues.popInto(tmpValue);
+    if (i != neighborIds.size()) {
+      neighborIds.set(i, tmpId);
+      neighborEdgeValues.set(i, tmpValue);
+    }
+    // If needed after the removal, trim the array.
+    trim();
+  }
+
+  @Override
+  public void remove(I targetVertexId) {
+    // Thanks to the constant-time implementation of removeAt(int),
+    // we can remove all matching edges in linear time.
+    I tmpId = neighborIds.getElementTypeOps().create();
+    for (int i = neighborIds.size() - 1; i >= 0; --i) {
+      neighborIds.getInto(i, tmpId);
+      if (tmpId.equals(targetVertexId)) {
+        removeAt(i);
+      }
+    }
+  }
+
+  @Override
+  public int size() {
+    return neighborIds.size();
+  }
+
+  @Override
+  public Iterator<Edge<I, E>> iterator() {
+    // Returns an iterator that reuses objects.
+    return new UnmodifiableIterator<Edge<I, E>>() {
+      private int index;
+
+      /** Representative edge object. */
+      private final Edge<I, E> representativeEdge = EdgeFactory.create(
+          neighborIds.getElementTypeOps().create(),
+          neighborEdgeValues.getElementTypeOps().create());
+
+      @Override
+      public boolean hasNext() {
+        return index < neighborIds.size();
+      }
+
+      @Override
+      public Edge<I, E> next() {
+        neighborIds.getInto(index, representativeEdge.getTargetVertexId());
+        neighborEdgeValues.getInto(index, representativeEdge.getValue());
+        index++;
+        return representativeEdge;
+      }
+    };
+  }
+
+  /** Helper class for a mutable edge that modifies the backing arrays. */
+  private class ArrayMutableEdge extends DefaultEdge<I, E> {
+    /** Index of the edge in the backing arrays. */
+    private int index;
+
+    /** Constructor. */
+    public ArrayMutableEdge() {
+      super(
+          neighborIds.getElementTypeOps().create(),
+          neighborEdgeValues.getElementTypeOps().create());
+    }
+
+    /**
+     * Make the edge point to the given index in the backing arrays.
+     *
+     * @param index Index in the arrays
+     */
+    public void setIndex(int index) {
+      // Update the id and value objects from the superclass.
+      neighborIds.getInto(index, getTargetVertexId());
+      neighborEdgeValues.getInto(index, getValue());
+      // Update the index.
+      this.index = index;
+    }
+
+    @Override
+    public void setValue(E value) {
+      // Update the value object from the superclass.
+      neighborEdgeValues.getElementTypeOps().set(getValue(), value);
+      // Update the value stored in the backing array.
+      neighborEdgeValues.set(index, value);
+    }
+  }
+
+  @Override
+  public Iterator<MutableEdge<I, E>> mutableIterator() {
+    return new Iterator<MutableEdge<I, E>>() {
+      /** Current position in the array. */
+      private int index = 0;
+      /** Representative edge object. */
+      private final ArrayMutableEdge representativeEdge =
+          new ArrayMutableEdge();
+
+      @Override
+      public boolean hasNext() {
+        return index < neighborIds.size();
+      }
+
+      @Override
+      public MutableEdge<I, E> next() {
+        representativeEdge.setIndex(index++);
+        return representativeEdge;
+      }
+
+      @Override
+      public void remove() {
+        // Since removeAt() might replace the deleted edge with the last edge
+        // in the array, we need to decrease the offset so that the latter
+        // won't be skipped.
+        removeAt(--index);
+      }
+    };
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    neighborIds.write(out);
+    neighborEdgeValues.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    neighborIds.readFields(in);
+    neighborEdgeValues.readFields(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/BooleanTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/BooleanTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/BooleanTypeOps.java
new file mode 100644
index 0000000..a65fa3b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/BooleanTypeOps.java
@@ -0,0 +1,53 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.BasicArrayList.BasicBooleanArrayList;
+import org.apache.hadoop.io.BooleanWritable;
+
+
+/** TypeOps implementation for working with BooleanWritable type */
+public enum BooleanTypeOps implements PrimitiveTypeOps<BooleanWritable> {
+  /** Singleton instance */
+  INSTANCE();
+
+  @Override
+  public Class<BooleanWritable> getTypeClass() {
+    return BooleanWritable.class;
+  }
+
+  @Override
+  public BooleanWritable create() {
+    return new BooleanWritable();
+  }
+
+  @Override
+  public BooleanWritable createCopy(BooleanWritable from) {
+    return new BooleanWritable(from.get());
+  }
+
+  @Override
+  public void set(BooleanWritable to, BooleanWritable from) {
+    to.set(from.get());
+  }
+
+  @Override
+  public BasicBooleanArrayList createArrayList(int capacity) {
+    return new BasicBooleanArrayList(capacity);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/ByteTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/ByteTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/ByteTypeOps.java
new file mode 100644
index 0000000..2b27ba5
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/ByteTypeOps.java
@@ -0,0 +1,52 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.BasicArrayList.BasicByteArrayList;
+import org.apache.hadoop.io.ByteWritable;
+
+/** TypeOps implementation for working with ByteWritable type */
+public enum ByteTypeOps implements PrimitiveTypeOps<ByteWritable> {
+  /** Singleton instance */
+  INSTANCE();
+
+  @Override
+  public Class<ByteWritable> getTypeClass() {
+    return ByteWritable.class;
+  }
+
+  @Override
+  public ByteWritable create() {
+    return new ByteWritable();
+  }
+
+  @Override
+  public ByteWritable createCopy(ByteWritable from) {
+    return new ByteWritable(from.get());
+  }
+
+  @Override
+  public void set(ByteWritable to, ByteWritable from) {
+    to.set(from.get());
+  }
+
+  @Override
+  public BasicByteArrayList createArrayList(int capacity) {
+    return new BasicByteArrayList(capacity);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/DoubleTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/DoubleTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/DoubleTypeOps.java
new file mode 100644
index 0000000..af8c38f
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/DoubleTypeOps.java
@@ -0,0 +1,52 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.BasicArrayList.BasicDoubleArrayList;
+import org.apache.hadoop.io.DoubleWritable;
+
+/** TypeOps implementation for working with DoubleWritable type */
+public enum DoubleTypeOps implements PrimitiveTypeOps<DoubleWritable> {
+  /** Singleton instance */
+  INSTANCE();
+
+  @Override
+  public Class<DoubleWritable> getTypeClass() {
+    return DoubleWritable.class;
+  }
+
+  @Override
+  public DoubleWritable create() {
+    return new DoubleWritable();
+  }
+
+  @Override
+  public DoubleWritable createCopy(DoubleWritable from) {
+    return new DoubleWritable(from.get());
+  }
+
+  @Override
+  public void set(DoubleWritable to, DoubleWritable from) {
+    to.set(from.get());
+  }
+
+  @Override
+  public BasicDoubleArrayList createArrayList(int capacity) {
+    return new BasicDoubleArrayList(capacity);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/FloatTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/FloatTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/FloatTypeOps.java
new file mode 100644
index 0000000..3ca8409
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/FloatTypeOps.java
@@ -0,0 +1,52 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.BasicArrayList.BasicFloatArrayList;
+import org.apache.hadoop.io.FloatWritable;
+
+/** TypeOps implementation for working with FloatWritable type */
+public enum FloatTypeOps implements PrimitiveTypeOps<FloatWritable> {
+  /** Singleton instance */
+  INSTANCE();
+
+  @Override
+  public Class<FloatWritable> getTypeClass() {
+    return FloatWritable.class;
+  }
+
+  @Override
+  public FloatWritable create() {
+    return new FloatWritable();
+  }
+
+  @Override
+  public FloatWritable createCopy(FloatWritable from) {
+    return new FloatWritable(from.get());
+  }
+
+  @Override
+  public void set(FloatWritable to, FloatWritable from) {
+    to.set(from.get());
+  }
+
+  @Override
+  public BasicFloatArrayList createArrayList(int capacity) {
+    return new BasicFloatArrayList(capacity);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/IntTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/IntTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/IntTypeOps.java
new file mode 100644
index 0000000..f9a32c0
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/IntTypeOps.java
@@ -0,0 +1,68 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap;
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap.BasicInt2ObjectOpenHashMap;
+import org.apache.giraph.types.ops.collections.BasicArrayList;
+import org.apache.giraph.types.ops.collections.BasicArrayList.BasicIntArrayList;
+import org.apache.giraph.types.ops.collections.BasicSet;
+import org.apache.giraph.types.ops.collections.BasicSet.BasicIntOpenHashSet;
+import org.apache.hadoop.io.IntWritable;
+
+/** TypeOps implementation for working with IntWritable type */
+public enum IntTypeOps implements PrimitiveIdTypeOps<IntWritable> {
+  /** Singleton instance */
+  INSTANCE;
+
+  @Override
+  public Class<IntWritable> getTypeClass() {
+    return IntWritable.class;
+  }
+
+  @Override
+  public IntWritable create() {
+    return new IntWritable();
+  }
+
+  @Override
+  public IntWritable createCopy(IntWritable from) {
+    return new IntWritable(from.get());
+  }
+
+  @Override
+  public void set(IntWritable to, IntWritable from) {
+    to.set(from.get());
+  }
+
+  @Override
+  public BasicSet<IntWritable> createOpenHashSet(int capacity) {
+    return new BasicIntOpenHashSet(capacity);
+  }
+
+  @Override
+  public BasicArrayList<IntWritable> createArrayList(int capacity) {
+    return new BasicIntArrayList(capacity);
+  }
+
+  @Override
+  public <V> Basic2ObjectMap<IntWritable, V> create2ObjectOpenHashMap(
+      int capacity) {
+    return new BasicInt2ObjectOpenHashMap<>(capacity);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/LongTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/LongTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/LongTypeOps.java
new file mode 100644
index 0000000..4e5ca54
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/LongTypeOps.java
@@ -0,0 +1,68 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap;
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap.BasicLong2ObjectOpenHashMap;
+import org.apache.giraph.types.ops.collections.BasicArrayList;
+import org.apache.giraph.types.ops.collections.BasicArrayList.BasicLongArrayList;
+import org.apache.giraph.types.ops.collections.BasicSet;
+import org.apache.giraph.types.ops.collections.BasicSet.BasicLongOpenHashSet;
+import org.apache.hadoop.io.LongWritable;
+
+/** TypeOps implementation for working with LongWritable type */
+public enum LongTypeOps implements PrimitiveIdTypeOps<LongWritable> {
+  /** Singleton instance */
+  INSTANCE;
+
+  @Override
+  public Class<LongWritable> getTypeClass() {
+    return LongWritable.class;
+  }
+
+  @Override
+  public LongWritable create() {
+    return new LongWritable();
+  }
+
+  @Override
+  public LongWritable createCopy(LongWritable from) {
+    return new LongWritable(from.get());
+  }
+
+  @Override
+  public void set(LongWritable to, LongWritable from) {
+    to.set(from.get());
+  }
+
+  @Override
+  public BasicSet<LongWritable> createOpenHashSet(int capacity) {
+    return new BasicLongOpenHashSet(capacity);
+  }
+
+  @Override
+  public BasicArrayList<LongWritable> createArrayList(int capacity) {
+    return new BasicLongArrayList(capacity);
+  }
+
+  @Override
+  public <V> Basic2ObjectMap<LongWritable, V> create2ObjectOpenHashMap(
+      int capacity) {
+    return new BasicLong2ObjectOpenHashMap<>(capacity);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/MapTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/MapTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/MapTypeOps.java
new file mode 100644
index 0000000..cd9f079
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/MapTypeOps.java
@@ -0,0 +1,47 @@
+/*
+ * 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.types.ops;
+
+import org.apache.hadoop.io.MapWritable;
+
+/** TypeOps implementation for working with MapWritable type */
+public enum MapTypeOps implements TypeOps<MapWritable> {
+  /** Singleton instance */
+  INSTANCE();
+
+  @Override
+  public Class<MapWritable> getTypeClass() {
+    return MapWritable.class;
+  }
+
+  @Override
+  public MapWritable create() {
+    return new MapWritable();
+  }
+
+  @Override
+  public MapWritable createCopy(MapWritable from) {
+    return new MapWritable(from);
+  }
+
+  @Override
+  public void set(MapWritable to, MapWritable from) {
+    to.clear();
+    to.putAll(from);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveIdTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveIdTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveIdTypeOps.java
new file mode 100644
index 0000000..29b0c6e
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveIdTypeOps.java
@@ -0,0 +1,55 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.Basic2ObjectMap;
+import org.apache.giraph.types.ops.collections.BasicSet;
+
+
+
+/**
+ * Additional type operations to TypeOps for types that can be IDs,
+ * and so can be used as keys in maps and values in sets.
+ *
+ * Using any of the provided operations should lead to no boxing/unboxing.
+ *
+ * Useful generic wrappers to fastutil libraries are provided,
+ * so that you can look at them generically.
+ *
+ * @param <T> Type
+ */
+public interface PrimitiveIdTypeOps<T> extends PrimitiveTypeOps<T> {
+  // primitive collections
+
+  /**
+   * Create BasicSet of type T, given capacity.
+   * @param capacity Capacity
+   * @return BasicSet
+   */
+  BasicSet<T> createOpenHashSet(int capacity);
+
+  /**
+   * Create Basic2ObjectMap with key type T, given capacity.
+   * Values are represented as object, even if they can be primitive.
+   *
+   * @param capacity Capacity
+   * @param <V> Type of values in the map
+   * @return Basic2ObjectMap
+   */
+  <V> Basic2ObjectMap<T, V> create2ObjectOpenHashMap(int capacity);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveTypeOps.java
new file mode 100644
index 0000000..72b684f
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/PrimitiveTypeOps.java
@@ -0,0 +1,42 @@
+/*
+ * 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.types.ops;
+
+import org.apache.giraph.types.ops.collections.BasicArrayList;
+
+
+/**
+ * Type operations, allowing working generically with types,
+ * but still having efficient code.
+ *
+ * Using any of the provided operations should lead to no boxing/unboxing.
+ *
+ * Useful generic wrappers to fastutil libraries are provided,
+ * so that you can look at them generically.
+ *
+ * @param <T> Type
+ */
+public interface PrimitiveTypeOps<T> extends TypeOps<T> {
+  // primitive collections
+  /**
+   * Create BasicArrayList of type T, given capacity.
+   * @param capacity Capacity
+   * @return BasicArrayList
+   */
+  BasicArrayList<T> createArrayList(int capacity);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/TextTypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/TextTypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/TextTypeOps.java
new file mode 100644
index 0000000..c785cd9
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/TextTypeOps.java
@@ -0,0 +1,46 @@
+/*
+ * 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.types.ops;
+
+import org.apache.hadoop.io.Text;
+
+/** TypeOps implementation for working with Text type */
+public enum TextTypeOps implements TypeOps<Text> {
+  /** Singleton instance */
+  INSTANCE();
+
+  @Override
+  public Class<Text> getTypeClass() {
+    return Text.class;
+  }
+
+  @Override
+  public Text create() {
+    return new Text();
+  }
+
+  @Override
+  public Text createCopy(Text from) {
+    return new Text(from);
+  }
+
+  @Override
+  public void set(Text to, Text from) {
+    to.set(from.getBytes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOps.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOps.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOps.java
new file mode 100644
index 0000000..b7f9479
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOps.java
@@ -0,0 +1,51 @@
+/*
+ * 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.types.ops;
+
+
+/**
+ * Type operations, allowing working generically with mutable types,
+ * but still having efficient code.
+ * For example, by reducing object allocation via reuse.
+ *
+ * @param <T> Type
+ */
+public interface TypeOps<T> {
+  /**
+   * Class object for generic type T.
+   * @return Class<T> object
+   */
+  Class<T> getTypeClass();
+  /**
+   * Create new instance of type T.
+   * @return new instance
+   */
+  T create();
+  /**
+   * Create a copy of passed object
+   * @param from Object to copy
+   * @return Copy
+   */
+  T createCopy(T from);
+  /**
+   * Copies value from first argument into the second.
+   * @param to Value of object to be copied
+   * @param from Object into which value should be copied
+   */
+  void set(T to, T from);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOpsUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOpsUtils.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOpsUtils.java
new file mode 100644
index 0000000..df5f2bd
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/TypeOpsUtils.java
@@ -0,0 +1,149 @@
+/*
+ * 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.types.ops;
+
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.ByteWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Utility functions for getting TypeOps instances from class types.
+ */
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TypeOpsUtils {
+  /** No instances */
+  private TypeOpsUtils() { }
+
+  /**
+   * Get PrimitiveIdTypeOps for given type, or null if there is none.
+   * @param type Class type
+   * @param <T> Type
+   * @return PrimitiveIdTypeOps
+   */
+  public static <T>
+  PrimitiveIdTypeOps<T> getPrimitiveIdTypeOpsOrNull(Class<T> type) {
+    if (type.equals(LongWritable.class)) {
+      return (PrimitiveIdTypeOps) LongTypeOps.INSTANCE;
+    } else if (type.equals(IntWritable.class)) {
+      return (PrimitiveIdTypeOps) IntTypeOps.INSTANCE;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Get PrimitiveIdTypeOps for given type.
+   * Exception will be thrown if there is none.
+   * @param type Class type
+   * @param <T> Type
+   * @return PrimitiveIdTypeOps
+   */
+  public static <T>
+  PrimitiveIdTypeOps<T> getPrimitiveIdTypeOps(Class<T> type) {
+    PrimitiveIdTypeOps<T> typeOps = getPrimitiveIdTypeOpsOrNull(type);
+    if (typeOps != null) {
+      return typeOps;
+    } else {
+      throw new IllegalArgumentException(
+          type + " not supported in PrimitiveIdTypeOps");
+    }
+  }
+
+  /**
+   * Get PrimitiveTypeOps for given type, or null if there is none.
+   * @param type Class type
+   * @param <T> Type
+   * @return PrimitiveTypeOps
+   */
+  public static <T>
+  PrimitiveTypeOps<T> getPrimitiveTypeOpsOrNull(Class<T> type) {
+    PrimitiveTypeOps<T> typeOps = getPrimitiveIdTypeOpsOrNull(type);
+    if (typeOps != null) {
+      return typeOps;
+    } else if (type.equals(FloatWritable.class)) {
+      return (PrimitiveTypeOps) FloatTypeOps.INSTANCE;
+    } else if (type.equals(DoubleWritable.class)) {
+      return (PrimitiveTypeOps) DoubleTypeOps.INSTANCE;
+    } else if (type.equals(BooleanWritable.class)) {
+      return (PrimitiveTypeOps) BooleanTypeOps.INSTANCE;
+    } else if (type.equals(ByteWritable.class)) {
+      return (PrimitiveTypeOps) ByteTypeOps.INSTANCE;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Get PrimitiveTypeOps for given type.
+   * Exception will be thrown if there is none.
+   * @param type Class type
+   * @param <T> Type
+   * @return PrimitiveTypeOps
+   */
+  public static <T>
+  PrimitiveTypeOps<T> getPrimitiveTypeOps(Class<T> type) {
+    PrimitiveTypeOps<T> typeOps = getPrimitiveTypeOpsOrNull(type);
+    if (typeOps != null) {
+      return typeOps;
+    } else {
+      throw new IllegalArgumentException(
+          type + " not supported in PrimitiveTypeOps");
+    }
+  }
+
+  /**
+   * Get TypeOps for given type, or null if there is none.
+   * @param type Class type
+   * @param <T> Type
+   * @return TypeOps
+   */
+  public static <T> TypeOps<T> getTypeOpsOrNull(Class<T> type) {
+    TypeOps<T> typeOps = getPrimitiveTypeOpsOrNull(type);
+    if (typeOps != null) {
+      return typeOps;
+    } else if (type.equals(Text.class)) {
+      return (TypeOps) TextTypeOps.INSTANCE;
+    } else if (type.equals(MapWritable.class)) {
+      return (TypeOps) MapTypeOps.INSTANCE;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Get TypeOps for given type.
+   * Exception will be thrown if there is none.
+   * @param type Class type
+   * @param <T> Type
+   * @return TypeOps
+   */
+  public static <T> TypeOps<T> getTypeOps(Class<T> type) {
+    TypeOps<T> typeOps = getTypeOpsOrNull(type);
+    if (typeOps != null) {
+      return typeOps;
+    } else {
+      throw new IllegalArgumentException(
+          type + " not supported in TypeOps");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/18520570/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/Basic2ObjectMap.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/Basic2ObjectMap.java b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/Basic2ObjectMap.java
new file mode 100644
index 0000000..f7ef570
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/types/ops/collections/Basic2ObjectMap.java
@@ -0,0 +1,322 @@
+/*
+ * 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.types.ops.collections;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.ints.IntIterator;
+import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
+import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
+import it.unimi.dsi.fastutil.longs.LongIterator;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.giraph.types.ops.IntTypeOps;
+import org.apache.giraph.types.ops.LongTypeOps;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Basic2ObjectMap with only basic set of operations.
+ * All operations that return object T are returning reusable object,
+ * which is modified after calling any other function.
+ *
+ * @param <K> Key type
+ * @param <V> Value type
+ */
+public abstract class Basic2ObjectMap<K, V> {
+  /** Removes all of the elements from this list. */
+  public abstract void clear();
+  /**
+   * Number of elements in this list
+   * @return size
+   */
+  public abstract int size();
+
+  /**
+   * Checks whether key is present in the map
+   * @param key Key
+   * @return true if present
+   */
+  public abstract boolean containsKey(K key);
+  /**
+   * Adds a pair to the map.
+   *
+   * @param key Key
+   * @param value Value.
+   * @return the old value, or null if no value was present for the given key.
+   */
+  public abstract V put(K key, V value);
+  /**
+   * Get value for a given key
+   * @param key Key
+   * @return Value, or null
+   */
+  public abstract V get(K key);
+  /**
+   * Removes the mapping with the given key.
+   *
+   * @param key Key
+   * @return the old value, or null if no value was present for the given key.
+   */
+  public abstract V remove(K key);
+
+  /**
+   * TypeOps for type of keys this object holds
+   * @return TypeOps
+   */
+  public abstract PrimitiveIdTypeOps<K> getKeyTypeOps();
+
+  /**
+   * Fast iterator over keys within this map, which doesn't allocate new
+   * element for each returned element.
+   *
+   * Object returned by next() is only valid until next() is called again,
+   * because it is reused.
+   *
+   * @return Iterator
+   */
+  public abstract Iterator<K> fastKeyIterator();
+
+  /**
+   * Serializes the object, given a writer for values.
+   * @param out <code>DataOuput</code> to serialize object into.
+   * @param writer Writer of values
+   * @throws IOException
+   */
+  public abstract void write(DataOutput out, WritableWriter<V> writer)
+    throws IOException;
+  /**
+   * Deserialize the object, given a writer for values.
+   * @param in <code>DataInput</code> to deseriablize object from.
+   * @param writer Writer of values
+   * @throws IOException
+   */
+  public abstract void readFields(DataInput in, WritableWriter<V> writer)
+    throws IOException;
+
+  /**
+   * Iterator that reuses key object.
+   *
+   * @param <Iter> Primitive key iterator type
+   */
+  protected abstract class ReusableIterator<Iter extends Iterator<?>>
+      implements Iterator<K> {
+    /** Primitive Key iterator */
+    protected final Iter iter;
+    /** Reusable key object */
+    protected final K reusableKey = getKeyTypeOps().create();
+
+    /**
+     * Constructor
+     * @param iter Primitive Key iterator
+     */
+    public ReusableIterator(Iter iter) {
+      this.iter = iter;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return iter.hasNext();
+    }
+
+    @Override
+    public void remove() {
+      iter.remove();
+    }
+  }
+
+  /** IntWritable implementation of Basic2ObjectMap */
+  public static final class BasicInt2ObjectOpenHashMap<V>
+      extends Basic2ObjectMap<IntWritable, V> {
+    /** Map */
+    private final Int2ObjectOpenHashMap<V> map;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicInt2ObjectOpenHashMap(int capacity) {
+      this.map = new Int2ObjectOpenHashMap<>(capacity);
+    }
+
+    @Override
+    public void clear() {
+      map.clear();
+    }
+
+    @Override
+    public int size() {
+      return map.size();
+    }
+
+    @Override
+    public boolean containsKey(IntWritable key) {
+      return map.containsKey(key.get());
+    }
+
+    @Override
+    public V put(IntWritable key, V value) {
+      return map.put(key.get(), value);
+    }
+
+    @Override
+    public V get(IntWritable key) {
+      return map.get(key.get());
+    }
+
+    @Override
+    public V remove(IntWritable key) {
+      return map.remove(key.get());
+    }
+
+    @Override
+    public PrimitiveIdTypeOps<IntWritable> getKeyTypeOps() {
+      return IntTypeOps.INSTANCE;
+    }
+
+    @Override
+    public Iterator<IntWritable> fastKeyIterator() {
+      return new ReusableIterator<IntIterator>(map.keySet().iterator()) {
+        @Override
+        public IntWritable next() {
+          reusableKey.set(iter.nextInt());
+          return reusableKey;
+        }
+      };
+    }
+
+    @Override
+    public void write(DataOutput out, WritableWriter<V> writer)
+      throws IOException {
+      out.writeInt(map.size());
+      ObjectIterator<Int2ObjectMap.Entry<V>> iterator =
+          map.int2ObjectEntrySet().fastIterator();
+      while (iterator.hasNext()) {
+        Int2ObjectMap.Entry<V> entry = iterator.next();
+        out.writeInt(entry.getIntKey());
+        writer.write(out, entry.getValue());
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in, WritableWriter<V> writer)
+      throws IOException {
+      int size = in.readInt();
+      map.clear();
+      map.trim(size);
+      while (size-- > 0) {
+        int key = in.readInt();
+        V value = writer.readFields(in);
+        map.put(key, value);
+      }
+    }
+  }
+
+  /** LongWritable implementation of Basic2ObjectMap */
+  public static final class BasicLong2ObjectOpenHashMap<V>
+      extends Basic2ObjectMap<LongWritable, V> {
+    /** Map */
+    private final Long2ObjectOpenHashMap<V> map;
+
+    /**
+     * Constructor
+     * @param capacity Capacity
+     */
+    public BasicLong2ObjectOpenHashMap(int capacity) {
+      this.map = new Long2ObjectOpenHashMap<>(capacity);
+    }
+
+    @Override
+    public void clear() {
+      map.clear();
+    }
+
+    @Override
+    public int size() {
+      return map.size();
+    }
+
+    @Override
+    public boolean containsKey(LongWritable key) {
+      return map.containsKey(key.get());
+    }
+
+    @Override
+    public V put(LongWritable key, V value) {
+      return map.put(key.get(), value);
+    }
+
+    @Override
+    public V get(LongWritable key) {
+      return map.get(key.get());
+    }
+
+    @Override
+    public V remove(LongWritable key) {
+      return map.remove(key.get());
+    }
+
+    @Override
+    public PrimitiveIdTypeOps<LongWritable> getKeyTypeOps() {
+      return LongTypeOps.INSTANCE;
+    }
+
+    @Override
+    public Iterator<LongWritable> fastKeyIterator() {
+      return new ReusableIterator<LongIterator>(map.keySet().iterator()) {
+        @Override
+        public LongWritable next() {
+          reusableKey.set(iter.nextLong());
+          return reusableKey;
+        }
+      };
+    }
+
+    @Override
+    public void write(DataOutput out, WritableWriter<V> writer)
+      throws IOException {
+      out.writeInt(map.size());
+      ObjectIterator<Long2ObjectMap.Entry<V>> iterator =
+          map.long2ObjectEntrySet().fastIterator();
+      while (iterator.hasNext()) {
+        Long2ObjectMap.Entry<V> entry = iterator.next();
+        out.writeLong(entry.getLongKey());
+        writer.write(out, entry.getValue());
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in, WritableWriter<V> writer)
+      throws IOException {
+      int size = in.readInt();
+      map.clear();
+      map.trim(size);
+      while (size-- > 0) {
+        long key = in.readLong();
+        V value = writer.readFields(in);
+        map.put(key, value);
+      }
+    }
+  }
+}


[43/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-946. Upgrade to Gora 0.5 (Renato Javier Marroquín Mogrovejo via rvs)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/7c61dcf4
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/7c61dcf4
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/7c61dcf4

Branch: refs/heads/release-1.1
Commit: 7c61dcf4abaa422637a68839bf0d547a164375a3
Parents: 0bcc76d
Author: Roman Shaposhnik <rv...@apache.org>
Authored: Sun Oct 19 16:54:39 2014 -0700
Committer: Roman Shaposhnik <rv...@apache.org>
Committed: Sun Oct 19 16:54:39 2014 -0700

----------------------------------------------------------------------
 giraph-gora/conf/edge_result.avsc               |  12 +
 giraph-gora/conf/vertex.avsc                    |  10 +
 giraph-gora/conf/vertex.json                    |  18 -
 giraph-gora/conf/vertex_result.avsc             |  10 +
 giraph-gora/pom.xml                             | 241 +++---
 .../giraph/io/gora/GoraEdgeInputFormat.java     |  12 +-
 .../giraph/io/gora/GoraEdgeOutputFormat.java    |  10 +-
 .../io/gora/GoraGEdgeEdgeInputFormat.java       |   9 +-
 .../io/gora/GoraGEdgeEdgeOutputFormat.java      |   5 +-
 .../io/gora/GoraGVertexVertexInputFormat.java   |  10 +-
 .../io/gora/GoraGVertexVertexOutputFormat.java  |  11 +-
 .../giraph/io/gora/GoraVertexInputFormat.java   |  14 +-
 .../giraph/io/gora/GoraVertexOutputFormat.java  |   8 +-
 .../apache/giraph/io/gora/generated/GEdge.java  | 842 +++++++++++++++---
 .../giraph/io/gora/generated/GEdgeResult.java   | 849 ++++++++++++++++---
 .../giraph/io/gora/generated/GVertex.java       | 641 +++++++++++---
 .../giraph/io/gora/generated/GVertexResult.java | 644 +++++++++++---
 .../io/gora/utils/ExtraGoraInputFormat.java     |   9 +-
 .../apache/giraph/io/gora/utils/GoraUtils.java  |  42 +-
 .../giraph/io/gora/GoraTestEdgeInputFormat.java |  14 +-
 .../io/gora/GoraTestEdgeOutputFormat.java       |  25 +-
 .../io/gora/GoraTestVertexInputFormat.java      |  12 +-
 .../io/gora/GoraTestVertexOutputFormat.java     |  23 +-
 .../giraph/io/gora/TestGoraEdgeInputFormat.java |   4 +-
 .../io/gora/TestGoraEdgeOutputFormat.java       |   2 +-
 pom.xml                                         |   4 +-
 26 files changed, 2728 insertions(+), 753 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/conf/edge_result.avsc
----------------------------------------------------------------------
diff --git a/giraph-gora/conf/edge_result.avsc b/giraph-gora/conf/edge_result.avsc
new file mode 100644
index 0000000..9319fa1
--- /dev/null
+++ b/giraph-gora/conf/edge_result.avsc
@@ -0,0 +1,12 @@
+{
+  "type": "record",
+  "name": "GEdgeResult",
+  "namespace": "org.apache.giraph.io.gora.generated",
+  "fields" : [
+    {"name": "edgeId", "type": "string"},
+    {"name": "edgeWeight", "type": "float"},
+    {"name": "vertexInId", "type": "string"},
+    {"name": "vertexOutId", "type": "string"},
+    {"name": "label", "type": "string"}
+  ]
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/conf/vertex.avsc
----------------------------------------------------------------------
diff --git a/giraph-gora/conf/vertex.avsc b/giraph-gora/conf/vertex.avsc
new file mode 100644
index 0000000..88d4234
--- /dev/null
+++ b/giraph-gora/conf/vertex.avsc
@@ -0,0 +1,10 @@
+{
+  "type": "record",
+  "name": "GVertex",
+  "namespace": "org.apache.giraph.io.gora.generated",
+  "fields" : [
+    {"name": "vertexId", "type": "string"},
+    {"name": "vertexValue", "type": "float"},
+    {"name": "edges", "type": {"type": "map", "values": "string"}}
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/conf/vertex.json
----------------------------------------------------------------------
diff --git a/giraph-gora/conf/vertex.json b/giraph-gora/conf/vertex.json
deleted file mode 100644
index 9f435fa..0000000
--- a/giraph-gora/conf/vertex.json
+++ /dev/null
@@ -1,18 +0,0 @@
-{
-  "type": "record",
-  "name": "Vertex",
-  "namespace": "org.apache.giraph.gora.generated",
-  "fields" : [
-    {"name": "vertexId", "type": "long"},
-    {"name": "value", "type": "float"},
-    {"name": "edges", "type": {"type":"array", "items": {
-      "name": "Edge",
-      "type": "record",
-      "namespace": "org.apache.giraph.gora.generated",
-      "fields": [
-        {"name": "vertexId", "type": "long"},
-        {"name": "edgeValue", "type": "float"}
-      ]
-    }}}
-  ]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/conf/vertex_result.avsc
----------------------------------------------------------------------
diff --git a/giraph-gora/conf/vertex_result.avsc b/giraph-gora/conf/vertex_result.avsc
new file mode 100644
index 0000000..b594ca9
--- /dev/null
+++ b/giraph-gora/conf/vertex_result.avsc
@@ -0,0 +1,10 @@
+{
+  "type": "record",
+  "name": "GVertexResult",
+  "namespace": "org.apache.giraph.io.gora.generated",
+  "fields" : [
+    {"name": "vertexId", "type": "string"},
+    {"name": "vertexValue", "type": "float"},
+    {"name": "edges", "type": {"type": "map", "values": "string"}}
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-gora/pom.xml b/giraph-gora/pom.xml
index 38e3ab7..7f499c6 100644
--- a/giraph-gora/pom.xml
+++ b/giraph-gora/pom.xml
@@ -1,122 +1,139 @@
-<!--
-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.
--->
+<!-- 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. -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
 
-  <parent>
-    <groupId>org.apache.giraph</groupId>
-    <artifactId>giraph-parent</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>giraph-gora</artifactId>
-  <packaging>jar</packaging>
+    <parent>
+        <groupId>org.apache.giraph</groupId>
+        <artifactId>giraph-parent</artifactId>
+        <version>1.1.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>giraph-gora</artifactId>
+    <packaging>jar</packaging>
 
-  <name>Apache Giraph Gora I/O</name>
-  <url>http://gora.apache.org/giraph-gora/</url>
-  <description>Giraph Gora input/output classes</description>
+    <name>Apache Giraph Gora I/O</name>
+    <url>http://gora.apache.org/giraph-gora/</url>
+    <description>Giraph Gora input/output classes</description>
 
-  <properties>
-    <top.dir>${project.basedir}/..</top.dir>
-  </properties>
+    <properties>
+        <top.dir>${project.basedir}/..</top.dir>
+    </properties>
 
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-assembly-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-javadoc-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-site-plugin</artifactId>
-        <configuration>
-          <siteDirectory>${project.basedir}/src/site</siteDirectory>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.6</version>
-        <configuration>
-          <systemProperties>
-            <property>
-              <name>prop.jarLocation</name>
-              <value>${top.dir}/giraph-core/target/giraph-${project.version}-${forHadoop}-jar-with-dependencies.jar</value>
-            </property>
-          </systemProperties>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-      </plugin>
-    </plugins>
-  </build>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-site-plugin</artifactId>
+                <configuration>
+                    <siteDirectory>${project.basedir}/src/site</siteDirectory>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>2.6</version>
+                <configuration>
+                    <systemProperties>
+                        <property>
+                            <name>prop.jarLocation</name>
+                            <value>${top.dir}/giraph-core/target/giraph-${project.version}-${forHadoop}-jar-with-dependencies.jar</value>
+                        </property>
+                    </systemProperties>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>findbugs-maven-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
 
-  <dependencies>
-    <!-- compile dependencies. sorted lexicographically. -->
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.giraph</groupId>
-      <artifactId>giraph-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.gora</groupId>
-      <artifactId>gora-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-    </dependency>
+    <dependencies>
+        <!-- compile dependencies. sorted lexicographically. -->
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.giraph</groupId>
+            <artifactId>giraph-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.gora</groupId>
+            <artifactId>gora-core</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.codehaus.jackson</groupId>
+                    <artifactId>jackson-core-asl</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.jackson</groupId>
+                    <artifactId>jackson-mapper-asl</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.gora</groupId>
+                    <artifactId>gora-shims-distribution</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>${dep.avro.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.codehaus.jackson</groupId>
+                    <artifactId>jackson-core-asl</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.jackson</groupId>
+                    <artifactId>jackson-mapper-asl</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+        </dependency>
 
-    <!-- runtime dependency -->
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-      <scope>runtime</scope>
-    </dependency>
+        <!-- runtime dependency -->
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <scope>runtime</scope>
+        </dependency>
 
-    <!-- test dependencies. sorted lexicographically. -->
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
+        <!-- test dependencies. sorted lexicographically. -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeInputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeInputFormat.java
index d0dcc32..8ef91f6 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeInputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeInputFormat.java
@@ -34,8 +34,8 @@ import org.apache.giraph.io.gora.utils.ExtraGoraInputFormat;
 import org.apache.giraph.io.gora.utils.GoraUtils;
 import org.apache.giraph.io.gora.utils.KeyFactory;
 import org.apache.gora.persistency.Persistent;
-import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
+import org.apache.gora.query.impl.QueryBase;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.util.GoraException;
 import org.apache.hadoop.conf.Configuration;
@@ -114,7 +114,7 @@ public abstract class GoraEdgeInputFormat
       setPersistentClass((Class<? extends Persistent>) persistentClass);
       setDatastoreClass((Class<? extends DataStore>) dataStoreClass);
       setKeyFactoryClass(keyFactoryClass);
-      setDataStore(createDataStore());
+      setDataStore(createDataStore(getConf()));
       GORA_INPUT_FORMAT.setDataStore(getDataStore());
     } catch (ClassNotFoundException e) {
       LOG.error("Error while reading Gora Input parameters");
@@ -154,8 +154,9 @@ public abstract class GoraEdgeInputFormat
     kFact.setDataStore(getDataStore());
     setStartKey(kFact.buildKey(sKey));
     setEndKey(kFact.buildKey(eKey));
-    Query tmpQuery = GoraUtils.getQuery(
+    QueryBase tmpQuery = GoraUtils.getQuery(
         getDataStore(), getStartKey(), getEndKey());
+    tmpQuery.setConf(context.getConfiguration());
     GORA_INPUT_FORMAT.setQuery(tmpQuery);
     List<InputSplit> splits = GORA_INPUT_FORMAT.getSplits(context);
     return splits;
@@ -273,12 +274,13 @@ public abstract class GoraEdgeInputFormat
 
   /**
    * Gets the data store object initialized.
+   * @param conf Configuration
    * @return DataStore created
    */
-  public DataStore createDataStore() {
+  public DataStore createDataStore(Configuration conf) {
     DataStore dsCreated = null;
     try {
-      dsCreated = GoraUtils.createSpecificDataStore(getDatastoreClass(),
+      dsCreated = GoraUtils.createSpecificDataStore(conf, getDatastoreClass(),
           getKeyClass(), getPersistentClass());
     } catch (GoraException e) {
       LOG.error("Error creating data store.");

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeOutputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeOutputFormat.java
index be9f472..d030d70 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeOutputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraEdgeOutputFormat.java
@@ -30,11 +30,12 @@ import org.apache.giraph.io.gora.utils.GoraUtils;
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.util.GoraException;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
 /**
@@ -83,12 +84,13 @@ public abstract class GoraEdgeOutputFormat<I extends WritableComparable,
 
   /**
    * Gets the data store object initialized.
+   * @param conf Configuration
    * @return DataStore created
    */
-  public DataStore createDataStore() {
+  public DataStore createDataStore(Configuration conf) {
     DataStore dsCreated = null;
     try {
-      dsCreated = GoraUtils.createSpecificDataStore(getDatastoreClass(),
+      dsCreated = GoraUtils.createSpecificDataStore(conf, getDatastoreClass(),
           getKeyClass(), getPersistentClass());
     } catch (GoraException e) {
       getLogger().error("Error creating data store.");
@@ -159,7 +161,7 @@ public abstract class GoraEdgeOutputFormat<I extends WritableComparable,
         setKeyClass(keyClass);
         setPersistentClass((Class<? extends Persistent>) persistentClass);
         setDatastoreClass((Class<? extends DataStore>) dataStoreClass);
-        setDataStore(createDataStore());
+        setDataStore(createDataStore(context.getConfiguration()));
         if (getDataStore() != null) {
           getLogger().debug("The data store has been created.");
         }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeInputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeInputFormat.java
index e738f36..a2a3417 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeInputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeInputFormat.java
@@ -69,14 +69,11 @@ public class GoraGEdgeEdgeInputFormat
     (Object goraObject) {
       Edge<LongWritable, FloatWritable> edge = null;
       GEdge goraEdge = (GEdge) goraObject;
-      Long dest;
-      Long value;
-      dest = Long.valueOf(goraEdge.getVertexOutId().toString());
       this.sourceId = new LongWritable();
       this.sourceId.set(Long.valueOf(goraEdge.getVertexInId().toString()));
-      value = (long) goraEdge.getEdgeWeight();
-      edge = EdgeFactory.create(new LongWritable(dest),
-          new FloatWritable(value));
+      edge = EdgeFactory.create(
+          new LongWritable(Long.valueOf(goraEdge.getVertexOutId().toString())),
+          new FloatWritable(goraEdge.getEdgeWeight()));
       return edge;
     }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeOutputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeOutputFormat.java
index d350d37..82b02d0 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeOutputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGEdgeEdgeOutputFormat.java
@@ -19,7 +19,6 @@ package org.apache.giraph.io.gora;
 
 import java.io.IOException;
 
-import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.io.gora.generated.GEdgeResult;
 import org.apache.gora.persistency.Persistent;
@@ -57,9 +56,9 @@ public class GoraGEdgeEdgeOutputFormat
     protected Persistent getGoraEdge(LongWritable srcId,
         DoubleWritable srcValue, Edge<LongWritable, FloatWritable> edge) {
       GEdgeResult tmpGEdge = new GEdgeResult();
-      tmpGEdge.setEdgeId(new Utf8(srcId.toString()));
+      tmpGEdge.setEdgeId(srcId.toString());
       tmpGEdge.setEdgeWeight(edge.getValue().get());
-      tmpGEdge.setVertexOutId(new Utf8(edge.getTargetVertexId().toString()));
+      tmpGEdge.setVertexOutId(edge.getTargetVertexId().toString());
       getLogger().debug("GoraObject created: " + tmpGEdge.toString());
       return tmpGEdge;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexInputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexInputFormat.java
index cb0f005..34ba7ec 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexInputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexInputFormat.java
@@ -20,7 +20,6 @@ package org.apache.giraph.io.gora;
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
@@ -75,12 +74,13 @@ public class GoraGVertexVertexInputFormat
       GVertex tmpGVertex = (GVertex) goraObject;
 
       LongWritable vrtxId = new LongWritable(
-          Long.parseLong(tmpGVertex.getVertexId().toString()));
-      DoubleWritable vrtxValue = new DoubleWritable(tmpGVertex.getValue());
+        Long.parseLong(tmpGVertex.getVertexId().toString()));
+      DoubleWritable vrtxValue = new DoubleWritable(
+        tmpGVertex.getVertexValue());
       vertex.initialize(vrtxId, vrtxValue);
       if (tmpGVertex.getEdges() != null && !tmpGVertex.getEdges().isEmpty()) {
-        Set<Utf8> keyIt = tmpGVertex.getEdges().keySet();
-        for (Utf8 key : keyIt) {
+        Set<CharSequence> keyIt = tmpGVertex.getEdges().keySet();
+        for (CharSequence key : keyIt) {
           String keyVal = key.toString();
           String valVal = tmpGVertex.getEdges().get(key).toString();
           Edge<LongWritable, FloatWritable> edge;

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexOutputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexOutputFormat.java
index 893e083..4a9c229 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexOutputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraGVertexVertexOutputFormat.java
@@ -20,7 +20,6 @@ package org.apache.giraph.io.gora;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.avro.util.Utf8;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexWriter;
@@ -60,15 +59,15 @@ public class GoraGVertexVertexOutputFormat
     protected Persistent getGoraVertex(
         Vertex<LongWritable, DoubleWritable, FloatWritable> vertex) {
       GVertexResult tmpGVertex = new GVertexResult();
-      tmpGVertex.setVertexId(new Utf8(vertex.getId().toString()));
-      tmpGVertex.setValue(Float.parseFloat(vertex.getValue().toString()));
+      tmpGVertex.setVertexId(vertex.getId().toString());
+      tmpGVertex.setVertexValue(Float.parseFloat(vertex.getValue().toString()));
       Iterator<Edge<LongWritable, FloatWritable>> it =
           vertex.getEdges().iterator();
       while (it.hasNext()) {
         Edge<LongWritable, FloatWritable> edge = it.next();
-        tmpGVertex.putToEdges(
-            new Utf8(edge.getTargetVertexId().toString()),
-            new Utf8(edge.getValue().toString()));
+        tmpGVertex.getEdges().put(
+            edge.getTargetVertexId().toString(),
+            edge.getValue().toString());
       }
       return tmpGVertex;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexInputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexInputFormat.java
index 9a6ad8c..ee5e7b6 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexInputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexInputFormat.java
@@ -30,12 +30,12 @@ import java.util.List;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexReader;
-import org.apache.giraph.io.gora.utils.KeyFactory;
 import org.apache.giraph.io.gora.utils.ExtraGoraInputFormat;
 import org.apache.giraph.io.gora.utils.GoraUtils;
+import org.apache.giraph.io.gora.utils.KeyFactory;
 import org.apache.gora.persistency.Persistent;
-import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
+import org.apache.gora.query.impl.QueryBase;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.util.GoraException;
 import org.apache.hadoop.conf.Configuration;
@@ -115,7 +115,7 @@ public abstract class GoraVertexInputFormat<
       setPersistentClass((Class<? extends Persistent>) persistentClass);
       setDatastoreClass((Class<? extends DataStore>) dataStoreClass);
       setKeyFactoryClass(keyFactoryClass);
-      setDataStore(createDataStore());
+      setDataStore(createDataStore(conf));
       GORA_INPUT_FORMAT.setDataStore(getDataStore());
     } catch (ClassNotFoundException e) {
       LOG.error("Error while reading Gora Input parameters");
@@ -170,8 +170,9 @@ public abstract class GoraVertexInputFormat<
       setStartKey(kFact.buildKey(sKey));
       setEndKey(kFact.buildKey(eKey));
     }
-    Query tmpQuery = GoraUtils.getQuery(
+    QueryBase tmpQuery = GoraUtils.getQuery(
         getDataStore(), getStartKey(), getEndKey());
+    tmpQuery.setConf(getConf());
     GORA_INPUT_FORMAT.setQuery(tmpQuery);
     List<InputSplit> splits = GORA_INPUT_FORMAT.getSplits(context);
     return splits;
@@ -179,12 +180,13 @@ public abstract class GoraVertexInputFormat<
 
   /**
    * Gets the data store object initialized.
+   * @param conf Configuration
    * @return DataStore created
    */
-  public DataStore createDataStore() {
+  public DataStore createDataStore(Configuration conf) {
     DataStore dsCreated = null;
     try {
-      dsCreated = GoraUtils.createSpecificDataStore(getDatastoreClass(),
+      dsCreated = GoraUtils.createSpecificDataStore(conf, getDatastoreClass(),
           getKeyClass(), getPersistentClass());
     } catch (GoraException e) {
       LOG.error("Error creating data store.");

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexOutputFormat.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexOutputFormat.java
index 5fcc684..93143da 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexOutputFormat.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/GoraVertexOutputFormat.java
@@ -30,6 +30,7 @@ import org.apache.giraph.io.gora.utils.GoraUtils;
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.util.GoraException;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -89,12 +90,13 @@ public abstract class GoraVertexOutputFormat<
 
   /**
    * Gets the data store object initialized.
+   * @param conf Configuration.
    * @return DataStore created
    */
-  public DataStore createDataStore() {
+  public DataStore createDataStore(Configuration conf) {
     DataStore dsCreated = null;
     try {
-      dsCreated = GoraUtils.createSpecificDataStore(getDatastoreClass(),
+      dsCreated = GoraUtils.createSpecificDataStore(conf, getDatastoreClass(),
           getKeyClass(), getPersistentClass());
     } catch (GoraException e) {
       getLogger().error("Error creating data store.");
@@ -166,7 +168,7 @@ public abstract class GoraVertexOutputFormat<
         setKeyClass(keyClass);
         setPersistentClass((Class<? extends Persistent>) persistentClass);
         setDatastoreClass((Class<? extends DataStore>) dataStoreClass);
-        setDataStore(createDataStore());
+        setDataStore(createDataStore(context.getConfiguration()));
         if (getDataStore() != null) {
           getLogger().info("The output data store has been created.");
         }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdge.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdge.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdge.java
index f6ac3f7..c8b0667 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdge.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdge.java
@@ -15,35 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.giraph.io.gora.generated;
 
-import org.apache.avro.Schema;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.util.Utf8;
-import org.apache.gora.persistency.StateManager;
-import org.apache.gora.persistency.impl.PersistentBase;
-import org.apache.gora.persistency.impl.StateManagerImpl;
-
 /**
  * Example class for defining a Giraph-Edge.
  */
 @SuppressWarnings("all")
-public class GEdge extends PersistentBase {
-  /**
-   * Schema used for the class.
-   */
-  public static final Schema OBJ_SCHEMA = Schema.parse("{\"type\":\"record\"," +
-    "\"name\":\"GEdge\",\"namespace\":\"org.apache.giraph.gora.generated\"," +
-    "\"fields\":[{\"name\":\"edgeId\",\"type\":\"string\"}," +
-    "{\"name\":\"edgeWeight\",\"type\":\"float\"}," +
-    "{\"name\":\"vertexInId\",\"type\":\"string\"}," +
-    "{\"name\":\"vertexOutId\",\"type\":\"string\"}," +
-    "{\"name\":\"label\",\"type\":\"string\"}]}");
+public class GEdge extends org.apache.gora.persistency.impl.PersistentBase
+    implements org.apache.avro.specific.SpecificRecord,
+    org.apache.gora.persistency.Persistent {
 
   /**
-   * Field enum
+   * Schema used for the class.
    */
+  public static final org.apache.avro.Schema SCHEMAS =
+      new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\"," +
+          "\"name\":\"GEdge\"," +
+          "\"namespace\":\"org.apache.giraph.gora.generated\"," +
+          "\"fields\":[{\"name\":\"edgeId\",\"type\":\"string\"}," +
+          "{\"name\":\"edgeWeight\",\"type\":\"float\"}," +
+          "{\"name\":\"vertexInId\",\"type\":\"string\"}," +
+          "{\"name\":\"vertexOutId\",\"type\":\"string\"}," +
+          "{\"name\":\"label\",\"type\":\"string\"}]}");
+
+  /** Enum containing all data bean's fields. */
   public static enum Field {
     /**
      * Edge id.
@@ -71,19 +66,19 @@ public class GEdge extends PersistentBase {
     LABEL(4, "label");
 
     /**
-     * Field index
+     * Field's index.
      */
     private int index;
 
     /**
-     * Field name
+     * Field's name.
      */
     private String name;
 
     /**
-     * Field constructor
-     * @param index of attribute
-     * @param name of attribute
+     * Field's constructor
+     * @param index field's index.
+     * @param name field's name.
      */
     Field(int index, String name) {
       this.index = index;
@@ -91,24 +86,24 @@ public class GEdge extends PersistentBase {
     }
 
     /**
-     * Gets index
-     * @return int of attribute.
+     * Gets field's index.
+     * @return int field's index.
      */
     public int getIndex() {
       return index;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's name.
+     * @return String field's name.
      */
     public String getName() {
       return name;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's attributes to string.
+     * @return String field's attributes to string.
      */
     public String toString() {
       return name;
@@ -119,77 +114,61 @@ public class GEdge extends PersistentBase {
    * Array containing all fields/
    */
   private static final String[] ALL_FIELDS = {
-    "edgeId", "edgeWeight", "vertexInId", "vertexOutId", "label"
-  };
-
-  static {
-    PersistentBase.registerFields(GEdge.class, ALL_FIELDS);
-  }
-
-  /**
-   * edgeId
-   */
-  private Utf8 edgeId;
+    "edgeId", "edgeWeight", "vertexInId", "vertexOutId", "label"};
 
   /**
-   * edgeWeight
+   * Tombstone.
    */
-  private float edgeWeight;
+  private static final Tombstone TOMBSTONE = new Tombstone();
 
   /**
-   * vertexInId
+   * edgeId.
    */
-  private Utf8 vertexInId;
+  private java.lang.CharSequence edgeId;
 
   /**
-   * vertexOutId
+   * edgeWeight.
    */
-  private Utf8 vertexOutId;
+  private float edgeWeight;
 
   /**
-   * label
+   * vertexInId.
    */
-  private Utf8 label;
+  private java.lang.CharSequence vertexInId;
 
   /**
-   * Default constructor.
+   * vertexOutId.
    */
-  public GEdge() {
-    this(new StateManagerImpl());
-  }
+  private java.lang.CharSequence vertexOutId;
 
   /**
-   * Constructor
-   * @param stateManager from which the object will be created.
+   * label.
    */
-  public GEdge(StateManager stateManager) {
-    super(stateManager);
-  }
+  private java.lang.CharSequence label;
 
   /**
-   * Creates a new instance
-   * @param stateManager from which the object will be created.
-   * @return GEdge created
+   * Gets the total field count.
+   * @return int field count
    */
-  public GEdge newInstance(StateManager stateManager) {
-    return new GEdge(stateManager);
+  public int getFieldsCount() {
+    return GEdge.ALL_FIELDS.length;
   }
 
   /**
-   * Gets the object schema
-   * @return Schema of the object.
+   * Gets the schema
+   * @return Schema
    */
-  public Schema getSchema() {
-    return OBJ_SCHEMA;
+  public org.apache.avro.Schema getSchema() {
+    return SCHEMAS;
   }
 
   /**
    * Gets field
-   * @param fieldIndex index field.
+   * @param field index field.
    * @return Object from an index.
    */
-  public Object get(int fieldIndex) {
-    switch (fieldIndex) {
+  public java.lang.Object get(int field) {
+    switch (field) {
     case 0:
       return edgeId;
     case 1:
@@ -201,114 +180,725 @@ public class GEdge extends PersistentBase {
     case 4:
       return label;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
    * Puts a value into a field.
-   * @param fieldIndex index of field used.
-   * @param fieldValue value of field used.
+   * @param field index of field used.
+   * @param value value of field used.
    */
   @SuppressWarnings(value = "unchecked")
-  public void put(int fieldIndex, Object fieldValue) {
-    if (isFieldEqual(fieldIndex, fieldValue)) {
-      return;
-    }
-    getStateManager().setDirty(this, fieldIndex);
-    switch (fieldIndex) {
+  public void put(int field, java.lang.Object value) {
+    switch (field) {
     case 0:
-      edgeId = (Utf8) fieldValue; break;
+      edgeId = (java.lang.CharSequence) value;
+      break;
     case 1:
-      edgeWeight = (Float) fieldValue; break;
+      edgeWeight = (java.lang.Float) value;
+      break;
     case 2:
-      vertexInId = (Utf8) fieldValue; break;
+      vertexInId = (java.lang.CharSequence) value;
+      break;
     case 3:
-      vertexOutId = (Utf8) fieldValue; break;
+      vertexOutId = (java.lang.CharSequence) value;
+      break;
     case 4:
-      label = (Utf8) fieldValue; break;
+      label = (java.lang.CharSequence) value;
+      break;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
-   * Gets edgeId
-   * @return Utf8 edgeId
+   * Gets the value of the 'edgeId' field.
+   * @return CharSequence.
+   */
+  public java.lang.CharSequence getEdgeId() {
+    return edgeId;
+  }
+
+  /**
+   * Sets the value of the 'edgeId' field.
+   * @param value the value to set.
+   */
+  public void setEdgeId(java.lang.CharSequence value) {
+    this.edgeId = value;
+    setDirty(0);
+  }
+
+  /**
+   * Checks the dirty status of the 'edgeId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
+   */
+  public boolean isEdgeIdDirty(java.lang.CharSequence value) {
+    return isDirty(0);
+  }
+
+  /**
+   * Gets the value of the 'edgeWeight' field.
+   * @return Float
    */
-  public Utf8 getEdgeId() {
-    return (Utf8) get(0);
+  public java.lang.Float getEdgeWeight() {
+    return edgeWeight;
   }
 
   /**
-   * Sets edgeId
-   * @param value edgeId
+   * Sets the value of the 'edgeWeight' field.
+   * @param value the value to set.
    */
-  public void setEdgeId(Utf8 value) {
-    put(0, value);
+  public void setEdgeWeight(java.lang.Float value) {
+    this.edgeWeight = value;
+    setDirty(1);
   }
 
   /**
-   * Gets edgeWeight
-   * @return float edgeWeight
+   * Checks the dirty status of the 'edgeWeight' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public float getEdgeWeight() {
-    return (Float) get(1);
+  public boolean isEdgeWeightDirty(java.lang.Float value) {
+    return isDirty(1);
   }
 
   /**
-   * Sets edgeWeight
-   * @param value edgeWeight
+   * Gets the value of the 'vertexInId' field.
+   * @return CharSequence
    */
-  public void setEdgeWeight(float value) {
-    put(1, value);
+  public java.lang.CharSequence getVertexInId() {
+    return vertexInId;
   }
 
   /**
-   * Gets edgeVertexInId
-   * @return Utf8 edgeVertexInId
+   * Sets the value of the 'vertexInId' field.
+   * @param value the value to set.
    */
-  public Utf8 getVertexInId() {
-    return (Utf8) get(2);
+  public void setVertexInId(java.lang.CharSequence value) {
+    this.vertexInId = value;
+    setDirty(2);
   }
 
   /**
-   * Sets edgeVertexInId
-   * @param value edgeVertexInId
+   * Checks the dirty status of the 'vertexInId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public void setVertexInId(Utf8 value) {
-    put(2, value);
+  public boolean isVertexInIdDirty(java.lang.CharSequence value) {
+    return isDirty(2);
   }
 
   /**
-   * Gets edgeVertexOutId
-   * @return Utf8 edgeVertexOutId
+   * Gets the value of the 'vertexOutId' field.
+   * @return CharSequence
    */
-  public Utf8 getVertexOutId() {
-    return (Utf8) get(3);
+  public java.lang.CharSequence getVertexOutId() {
+    return vertexOutId;
   }
 
   /**
-   * Sets edgeVertexOutId
-   * @param value edgeVertexOutId
+   * Sets the value of the 'vertexOutId' field.
+   * @param value the value to set.
    */
-  public void setVertexOutId(Utf8 value) {
-    put(3, value);
+  public void setVertexOutId(java.lang.CharSequence value) {
+    this.vertexOutId = value;
+    setDirty(3);
   }
 
   /**
-   * Gets edgeLabel
-   * @return Utf8 edgeLabel
+   * Checks the dirty status of the 'vertexOutId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public Utf8 getLabel() {
-    return (Utf8) get(4);
+  public boolean isVertexOutIdDirty(java.lang.CharSequence value) {
+    return isDirty(3);
   }
 
   /**
-   * Sets edgeLabel
-   * @param value edgeLabel
+   * Gets the value of the 'label' field.
+   * @return CharSequence
    */
-  public void setLabel(Utf8 value) {
-    put(4, value);
+  public java.lang.CharSequence getLabel() {
+    return label;
+  }
+
+  /**
+   * Sets the value of the 'label' field.
+   * @param value the value to set.
+   */
+  public void setLabel(java.lang.CharSequence value) {
+    this.label = value;
+    setDirty(4);
+  }
+
+  /**
+   * Checks the dirty status of the 'label' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
+   */
+  public boolean isLabelDirty(java.lang.CharSequence value) {
+    return isDirty(4);
+  }
+
+  /**
+   * Creates a new GEdge RecordBuilder.
+   * @return GEdge.Builder
+   */
+  public static org.apache.giraph.io.gora.generated.GEdge.Builder newBuilder() {
+    return new org.apache.giraph.io.gora.generated.GEdge.Builder();
+  }
+
+  /**
+   * Creates a new GEdge RecordBuilder by copying an existing Builder
+   * @param other GEdge.Builder
+   * @return org.apache.giraph.io.gora.generated.GEdge.Builder
+   */
+  public static org.apache.giraph.io.gora.generated.GEdge.Builder newBuilder(
+      org.apache.giraph.io.gora.generated.GEdge.Builder other) {
+    return new org.apache.giraph.io.gora.generated.GEdge.Builder(other);
+  }
+
+  /**
+   * Creates a new GEdge RecordBuilder by copying an existing GEdge instance.
+   * @param other GEdge
+   * @return org.apache.giraph.io.gora.generated.GEdge.Builder
+   */
+  public static org.apache.giraph.io.gora.generated.GEdge.Builder newBuilder(
+      org.apache.giraph.io.gora.generated.GEdge other) {
+    return new org.apache.giraph.io.gora.generated.GEdge.Builder(other);
+  }
+
+  /**
+   * Makes a deep copy from a bytebuffer.
+   * @param input ByteBuffer
+   * @return ByteBuffer
+   */
+  private static java.nio.ByteBuffer deepCopyToReadOnlyBuffer(
+      java.nio.ByteBuffer input) {
+    java.nio.ByteBuffer copy = java.nio.ByteBuffer.allocate(input.capacity());
+    int position = input.position();
+    input.reset();
+    int mark = input.position();
+    int limit = input.limit();
+    input.rewind();
+    input.limit(input.capacity());
+    copy.put(input);
+    input.rewind();
+    copy.rewind();
+    input.position(mark);
+    input.mark();
+    copy.position(mark);
+    copy.mark();
+    input.position(position);
+    copy.position(position);
+    input.limit(limit);
+    copy.limit(limit);
+    return copy.asReadOnlyBuffer();
+  }
+
+  /**
+   * RecordBuilder for GEdge instances.
+   */
+  public static class Builder extends
+      org.apache.avro.specific.SpecificRecordBuilderBase<GEdge> implements
+      org.apache.avro.data.RecordBuilder<GEdge> {
+
+    /**
+     * edgeId.
+     */
+    private java.lang.CharSequence edgeId;
+
+    /**
+     * edgeWeight.
+     */
+    private float edgeWeight;
+
+    /**
+     * vertexInId
+     */
+    private java.lang.CharSequence vertexInId;
+
+    /**
+     * vertexOutId.
+     */
+    private java.lang.CharSequence vertexOutId;
+
+    /**
+     * label.
+     */
+    private java.lang.CharSequence label;
+
+    /**
+     * Creates a new Builder
+     */
+    private Builder() {
+      super(org.apache.giraph.io.gora.generated.GEdge.SCHEMAS);
+    }
+
+    /**
+     * Creates a Builder by copying an existing Builder.
+     * @param other GEdge.Builder
+     */
+    private Builder(org.apache.giraph.io.gora.generated.GEdge.Builder other) {
+      super(other);
+    }
+
+    /**
+     * Creates a Builder by copying an existing GEdge instance.
+     * @param other GEdge
+     */
+    // CHECKSTYLE: stop Indentation
+    private Builder(org.apache.giraph.io.gora.generated.GEdge other) {
+      super(org.apache.giraph.io.gora.generated.GEdge.SCHEMAS);
+      if (isValidValue(fields()[0], other.edgeId)) {
+        this.edgeId = (java.lang.CharSequence) data().deepCopy(
+          fields()[0].schema(), other.edgeId);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.edgeWeight)) {
+        this.edgeWeight = (java.lang.Float) data().deepCopy(
+          fields()[1].schema(), other.edgeWeight);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.vertexInId)) {
+        this.vertexInId = (java.lang.CharSequence) data().deepCopy(
+          fields()[2].schema(), other.vertexInId);
+        fieldSetFlags()[2] = true;
+      }
+      if (isValidValue(fields()[3], other.vertexOutId)) {
+        this.vertexOutId = (java.lang.CharSequence) data().deepCopy(
+          fields()[3].schema(), other.vertexOutId);
+        fieldSetFlags()[3] = true;
+      }
+      if (isValidValue(fields()[4], other.label)) {
+        this.label = (java.lang.CharSequence) data().deepCopy(
+          fields()[4].schema(), other.label);
+        fieldSetFlags()[4] = true;
+      }
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'edgeId' field
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getEdgeId() {
+      return edgeId;
+    }
+
+    /**
+     * Sets the value of the 'edgeId' field
+     * @param value CharSequence
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder setEdgeId(
+      java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.edgeId = value;
+      fieldSetFlags()[0] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'edgeId' field has been set.
+     * @return boolean.
+     */
+    public boolean hasEdgeId() {
+      return fieldSetFlags()[0];
+    }
+
+    /**
+     * Clears the value of the 'edgeId' field.
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder clearEdgeId() {
+      edgeId = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'edgeWeight' field.
+     * @return Float
+     */
+    public java.lang.Float getEdgeWeight() {
+      return edgeWeight;
+    }
+
+    /**
+     * Sets the value of the 'edgeWeight' field
+     * @param value float
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder setEdgeWeight(
+      float value) {
+      validate(fields()[1], value);
+      this.edgeWeight = value;
+      fieldSetFlags()[1] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'edgeWeight' field has been set.
+     * @return boolean
+     */
+    public boolean hasEdgeWeight() {
+      return fieldSetFlags()[1];
+    }
+
+    /**
+     * Clears the value of the 'edgeWeight' field.
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder clearEdgeWeight() {
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexInId' field
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexInId() {
+      return vertexInId;
+    }
+
+    /**
+     * Sets the value of the 'vertexInId' field.
+     * @param value CharSequence
+     * @return value
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder setVertexInId(
+      java.lang.CharSequence value) {
+      validate(fields()[2], value);
+      this.vertexInId = value;
+      fieldSetFlags()[2] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexInId' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexInId() {
+      return fieldSetFlags()[2];
+    }
+
+    /**
+     * Clears the value of the 'vertexInId' field.
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder
+    clearVertexInId() {
+      vertexInId = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexOutId' field.
+     * @return java.lang.CharSequence
+     */
+    public java.lang.CharSequence getVertexOutId() {
+      return vertexOutId;
+    }
+
+    /**
+     * Sets the value of the 'vertexOutId' field.
+     * @param value CharSequence
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder setVertexOutId(
+        java.lang.CharSequence value) {
+      validate(fields()[3], value);
+      this.vertexOutId = value;
+        fieldSetFlags()[3] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexOutId' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexOutId() {
+      return fieldSetFlags()[3];
+    }
+
+    /**
+     * Clears the value of the 'vertexOutId' field.
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder
+    clearVertexOutId() {
+      vertexOutId = null;
+      fieldSetFlags()[3] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'label' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getLabel() {
+      return label;
+    }
+
+    /**
+     * Sets the value of the 'label' field.
+     * @param value CharSequence
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder setLabel(
+        java.lang.CharSequence value) {
+      validate(fields()[4], value);
+      this.label = value;
+      fieldSetFlags()[4] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'label' field has been set.
+     * @return boolean
+     */
+    public boolean hasLabel() {
+      return fieldSetFlags()[4];
+    }
+
+    /**
+     * Clears the value of the 'label' field.
+     * @return GEdge.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdge.Builder clearLabel() {
+      label = null;
+      fieldSetFlags()[4] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    @Override
+    /**
+     * Builds a GEdge.
+     * @return GEdge
+     */
+    // CHECKSTYLE: stop IllegalCatch
+    public GEdge build() {
+      try {
+        GEdge record = new GEdge();
+        record.edgeId = fieldSetFlags()[0] ? this.edgeId :
+          (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.edgeWeight = fieldSetFlags()[1] ? this.edgeWeight :
+          (java.lang.Float) defaultValue(fields()[1]);
+        record.vertexInId = fieldSetFlags()[2] ? this.vertexInId :
+          (java.lang.CharSequence) defaultValue(fields()[2]);
+        record.vertexOutId = fieldSetFlags()[3] ? this.vertexOutId :
+          (java.lang.CharSequence) defaultValue(fields()[3]);
+        record.label = fieldSetFlags()[4] ? this.label :
+          (java.lang.CharSequence) defaultValue(fields()[4]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+    // CHECKSTYLE: resume IllegalCatch
+  }
+
+  /**
+   * Gets a tombstone
+   * @return GEdge.Tombstone
+   */
+  public GEdge.Tombstone getTombstone() {
+    return TOMBSTONE;
+  }
+
+  /**
+   * Gets a new instance
+   * @return GEdge.
+   */
+  public GEdge newInstance() {
+    return newBuilder().build();
+  }
+
+  /**
+   * Tombstone class.
+   */
+  public static final class Tombstone extends GEdge implements
+      org.apache.gora.persistency.Tombstone {
+
+    /**
+     * Default constructor.
+     */
+    private Tombstone() {
+    }
+
+    /**
+     * Gets the value of the 'edgeId' field.
+     * @return java.lang.CharSequence
+     */
+    public java.lang.CharSequence getEdgeId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'edgeId' field.
+     * @param value the value to set.
+     */
+    public void setEdgeId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'edgeId' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isEdgeIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'edgeWeight' field.
+     * @return Float
+     */
+    public java.lang.Float getEdgeWeight() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'edgeWeight' field.
+     * @param value the value to set.
+     */
+    public void setEdgeWeight(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'edgeWeight' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isEdgeWeightDirty(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'vertexInId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexInId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexInId' field.
+     * @param value the value to set.
+     */
+    public void setVertexInId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexInId' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexInIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'vertexOutId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexOutId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexOutId' field.
+     * @param value the value to set.
+     */
+    public void setVertexOutId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexOutId' field. A field is dirty if
+     * it represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexOutIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'label' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getLabel() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'label' field.
+     * @param value the value to set.
+     */
+    public void setLabel(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'label' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isLabelDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/7c61dcf4/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdgeResult.java
----------------------------------------------------------------------
diff --git a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdgeResult.java b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdgeResult.java
index 0c3501c..c54e40e 100644
--- a/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdgeResult.java
+++ b/giraph-gora/src/main/java/org/apache/giraph/io/gora/generated/GEdgeResult.java
@@ -15,35 +15,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.giraph.io.gora.generated;
 
-import org.apache.avro.Schema;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.util.Utf8;
-import org.apache.gora.persistency.StateManager;
-import org.apache.gora.persistency.impl.PersistentBase;
-import org.apache.gora.persistency.impl.StateManagerImpl;
-
 /**
- * Example Class for defining a Giraph-Edge.
+ * Example class for defining a Giraph-Edge result.
  */
 @SuppressWarnings("all")
-public class GEdgeResult extends PersistentBase {
-  /**
-   * Schema used for the class.
-   */
-  public static final Schema OBJ_SCHEMA = Schema.parse("{\"type\":\"record\"," +
-    "\"name\":\"GEdge\",\"namespace\":\"org.apache.giraph.gora.generated\"," +
-    "\"fields\":[{\"name\":\"edgeId\",\"type\":\"string\"}," +
-    "{\"name\":\"edgeWeight\",\"type\":\"float\"}," +
-    "{\"name\":\"vertexInId\",\"type\":\"string\"}," +
-    "{\"name\":\"vertexOutId\",\"type\":\"string\"}," +
-    "{\"name\":\"label\",\"type\":\"string\"}]}");
+public class GEdgeResult extends
+    org.apache.gora.persistency.impl.PersistentBase implements
+    org.apache.avro.specific.SpecificRecord,
+    org.apache.gora.persistency.Persistent {
 
   /**
-   * Field enum
+   * Schema used for the class.
    */
+  public static final org.apache.avro.Schema SCHEMAS =
+      new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\"," +
+            "\"name\":\"GEdgeResult\"," +
+            "\"namespace\":\"org.apache.giraph.io.gora.generated\"," +
+            "\"fields\":[{\"name\":\"edgeId\",\"type\":\"string\"}," +
+            "{\"name\":\"edgeWeight\",\"type\":\"float\"}," +
+            "{\"name\":\"vertexInId\",\"type\":\"string\"}," +
+            "{\"name\":\"vertexOutId\",\"type\":\"string\"}," +
+            "{\"name\":\"label\",\"type\":\"string\"}]}");
+
+  /** Enum containing all data bean's fields. */
   public static enum Field {
     /**
      * Edge id.
@@ -71,19 +67,19 @@ public class GEdgeResult extends PersistentBase {
     LABEL(4, "label");
 
     /**
-     * Field index
+     * Field's index.
      */
     private int index;
 
     /**
-     * Field name
+     * Field's name.
      */
     private String name;
 
     /**
-     * Field constructor
-     * @param index of attribute
-     * @param name of attribute
+     * Field's constructor
+     * @param index field's index.
+     * @param name field's name.
      */
     Field(int index, String name) {
       this.index = index;
@@ -91,24 +87,24 @@ public class GEdgeResult extends PersistentBase {
     }
 
     /**
-     * Gets index
-     * @return int of attribute.
+     * Gets field's index.
+     * @return int field's index.
      */
     public int getIndex() {
       return index;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's name.
+     * @return String field's name.
      */
     public String getName() {
       return name;
     }
 
     /**
-     * Gets name
-     * @return String of name.
+     * Gets field's attributes to string.
+     * @return String field's attributes to string.
      */
     public String toString() {
       return name;
@@ -119,77 +115,61 @@ public class GEdgeResult extends PersistentBase {
    * Array containing all fields/
    */
   private static final String[] ALL_FIELDS = {
-    "edgeId", "edgeWeight", "vertexInId", "vertexOutId", "label"
-  };
-
-  static {
-    PersistentBase.registerFields(GEdgeResult.class, ALL_FIELDS);
-  }
-
-  /**
-   * edgeId
-   */
-  private Utf8 edgeId;
+    "edgeId", "edgeWeight", "vertexInId", "vertexOutId", "label"};
 
   /**
-   * edgeWeight
+   * Tombstone.
    */
-  private float edgeWeight;
+  private static final Tombstone TOMBSTONE = new Tombstone();
 
   /**
-   * vertexInId
+   * edgeId.
    */
-  private Utf8 vertexInId;
+  private java.lang.CharSequence edgeId;
 
   /**
-   * vertexOutId
+   * edgeWeight.
    */
-  private Utf8 vertexOutId;
+  private float edgeWeight;
 
   /**
-   * label
+   * vertexInId.
    */
-  private Utf8 label;
+  private java.lang.CharSequence vertexInId;
 
   /**
-   * Default constructor.
+   * vertexOutId.
    */
-  public GEdgeResult() {
-    this(new StateManagerImpl());
-  }
+  private java.lang.CharSequence vertexOutId;
 
   /**
-   * Constructor
-   * @param stateManager from which the object will be created.
+   * label.
    */
-  public GEdgeResult(StateManager stateManager) {
-    super(stateManager);
-  }
+  private java.lang.CharSequence label;
 
   /**
-   * Creates a new instance
-   * @param stateManager from which the object will be created.
-   * @return GEdge created
+   * Gets the total field count.
+   * @return int field count
    */
-  public GEdgeResult newInstance(StateManager stateManager) {
-    return new GEdgeResult(stateManager);
+  public int getFieldsCount() {
+    return GEdgeResult.ALL_FIELDS.length;
   }
 
   /**
-   * Gets the object schema
-   * @return Schema of the object.
+   * Gets the schema
+   * @return Schema
    */
-  public Schema getSchema() {
-    return OBJ_SCHEMA;
+  public org.apache.avro.Schema getSchema() {
+    return SCHEMAS;
   }
 
   /**
    * Gets field
-   * @param fieldIndex index field.
+   * @param field index field.
    * @return Object from an index.
    */
-  public Object get(int fieldIndex) {
-    switch (fieldIndex) {
+  public java.lang.Object get(int field) {
+    switch (field) {
     case 0:
       return edgeId;
     case 1:
@@ -201,114 +181,729 @@ public class GEdgeResult extends PersistentBase {
     case 4:
       return label;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
    * Puts a value into a field.
-   * @param fieldIndex index of field used.
-   * @param fieldValue value of field used.
+   * @param field index of field used.
+   * @param value value of field used.
    */
   @SuppressWarnings(value = "unchecked")
-  public void put(int fieldIndex, Object fieldValue) {
-    if (isFieldEqual(fieldIndex, fieldValue)) {
-      return;
-    }
-    getStateManager().setDirty(this, fieldIndex);
-    switch (fieldIndex) {
+  public void put(int field, java.lang.Object value) {
+    switch (field) {
     case 0:
-      edgeId = (Utf8) fieldValue; break;
+      edgeId = (java.lang.CharSequence) value;
+      break;
     case 1:
-      edgeWeight = (Float) fieldValue; break;
+      edgeWeight = (java.lang.Float) value;
+      break;
     case 2:
-      vertexInId = (Utf8) fieldValue; break;
+      vertexInId = (java.lang.CharSequence) value;
+      break;
     case 3:
-      vertexOutId = (Utf8) fieldValue; break;
+      vertexOutId = (java.lang.CharSequence) value;
+      break;
     case 4:
-      label = (Utf8) fieldValue; break;
+      label = (java.lang.CharSequence) value;
+      break;
     default:
-      throw new AvroRuntimeException("Bad index");
+      throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
 
   /**
-   * Gets edgeId
-   * @return Utf8 edgeId
+   * Gets the value of the 'edgeId' field.
+   * @return CharSequence.
+   */
+  public java.lang.CharSequence getEdgeId() {
+    return edgeId;
+  }
+
+  /**
+   * Sets the value of the 'edgeId' field.
+   * @param value the value to set.
+   */
+  public void setEdgeId(java.lang.CharSequence value) {
+    this.edgeId = value;
+    setDirty(0);
+  }
+
+  /**
+   * Checks the dirty status of the 'edgeId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
+   */
+  public boolean isEdgeIdDirty(java.lang.CharSequence value) {
+    return isDirty(0);
+  }
+
+  /**
+   * Gets the value of the 'edgeWeight' field.
+   * @return Float
+   */
+  public java.lang.Float getEdgeWeight() {
+    return edgeWeight;
+  }
+
+  /**
+   * Sets the value of the 'edgeWeight' field.
+   * @param value the value to set.
+   */
+  public void setEdgeWeight(java.lang.Float value) {
+    this.edgeWeight = value;
+    setDirty(1);
+  }
+
+  /**
+   * Checks the dirty status of the 'edgeWeight' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
+   */
+  public boolean isEdgeWeightDirty(java.lang.Float value) {
+    return isDirty(1);
+  }
+
+  /**
+   * Gets the value of the 'vertexInId' field.
+   * @return CharSequence
+   */
+  public java.lang.CharSequence getVertexInId() {
+    return vertexInId;
+  }
+
+  /**
+   * Sets the value of the 'vertexInId' field.
+   * @param value the value to set.
    */
-  public Utf8 getEdgeId() {
-    return (Utf8) get(0);
+  public void setVertexInId(java.lang.CharSequence value) {
+    this.vertexInId = value;
+    setDirty(2);
   }
 
   /**
-   * Sets edgeId
-   * @param value edgeId
+   * Checks the dirty status of the 'vertexInId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public void setEdgeId(Utf8 value) {
-    put(0, value);
+  public boolean isVertexInIdDirty(java.lang.CharSequence value) {
+    return isDirty(2);
   }
 
   /**
-   * Gets edgeWeight
-   * @return float edgeWeight
+   * Gets the value of the 'vertexOutId' field.
+   * @return CharSequence
    */
-  public float getEdgeWeight() {
-    return (Float) get(1);
+  public java.lang.CharSequence getVertexOutId() {
+    return vertexOutId;
   }
 
   /**
-   * Sets edgeWeight
-   * @param value edgeWeight
+   * Sets the value of the 'vertexOutId' field.
+   * @param value the value to set.
    */
-  public void setEdgeWeight(float value) {
-    put(1, value);
+  public void setVertexOutId(java.lang.CharSequence value) {
+    this.vertexOutId = value;
+    setDirty(3);
   }
 
   /**
-   * Gets edgeVertexInId
-   * @return Utf8 edgeVertexInId
+   * Checks the dirty status of the 'vertexOutId' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public Utf8 getVertexInId() {
-    return (Utf8) get(2);
+  public boolean isVertexOutIdDirty(java.lang.CharSequence value) {
+    return isDirty(3);
   }
 
   /**
-   * Sets edgeVertexInId
-   * @param value edgeVertexInId
+   * Gets the value of the 'label' field.
+   * @return CharSequence
    */
-  public void setVertexInId(Utf8 value) {
-    put(2, value);
+  public java.lang.CharSequence getLabel() {
+    return label;
   }
 
   /**
-   * Gets edgeVertexOutId
-   * @return Utf8 edgeVertexOutId
+   * Sets the value of the 'label' field.
+   * @param value the value to set.
    */
-  public Utf8 getVertexOutId() {
-    return (Utf8) get(3);
+  public void setLabel(java.lang.CharSequence value) {
+    this.label = value;
+    setDirty(4);
   }
 
   /**
-   * Sets edgeVertexOutId
-   * @param value edgeVertexOutId
+   * Checks the dirty status of the 'label' field. A field is dirty if it
+   * represents a change that has not yet been written to the database.
+   * @param value the value to set.
+   * @return boolean
    */
-  public void setVertexOutId(Utf8 value) {
-    put(3, value);
+  public boolean isLabelDirty(java.lang.CharSequence value) {
+    return isDirty(4);
   }
 
   /**
-   * Gets edgeLabel
-   * @return Utf8 edgeLabel
+   * Creates a new GEdgeResult RecordBuilder.
+   * @return GEdge.Builder
    */
-  public Utf8 getLabel() {
-    return (Utf8) get(4);
+  public static org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+  newBuilder() {
+    return new org.apache.giraph.io.gora.generated.GEdgeResult.Builder();
   }
 
   /**
-   * Sets edgeLabel
-   * @param value edgeLabel
+   * Creates a new GEdgeResult RecordBuilder by copying an existing Builder.
+   * @param other GEdgeResult.Builder
+   * @return GEdge.Builder
    */
-  public void setLabel(Utf8 value) {
-    put(4, value);
+  public static org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+  newBuilder(org.apache.giraph.io.gora.generated.GEdgeResult.Builder other) {
+    return new org.apache.giraph.io.gora.generated.GEdgeResult.Builder(other);
+  }
+
+  /**
+   * Creates a new GEdgeResult RecordBuilder by copying an existing GEdgeResult
+   * instance
+   * @param other GEdgeResult
+   * @return GEdge.Builder
+   */
+  public static org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+  newBuilder(org.apache.giraph.io.gora.generated.GEdgeResult other) {
+    return new org.apache.giraph.io.gora.generated.GEdgeResult.Builder(other);
+  }
+
+  /**
+   * Makes a deep copy from a bytebuffer.
+   * @param input ByteBuffer
+   * @return ByteBuffer
+   */
+  private static java.nio.ByteBuffer deepCopyToReadOnlyBuffer(
+      java.nio.ByteBuffer input) {
+    java.nio.ByteBuffer copy = java.nio.ByteBuffer.allocate(input.capacity());
+    int position = input.position();
+    input.reset();
+    int mark = input.position();
+    int limit = input.limit();
+    input.rewind();
+    input.limit(input.capacity());
+    copy.put(input);
+    input.rewind();
+    copy.rewind();
+    input.position(mark);
+    input.mark();
+    copy.position(mark);
+    copy.mark();
+    input.position(position);
+    copy.position(position);
+    input.limit(limit);
+    copy.limit(limit);
+    return copy.asReadOnlyBuffer();
+  }
+
+  /**
+   * RecordBuilder for GEdgeResult instances.
+   */
+  public static class Builder extends
+      org.apache.avro.specific.SpecificRecordBuilderBase<GEdgeResult> implements
+      org.apache.avro.data.RecordBuilder<GEdgeResult> {
+
+    /**
+     * edgeId.
+     */
+    private java.lang.CharSequence edgeId;
+
+    /**
+     * edgeWeight.
+     */
+    private float edgeWeight;
+
+    /**
+     * vertexInId
+     */
+    private java.lang.CharSequence vertexInId;
+
+    /**
+     * vertexOutId.
+     */
+    private java.lang.CharSequence vertexOutId;
+
+    /**
+     * label.
+     */
+    private java.lang.CharSequence label;
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(org.apache.giraph.io.gora.generated.GEdgeResult.SCHEMAS);
+    }
+
+    /**
+     * Creates a Builder by copying an existing Builder.
+     * @param other GEdgeResult.Builder
+     */
+    private Builder(
+        org.apache.giraph.io.gora.generated.GEdgeResult.Builder other) {
+      super(other);
+    }
+
+    /**
+     * Creates a Builder by copying an existing GEdgeResult instance.
+     * @param other GEdgeResult
+     */
+    // CHECKSTYLE: stop Indentation
+    private Builder(org.apache.giraph.io.gora.generated.GEdgeResult other) {
+      super(org.apache.giraph.io.gora.generated.GEdgeResult.SCHEMAS);
+      if (isValidValue(fields()[0], other.edgeId)) {
+        this.edgeId = (java.lang.CharSequence) data().deepCopy(
+          fields()[0].schema(), other.edgeId);
+        fieldSetFlags()[0] = true;
+      }
+      if (isValidValue(fields()[1], other.edgeWeight)) {
+        this.edgeWeight = (java.lang.Float) data().deepCopy(
+          fields()[1].schema(), other.edgeWeight);
+        fieldSetFlags()[1] = true;
+      }
+      if (isValidValue(fields()[2], other.vertexInId)) {
+        this.vertexInId = (java.lang.CharSequence) data().deepCopy(
+          fields()[2].schema(), other.vertexInId);
+        fieldSetFlags()[2] = true;
+      }
+      if (isValidValue(fields()[3], other.vertexOutId)) {
+        this.vertexOutId = (java.lang.CharSequence) data().deepCopy(
+          fields()[3].schema(), other.vertexOutId);
+        fieldSetFlags()[3] = true;
+      }
+      if (isValidValue(fields()[4], other.label)) {
+        this.label = (java.lang.CharSequence) data().deepCopy(
+          fields()[4].schema(), other.label);
+        fieldSetFlags()[4] = true;
+      }
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'edgeId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getEdgeId() {
+      return edgeId;
+    }
+
+    /**
+     * Sets the value of the 'edgeId' field.
+     * @param value CharSequence
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder setEdgeId(
+        java.lang.CharSequence value) {
+      validate(fields()[0], value);
+      this.edgeId = value;
+      fieldSetFlags()[0] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'edgeId' field has been set.
+     * @return boolean
+     */
+    public boolean hasEdgeId() {
+      return fieldSetFlags()[0];
+    }
+
+    /**
+     * Clears the value of the 'edgeId' field.
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    clearEdgeId() {
+      edgeId = null;
+      fieldSetFlags()[0] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'edgeWeight' field.
+     * @return Float
+     */
+    public java.lang.Float getEdgeWeight() {
+      return edgeWeight;
+    }
+
+    /**
+     * Sets the value of the 'edgeWeight' field.
+     * @param value float
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    setEdgeWeight(float value) {
+      validate(fields()[1], value);
+      this.edgeWeight = value;
+      fieldSetFlags()[1] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'edgeWeight' field has been set.
+     * @return boolean
+     */
+    public boolean hasEdgeWeight() {
+      return fieldSetFlags()[1];
+    }
+
+    /**
+     * Clears the value of the 'edgeWeight' field.
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    clearEdgeWeight() {
+      fieldSetFlags()[1] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexInId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexInId() {
+      return vertexInId;
+    }
+
+    /**
+     * Sets the value of the 'vertexInId' field.
+     * @param value CharSequence
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    setVertexInId(java.lang.CharSequence value) {
+      validate(fields()[2], value);
+      this.vertexInId = value;
+      fieldSetFlags()[2] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexInId' field has been set.
+     * @return boolean.
+     */
+    public boolean hasVertexInId() {
+      return fieldSetFlags()[2];
+    }
+
+    /**
+     * Clears the value of the 'vertexInId' field.
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    clearVertexInId() {
+      vertexInId = null;
+      fieldSetFlags()[2] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'vertexOutId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexOutId() {
+      return vertexOutId;
+    }
+
+    /**
+     * Sets the value of the 'vertexOutId' field.
+     * @param value CharSequence
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    setVertexOutId(java.lang.CharSequence value) {
+      validate(fields()[3], value);
+      this.vertexOutId = value;
+      fieldSetFlags()[3] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'vertexOutId' field has been set.
+     * @return boolean
+     */
+    public boolean hasVertexOutId() {
+      return fieldSetFlags()[3];
+    }
+
+    /**
+     * Clears the value of the 'vertexOutId' field.
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    clearVertexOutId() {
+      vertexOutId = null;
+      fieldSetFlags()[3] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Gets the value of the 'label' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getLabel() {
+      return label;
+    }
+
+    /**
+     * Sets the value of the 'label' field.
+     * @param value CharSequence
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder setLabel(
+        java.lang.CharSequence value) {
+      validate(fields()[4], value);
+      this.label = value;
+      fieldSetFlags()[4] = true;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    /**
+     * Checks whether the 'label' field has been set.
+     * @return boolean
+     */
+    public boolean hasLabel() {
+      return fieldSetFlags()[4];
+    }
+
+    /**
+     * Clears the value of the 'label' field.
+     * @return GEdgeResult.Builder
+     */
+    // CHECKSTYLE: stop Indentation
+    public org.apache.giraph.io.gora.generated.GEdgeResult.Builder
+    clearLabel() {
+      label = null;
+      fieldSetFlags()[4] = false;
+      return this;
+    }
+    // CHECKSTYLE: resume Indentation
+
+    @Override
+    /**
+     * Builds a GEdgeResult.
+     * @return GEdgeResult
+     */
+    // CHECKSTYLE: stop IllegalCatch
+    public GEdgeResult build() {
+      try {
+        GEdgeResult record = new GEdgeResult();
+        record.edgeId = fieldSetFlags()[0] ? this.edgeId :
+          (java.lang.CharSequence) defaultValue(fields()[0]);
+        record.edgeWeight = fieldSetFlags()[1] ? this.edgeWeight :
+          (java.lang.Float) defaultValue(fields()[1]);
+        record.vertexInId = fieldSetFlags()[2] ? this.vertexInId :
+          (java.lang.CharSequence) defaultValue(fields()[2]);
+        record.vertexOutId = fieldSetFlags()[3] ? this.vertexOutId :
+          (java.lang.CharSequence) defaultValue(fields()[3]);
+        record.label = fieldSetFlags()[4] ? this.label :
+          (java.lang.CharSequence) defaultValue(fields()[4]);
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+    // CHECKSTYLE: resume IllegalCatch
+  }
+
+  /**
+   * Gets tombstone
+   * @return GEdgeResult.Tombstone
+   */
+  public GEdgeResult.Tombstone getTombstone() {
+    return TOMBSTONE;
+  }
+
+  /**
+   * Gets a new instance
+   * @return GEdgeResult.
+   */
+  public GEdgeResult newInstance() {
+    return newBuilder().build();
+  }
+
+  /**
+   * Tombstone class.
+   */
+  public static final class Tombstone extends GEdgeResult implements
+      org.apache.gora.persistency.Tombstone {
+
+    /**
+     * Default constructor.
+     */
+    private Tombstone() {
+    }
+
+    /**
+     * Gets the value of the 'edgeId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getEdgeId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'edgeId' field.
+     * @param value the value to set.
+     */
+    public void setEdgeId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'edgeId' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isEdgeIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'edgeWeight' field.
+     * @return Float
+     */
+    public java.lang.Float getEdgeWeight() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'edgeWeight' field.
+     * @param value the value to set.
+     */
+    public void setEdgeWeight(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'edgeWeight' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isEdgeWeightDirty(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'vertexInId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexInId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexInId' field.
+    * @param value the value to set.
+     */
+    public void setVertexInId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexInId' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexInIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'vertexOutId' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getVertexOutId() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'vertexOutId' field.
+     * @param value the value to set.
+     */
+    public void setVertexOutId(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'vertexOutId' field. A field is dirty if
+     * it represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isVertexOutIdDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    /**
+     * Gets the value of the 'label' field.
+     * @return CharSequence
+     */
+    public java.lang.CharSequence getLabel() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    /**
+     * Sets the value of the 'label' field.
+     * @param value the value to set.
+     */
+    public void setLabel(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    /**
+     * Checks the dirty status of the 'label' field. A field is dirty if it
+     * represents a change that has not yet been written to the database.
+     * @param value the value to set.
+     * @return boolean
+     */
+    public boolean isLabelDirty(java.lang.CharSequence value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
   }
 }


[45/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
Auto-restart from checkpoint doesn't pick up latest checkpoint

Summary:
While running different jobs with checkpoints enabled I noticed some issues:
1) The way we pick up latest checkpoint is not correct. Current implementation just picks whatever is returned last from FileSystem.list(), which is not necessarily the last checkpoint
2) If job restarts from checkpoint it immediately creates another checkpoint.
3) We need more flexibility in GiraphJobRetryChecker to allow restarts after multiple failures.

Test Plan: Run our production jobs with checkpointing

Reviewers: majakabiljo, pavanka, pavanka.26, maja.kabiljo

Reviewed By: maja.kabiljo

Differential Revision: https://reviews.facebook.net/D23913


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/54a1a8de
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/54a1a8de
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/54a1a8de

Branch: refs/heads/release-1.1
Commit: 54a1a8dedbd4af64ca8079da69878f89a69d9e5b
Parents: d32c429
Author: Sergey Edunov <ed...@fb.com>
Authored: Mon Oct 20 10:08:56 2014 -0700
Committer: Maja Kabiljo <ma...@fb.com>
Committed: Mon Oct 20 10:15:57 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/giraph/bsp/BspService.java  |  72 +---------
 .../job/DefaultGiraphJobRetryChecker.java       |   5 +-
 .../java/org/apache/giraph/job/GiraphJob.java   |  24 +---
 .../giraph/job/GiraphJobRetryChecker.java       |   9 +-
 .../apache/giraph/master/BspServiceMaster.java  |  10 +-
 .../apache/giraph/utils/CheckpointingUtils.java |  94 +++++++++++++
 .../org/apache/giraph/utils/WritableUtils.java  | 139 +++++++++++++++----
 .../apache/giraph/worker/BspServiceWorker.java  |  32 +++--
 .../apache/giraph/utils/TestWritableUtils.java  |  40 +++++-
 9 files changed, 283 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
----------------------------------------------------------------------
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 2a50489..579c772 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
@@ -31,10 +31,7 @@ import org.apache.giraph.zk.BspEvent;
 import org.apache.giraph.zk.PredicateLock;
 import org.apache.giraph.zk.ZooKeeperExt;
 import org.apache.giraph.zk.ZooKeeperManager;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -52,9 +49,7 @@ import org.json.JSONObject;
 import java.io.IOException;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
-import java.security.InvalidParameterException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -190,28 +185,7 @@ public abstract class BspService<I extends WritableComparable,
   public static final String WORKER_SUFFIX = "_worker";
   /** Suffix denotes a master */
   public static final String MASTER_SUFFIX = "_master";
-  /** If at the end of a checkpoint file, indicates metadata */
-  public static final String CHECKPOINT_METADATA_POSTFIX = ".metadata";
-  /**
-   * If at the end of a checkpoint file, indicates vertices, edges,
-   * messages, etc.
-   */
-  public static final String CHECKPOINT_VERTICES_POSTFIX = ".vertices";
-  /**
-   * If at the end of a checkpoint file, indicates metadata and data is valid
-   * for the same filenames without .valid
-   */
-  public static final String CHECKPOINT_VALID_POSTFIX = ".valid";
-  /**
-   * If at the end of a checkpoint file,
-   * indicates that we store WorkerContext and aggregator handler data.
-   */
-  public static final String CHECKPOINT_DATA_POSTFIX = ".data";
-  /**
-   * If at the end of a checkpoint file, indicates the stitched checkpoint
-   * file prefixes.  A checkpoint is not valid if this file does not exist.
-   */
-  public static final String CHECKPOINT_FINALIZED_POSTFIX = ".finalized";
+
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(BspService.class);
   /** Path to the job's root */
@@ -606,22 +580,6 @@ public abstract class BspService<I extends WritableComparable,
     return savedCheckpointBasePath + "/" + superstep;
   }
 
-  /**
-   * Get the checkpoint from a finalized checkpoint path
-   *
-   * @param finalizedPath Path of the finalized checkpoint
-   * @return Superstep referring to a checkpoint of the finalized path
-   */
-  public static long getCheckpoint(Path finalizedPath) {
-    if (!finalizedPath.getName().endsWith(CHECKPOINT_FINALIZED_POSTFIX)) {
-      throw new InvalidParameterException(
-          "getCheckpoint: " + finalizedPath + "Doesn't end in " +
-              CHECKPOINT_FINALIZED_POSTFIX);
-    }
-    String checkpointString =
-        finalizedPath.getName().replace(CHECKPOINT_FINALIZED_POSTFIX, "");
-    return Long.parseLong(checkpointString);
-  }
 
   /**
    * Get the ZooKeeperExt instance.
@@ -1235,22 +1193,8 @@ public abstract class BspService<I extends WritableComparable,
    * @throws IOException
    */
   protected long getLastCheckpointedSuperstep() throws IOException {
-    FileStatus[] fileStatusArray =
-        getFs().listStatus(new Path(savedCheckpointBasePath),
-            new FinalizedCheckpointPathFilter());
-    if (fileStatusArray == null) {
-      return -1;
-    }
-    Arrays.sort(fileStatusArray);
-    long lastCheckpointedSuperstep = getCheckpoint(
-        fileStatusArray[fileStatusArray.length - 1].getPath());
-    if (LOG.isInfoEnabled()) {
-      LOG.info("getLastGoodCheckpoint: Found last good checkpoint " +
-          lastCheckpointedSuperstep + " from " +
-          fileStatusArray[fileStatusArray.length - 1].
-              getPath().toString());
-    }
-    return lastCheckpointedSuperstep;
+    return CheckpointingUtils.getLastCheckpointedSuperstep(getFs(),
+        savedCheckpointBasePath);
   }
 
   @Override
@@ -1258,15 +1202,5 @@ public abstract class BspService<I extends WritableComparable,
     return getGraphTaskManager().getJobProgressTracker();
   }
 
-  /**
-   * Only get the finalized checkpoint files
-   */
-  private static class FinalizedCheckpointPathFilter implements PathFilter {
-    @Override
-    public boolean accept(Path path) {
-      return path.getName().endsWith(BspService.CHECKPOINT_FINALIZED_POSTFIX);
-    }
-
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java b/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
index edf6bce..405bde3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java
@@ -32,7 +32,8 @@ public class DefaultGiraphJobRetryChecker implements GiraphJobRetryChecker {
   }
 
   @Override
-  public boolean shouldRestartCheckpoint() {
-    return false;
+  public String shouldRestartCheckpoint(Job lastAttempt) {
+    return null;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
index ca1ad1c..11ae7fc 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java
@@ -24,13 +24,9 @@ import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GraphMapper;
-import org.apache.giraph.utils.CheckpointingUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobID;
 import org.apache.log4j.Logger;
 
 import java.io.IOException;
@@ -276,22 +272,12 @@ public class GiraphJob {
       }
       jobObserver.jobFinished(submittedJob, passed);
 
-      FileSystem fs = FileSystem.get(conf);
-      JobID jobID = HadoopUtils.getJobID(submittedJob);
-      if (jobID != null) {
-        Path checkpointMark =
-            CheckpointingUtils.getCheckpointMarkPath(conf, jobID.toString());
-
-        if (fs.exists(checkpointMark)) {
-          if (retryChecker.shouldRestartCheckpoint()) {
-            GiraphConstants.RESTART_JOB_ID.set(conf, jobID.toString());
-            continue;
-          }
+      if (!passed) {
+        String restartFrom = retryChecker.shouldRestartCheckpoint(submittedJob);
+        if (restartFrom != null) {
+          GiraphConstants.RESTART_JOB_ID.set(conf, restartFrom);
+          continue;
         }
-      } else {
-        LOG.warn("jobID is null, are you using hadoop 0.20.203? " +
-            "Please report this issue here " +
-            "https://issues.apache.org/jira/browse/GIRAPH-933");
       }
 
       if (passed || !retryChecker.shouldRetry(submittedJob, tryCount)) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
index 556b128..b213c51 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java
@@ -20,6 +20,9 @@ package org.apache.giraph.job;
 
 import org.apache.hadoop.mapreduce.Job;
 
+import java.io.IOException;
+
+
 /**
  * Class to decide whether a GiraphJob should be restarted after failure.
  */
@@ -36,7 +39,9 @@ public interface GiraphJobRetryChecker {
 
   /**
    * The job has been checkpointed and halted. Should we now restart it?
-   * @return true if checkpointed job should be automatically restarted.
+   * @param lastAttempt latest failed job
+   * @return JobID of job to be restarted or null if
+   * we don't want to restart anything
    */
-  boolean shouldRestartCheckpoint();
+  String shouldRestartCheckpoint(Job lastAttempt) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index af7e5fd..62b089c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -772,8 +772,8 @@ public class BspServiceMaster<I extends WritableComparable,
     throws IOException, KeeperException, InterruptedException {
     List<PartitionOwner> partitionOwners = new ArrayList<>();
     FileSystem fs = getFs();
-    String finalizedCheckpointPath =
-        getSavedCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
+    String finalizedCheckpointPath = getSavedCheckpointBasePath(superstep) +
+        CheckpointingUtils.CHECKPOINT_FINALIZED_POSTFIX;
     LOG.info("Loading checkpoint from " + finalizedCheckpointPath);
     DataInputStream finalizedStream =
         fs.open(new Path(finalizedCheckpointPath));
@@ -796,7 +796,7 @@ public class BspServiceMaster<I extends WritableComparable,
       int mrTaskId = finalizedStream.readInt();
 
       DataInputStream metadataStream = fs.open(new Path(checkpointFile +
-          "." + mrTaskId + CHECKPOINT_METADATA_POSTFIX));
+          "." + mrTaskId + CheckpointingUtils.CHECKPOINT_METADATA_POSTFIX));
       long partitions = metadataStream.readInt();
       WorkerInfo worker = workersMap.get(mrTaskId);
       for (long p = 0; p < partitions; ++p) {
@@ -1081,7 +1081,7 @@ public class BspServiceMaster<I extends WritableComparable,
     throws IOException, KeeperException, InterruptedException {
     Path finalizedCheckpointPath =
         new Path(getCheckpointBasePath(superstep) +
-            CHECKPOINT_FINALIZED_POSTFIX);
+            CheckpointingUtils.CHECKPOINT_FINALIZED_POSTFIX);
     try {
       getFs().delete(finalizedCheckpointPath, false);
     } catch (IOException e) {
@@ -1761,7 +1761,7 @@ public class BspServiceMaster<I extends WritableComparable,
     if (checkpointFrequency == 0) {
       return CheckpointStatus.NONE;
     }
-    long firstCheckpoint = INPUT_SUPERSTEP + 1;
+    long firstCheckpoint = INPUT_SUPERSTEP + 1 + checkpointFrequency;
     if (getRestartedSuperstep() != UNSET_SUPERSTEP) {
       firstCheckpoint = getRestartedSuperstep() + checkpointFrequency;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
index 11d5e4f..f431260 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/CheckpointingUtils.java
@@ -19,7 +19,14 @@
 package org.apache.giraph.utils;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.security.InvalidParameterException;
 
 import static org.apache.giraph.conf.GiraphConstants.CHECKPOINT_DIRECTORY;
 
@@ -29,6 +36,32 @@ import static org.apache.giraph.conf.GiraphConstants.CHECKPOINT_DIRECTORY;
  */
 public class CheckpointingUtils {
 
+  /** If at the end of a checkpoint file, indicates metadata */
+  public static final String CHECKPOINT_METADATA_POSTFIX = ".metadata";
+  /**
+   * If at the end of a checkpoint file, indicates vertices, edges,
+   * messages, etc.
+   */
+  public static final String CHECKPOINT_VERTICES_POSTFIX = ".vertices";
+  /**
+   * If at the end of a checkpoint file, indicates metadata and data is valid
+   * for the same filenames without .valid
+   */
+  public static final String CHECKPOINT_VALID_POSTFIX = ".valid";
+  /**
+   * If at the end of a checkpoint file,
+   * indicates that we store WorkerContext and aggregator handler data.
+   */
+  public static final String CHECKPOINT_DATA_POSTFIX = ".data";
+  /**
+   * If at the end of a checkpoint file, indicates the stitched checkpoint
+   * file prefixes.  A checkpoint is not valid if this file does not exist.
+   */
+  public static final String CHECKPOINT_FINALIZED_POSTFIX = ".finalized";
+
+  /** Class logger */
+  private static final Logger LOG = Logger.getLogger(CheckpointingUtils.class);
+
   /**
    * Do not call constructor.
    */
@@ -59,4 +92,65 @@ public class CheckpointingUtils {
                                            String jobId) {
     return new Path(getCheckpointBasePath(conf, jobId), "halt");
   }
+
+  /**
+   * Get the last saved superstep.
+   *
+   * @param fs file system where checkpoint is stored.
+   * @param checkpointBasePath path to checkpoints folder
+   * @return Last good superstep number
+   * @throws java.io.IOException
+   */
+  public static long getLastCheckpointedSuperstep(
+      FileSystem fs, String checkpointBasePath) throws IOException {
+    FileStatus[] fileStatusArray =
+        fs.listStatus(new Path(checkpointBasePath),
+            new FinalizedCheckpointPathFilter());
+    if (fileStatusArray == null) {
+      return -1;
+    }
+    long lastCheckpointedSuperstep = Long.MIN_VALUE;
+    for (FileStatus file : fileStatusArray) {
+      long superstep = getCheckpoint(file);
+      if (superstep > lastCheckpointedSuperstep) {
+        lastCheckpointedSuperstep = superstep;
+      }
+    }
+    if (LOG.isInfoEnabled()) {
+      LOG.info("getLastGoodCheckpoint: Found last good checkpoint " +
+          lastCheckpointedSuperstep);
+    }
+    return lastCheckpointedSuperstep;
+  }
+
+  /**
+   * Get the checkpoint from a finalized checkpoint path
+   *
+   * @param finalizedPath Path of the finalized checkpoint
+   * @return Superstep referring to a checkpoint of the finalized path
+   */
+  private static long getCheckpoint(FileStatus finalizedPath) {
+    if (!finalizedPath.getPath().getName().
+        endsWith(CHECKPOINT_FINALIZED_POSTFIX)) {
+      throw new InvalidParameterException(
+          "getCheckpoint: " + finalizedPath + "Doesn't end in " +
+              CHECKPOINT_FINALIZED_POSTFIX);
+    }
+    String checkpointString =
+        finalizedPath.getPath().getName().
+            replace(CHECKPOINT_FINALIZED_POSTFIX, "");
+    return Long.parseLong(checkpointString);
+  }
+
+
+  /**
+   * Only get the finalized checkpoint files
+   */
+  private static class FinalizedCheckpointPathFilter implements PathFilter {
+    @Override
+    public boolean accept(Path path) {
+      return path.getName().endsWith(CHECKPOINT_FINALIZED_POSTFIX);
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 8c24697..0081fc0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -686,21 +686,24 @@ public class WritableUtils {
    * @param output the output stream
    * @throws IOException
    */
-  public static void writeList(List<Writable> list, DataOutput output)
+  public static void writeList(List<? extends Writable> list, DataOutput output)
     throws IOException {
-    output.writeInt(list.size());
-    Class<? extends Writable> clazz = null;
-    for (Writable element : list) {
-      output.writeBoolean(element == null);
-      if (element != null) {
-        if (element.getClass() != clazz) {
-          clazz = element.getClass();
-          output.writeBoolean(true);
-          writeClass(clazz, output);
-        } else {
-          output.writeBoolean(false);
+    output.writeBoolean(list != null);
+    if (list != null) {
+      output.writeInt(list.size());
+      Class<? extends Writable> clazz = null;
+      for (Writable element : list) {
+        output.writeBoolean(element == null);
+        if (element != null) {
+          if (element.getClass() != clazz) {
+            clazz = element.getClass();
+            output.writeBoolean(true);
+            writeClass(clazz, output);
+          } else {
+            output.writeBoolean(false);
+          }
+          element.write(output);
         }
-        element.write(output);
       }
     }
   }
@@ -712,24 +715,27 @@ public class WritableUtils {
    * @return deserialized list
    * @throws IOException
    */
-  public static List<Writable> readList(DataInput input) throws IOException {
+  public static List<? extends Writable> readList(DataInput input)
+    throws IOException {
     try {
-
-      int size = input.readInt();
-      List<Writable> res = new ArrayList<>(size);
-      Class<? extends Writable> clazz = null;
-      for (int i = 0; i < size; i++) {
-        boolean isNull = input.readBoolean();
-        if (isNull) {
-          res.add(null);
-        } else {
-          boolean hasClassInfo = input.readBoolean();
-          if (hasClassInfo) {
-            clazz = readClass(input);
+      List<Writable> res = null;
+      if (input.readBoolean()) {
+        int size = input.readInt();
+        res = new ArrayList<>(size);
+        Class<? extends Writable> clazz = null;
+        for (int i = 0; i < size; i++) {
+          boolean isNull = input.readBoolean();
+          if (isNull) {
+            res.add(null);
+          } else {
+            boolean hasClassInfo = input.readBoolean();
+            if (hasClassInfo) {
+              clazz = readClass(input);
+            }
+            Writable element = clazz.newInstance();
+            element.readFields(input);
+            res.add(element);
           }
-          Writable element = clazz.newInstance();
-          element.readFields(input);
-          res.add(element);
         }
       }
       return res;
@@ -775,4 +781,79 @@ public class WritableUtils {
     return copy;
   }
 
+  /**
+   * Writes primitive int array of ints into output stream.
+   * Array can be null or empty.
+   * @param array array to be written
+   * @param dataOutput output stream
+   * @throws IOException
+   */
+  public static void writeIntArray(int[] array, DataOutput dataOutput)
+    throws IOException {
+    if (array != null) {
+      dataOutput.writeInt(array.length);
+      for (int r : array) {
+        dataOutput.writeInt(r);
+      }
+    } else {
+      dataOutput.writeInt(-1);
+    }
+  }
+
+  /**
+   * Reads primitive int array from input stream.
+   * @param dataInput input stream to read from
+   * @return may return null or empty array.
+   * @throws IOException
+   */
+  public static int[] readIntArray(DataInput dataInput)
+    throws IOException {
+    int [] res = null;
+    int size = dataInput.readInt();
+    if (size >= 0) {
+      res = new int[size];
+      for (int i = 0; i < size; i++) {
+        res[i] = dataInput.readInt();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Writes primitive long array of ints into output stream.
+   * Array can be null or empty.
+   * @param array array to be written
+   * @param dataOutput output stream
+   * @throws IOException
+   */
+  public static void writeLongArray(DataOutput dataOutput, long[] array)
+    throws IOException {
+    if (array != null) {
+      dataOutput.writeInt(array.length);
+      for (long r : array) {
+        dataOutput.writeLong(r);
+      }
+    } else {
+      dataOutput.writeInt(-1);
+    }
+  }
+  /**
+   * Reads primitive long array from input stream.
+   * @param dataInput input stream to read from
+   * @return may return null or empty array.
+   * @throws IOException
+   */
+  public static long[] readLongArray(DataInput dataInput)
+    throws IOException {
+    long [] res = null;
+    int size = dataInput.readInt();
+    if (size >= 0) {
+      res = new long[size];
+      for (int i = 0; i < size; i++) {
+        res[i] = dataInput.readLong();
+      }
+    }
+    return res;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index f61e817..4ad8400 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -86,6 +86,7 @@ import org.apache.giraph.partition.PartitionStats;
 import org.apache.giraph.partition.PartitionStore;
 import org.apache.giraph.partition.WorkerGraphPartitioner;
 import org.apache.giraph.utils.CallableFactory;
+import org.apache.giraph.utils.CheckpointingUtils;
 import org.apache.giraph.utils.JMapHistoDumper;
 import org.apache.giraph.utils.LoggerUtils;
 import org.apache.giraph.utils.MemoryUtils;
@@ -1388,12 +1389,12 @@ public class BspServiceWorker<I extends WritableComparable,
 
     // Algorithm:
     // For each partition, dump vertices and messages
-    Path metadataFilePath =
-        createCheckpointFilePathSafe(CHECKPOINT_METADATA_POSTFIX);
-    Path validFilePath =
-        createCheckpointFilePathSafe(CHECKPOINT_VALID_POSTFIX);
-    Path checkpointFilePath =
-        createCheckpointFilePathSafe(CHECKPOINT_DATA_POSTFIX);
+    Path metadataFilePath = createCheckpointFilePathSafe(
+        CheckpointingUtils.CHECKPOINT_METADATA_POSTFIX);
+    Path validFilePath = createCheckpointFilePathSafe(
+        CheckpointingUtils.CHECKPOINT_VALID_POSTFIX);
+    Path checkpointFilePath = createCheckpointFilePathSafe(
+        CheckpointingUtils.CHECKPOINT_DATA_POSTFIX);
 
 
     // Metadata is buffered and written at the end since it's small and
@@ -1521,7 +1522,7 @@ public class BspServiceWorker<I extends WritableComparable,
               }
               Path path =
                   createCheckpointFilePathSafe("_" + partitionId +
-                      CHECKPOINT_VERTICES_POSTFIX);
+                      CheckpointingUtils.CHECKPOINT_VERTICES_POSTFIX);
 
               FSDataOutputStream uncompressedStream =
                   getFs().create(path);
@@ -1592,7 +1593,7 @@ public class BspServiceWorker<I extends WritableComparable,
               }
               Path path =
                   getSavedCheckpoint(superstep, "_" + partitionId +
-                      CHECKPOINT_VERTICES_POSTFIX);
+                      CheckpointingUtils.CHECKPOINT_VERTICES_POSTFIX);
 
               FSDataInputStream compressedStream =
                   getFs().open(path);
@@ -1626,11 +1627,11 @@ public class BspServiceWorker<I extends WritableComparable,
 
   @Override
   public VertexEdgeCount loadCheckpoint(long superstep) {
-    Path metadataFilePath =
-        getSavedCheckpoint(superstep, CHECKPOINT_METADATA_POSTFIX);
+    Path metadataFilePath = getSavedCheckpoint(
+        superstep, CheckpointingUtils.CHECKPOINT_METADATA_POSTFIX);
 
-    Path checkpointFilePath =
-        getSavedCheckpoint(superstep, CHECKPOINT_DATA_POSTFIX);
+    Path checkpointFilePath = getSavedCheckpoint(
+        superstep, CheckpointingUtils.CHECKPOINT_DATA_POSTFIX);
     // Algorithm:
     // Examine all the partition owners and load the ones
     // that match my hostname and id from the master designated checkpoint
@@ -1659,8 +1660,8 @@ public class BspServiceWorker<I extends WritableComparable,
       // Load global stats and superstep classes
       GlobalStats globalStats = new GlobalStats();
       SuperstepClasses superstepClasses = new SuperstepClasses();
-      String finalizedCheckpointPath =
-          getSavedCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
+      String finalizedCheckpointPath = getSavedCheckpointBasePath(superstep) +
+          CheckpointingUtils.CHECKPOINT_FINALIZED_POSTFIX;
       DataInputStream finalizedStream =
           getFs().open(new Path(finalizedCheckpointPath));
       globalStats.readFields(finalizedStream);
@@ -1674,7 +1675,8 @@ public class BspServiceWorker<I extends WritableComparable,
             checkpointStream, partitionId);
       }
 
-      List<Writable> w2wMessages = WritableUtils.readList(checkpointStream);
+      List<Writable> w2wMessages = (List<Writable>) WritableUtils.readList(
+          checkpointStream);
       getServerData().getCurrentWorkerToWorkerMessages().addAll(w2wMessages);
 
       checkpointStream.close();

http://git-wip-us.apache.org/repos/asf/giraph/blob/54a1a8de/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java b/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
index c712b5a..8c97b69 100644
--- a/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java
@@ -61,10 +61,48 @@ public class TestWritableUtils {
     DataInputStream input =
         new DataInputStream(new ByteArrayInputStream(data));
 
-    List<Writable> result = WritableUtils.readList(input);
+    List<Writable> result = (List<Writable>) WritableUtils.readList(input);
 
     Assert.assertEquals(list, result);
 
   }
 
+  @Test
+  public void testIntArray() throws IOException {
+    int[] array = new int[] {1, 2, 3, 4, 5};
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(bos);
+    WritableUtils.writeIntArray(array, dos);
+    dos.close();
+
+    byte[] data = bos.toByteArray();
+
+    DataInputStream input =
+        new DataInputStream(new ByteArrayInputStream(data));
+
+    int[] result = WritableUtils.readIntArray(input);
+
+    Assert.assertArrayEquals(array, result);
+  }
+
+  @Test
+  public void testLongArray() throws IOException {
+    long[] array = new long[] {1, 2, 3, 4, 5};
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(bos);
+    WritableUtils.writeLongArray(dos, array);
+    dos.close();
+
+    byte[] data = bos.toByteArray();
+
+    DataInputStream input =
+        new DataInputStream(new ByteArrayInputStream(data));
+
+    long[] result = WritableUtils.readLongArray(input);
+
+    Assert.assertArrayEquals(array, result);
+  }
+
+
+
 }


[03/47] git commit: updated refs/heads/release-1.1 to 4c139ee

Posted by rv...@apache.org.
GIRAPH-915: With BigDataIO some messages can get ignored (majakabiljo via pavanka)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/58576c81
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/58576c81
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/58576c81

Branch: refs/heads/release-1.1
Commit: 58576c81fed9f4eccb03366151c1cf280765d237
Parents: ffdddff
Author: Pavan Kumar <pa...@fb.com>
Authored: Wed Jun 18 15:38:30 2014 -0700
Committer: Pavan Kumar <pa...@fb.com>
Committed: Wed Jun 18 15:38:30 2014 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  2 ++
 .../SendWorkerOneToAllMessagesRequest.java      |  2 +-
 .../org/apache/giraph/edge/ByteArrayEdges.java  |  2 +-
 .../apache/giraph/utils/ByteStructIterator.java |  2 +-
 .../utils/ByteStructVertexIdIterator.java       |  2 +-
 .../utils/ExtendedByteArrayDataInput.java       |  5 ++++
 .../apache/giraph/utils/ExtendedDataInput.java  |  7 +++++
 .../apache/giraph/utils/UnsafeArrayReads.java   |  5 ++++
 .../org/apache/giraph/utils/VertexIterator.java |  2 +-
 .../apache/giraph/utils/io/BigDataInput.java    | 31 +++++++++++++-------
 10 files changed, 44 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index f35d4ba..659edfd 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-915: With BigDataIO some messages can get ignored (majakabiljo via pavanka)
+
   GIRAPH-918: GIRAPH-908 has a small bug reg counting entries (pavanka)
 
   GIRAPH-842: option to dump histogram of memory usage when heap is low on memory (pavanka)

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
index 8745adb..5f1ed53 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerOneToAllMessagesRequest.java
@@ -116,7 +116,7 @@ public class SendWorkerOneToAllMessagesRequest<I extends WritableComparable,
     int idCount = 0;
     int partitionId = 0;
     try {
-      while (reader.available() != 0) {
+      while (!reader.endOfInput()) {
         msg.readFields(reader);
         idCount = reader.readInt();
         for (int i = 0; i < idCount; i++) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
index 271e9c5..509546c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
@@ -159,7 +159,7 @@ public class ByteArrayEdges<I extends WritableComparable, E extends Writable>
 
     @Override
     public boolean hasNext() {
-      return serializedEdges != null && extendedDataInput.available() > 0;
+      return serializedEdges != null && !extendedDataInput.endOfInput();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructIterator.java
index 322365c..1f1b90e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructIterator.java
@@ -44,7 +44,7 @@ public abstract class ByteStructIterator<T extends Writable> implements
 
   @Override
   public boolean hasNext() {
-    return extendedDataInput.available() > 0;
+    return !extendedDataInput.endOfInput();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdIterator.java
index 3d564cd..3b880a0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteStructVertexIdIterator.java
@@ -54,7 +54,7 @@ public abstract class ByteStructVertexIdIterator<I extends WritableComparable>
 
   @Override
   public boolean hasNext() {
-    return extendedDataInput.available() > 0;
+    return !extendedDataInput.endOfInput();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayDataInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayDataInput.java b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayDataInput.java
index 3eae25b..56c79c4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayDataInput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedByteArrayDataInput.java
@@ -57,6 +57,11 @@ public class ExtendedByteArrayDataInput extends ByteArrayInputStream
   }
 
   @Override
+  public boolean endOfInput() {
+    return available() == 0;
+  }
+
+  @Override
   public void readFully(byte[] b) throws IOException {
     dataInput.readFully(b);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataInput.java b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataInput.java
index f1c6809..96096ed 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataInput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ExtendedDataInput.java
@@ -36,4 +36,11 @@ public interface ExtendedDataInput extends DataInput {
    * @return Bytes available
    */
   int available();
+
+  /**
+   * Check if we read everything from the input
+   *
+   * @return True iff we read everything from the input
+   */
+  boolean endOfInput();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
index db19fda..1ab8de6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/UnsafeArrayReads.java
@@ -84,6 +84,11 @@ public class UnsafeArrayReads extends UnsafeReads {
     return (int) (bufLength - pos);
   }
 
+  @Override
+  public boolean endOfInput() {
+    return available() == 0;
+  }
+
 
   @Override
   public int getPos() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/VertexIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIterator.java
index dced9bd..dd73b1f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIterator.java
@@ -77,7 +77,7 @@ public class VertexIterator<I extends WritableComparable,
    * @return True if the iteration has more elements.
    */
   public boolean hasNext() {
-    return extendedDataInput.available() > 0;
+    return !extendedDataInput.endOfInput();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/58576c81/giraph-core/src/main/java/org/apache/giraph/utils/io/BigDataInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/io/BigDataInput.java b/giraph-core/src/main/java/org/apache/giraph/utils/io/BigDataInput.java
index f73819a..2454a37 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/io/BigDataInput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/io/BigDataInput.java
@@ -76,7 +76,7 @@ public class BigDataInput implements ExtendedDataInput {
    * next one if needed.
    */
   private void checkIfShouldMoveToNextDataInput() {
-    if (currentInput.available() == 0) {
+    if (currentInput.endOfInput()) {
       moveToNextDataInput();
     }
   }
@@ -168,12 +168,17 @@ public class BigDataInput implements ExtendedDataInput {
   @Override
   public int skipBytes(int n) throws IOException {
     int bytesLeftToSkip = n;
-    while (bytesLeftToSkip >= currentInput.available()) {
-      bytesLeftToSkip -= currentInput.available();
-      moveToNextDataInput();
+    while (bytesLeftToSkip > 0) {
+      int bytesSkipped = currentInput.skipBytes(bytesLeftToSkip);
+      bytesLeftToSkip -= bytesSkipped;
+      if (bytesLeftToSkip > 0) {
+        moveToNextDataInput();
+        if (endOfInput()) {
+          break;
+        }
+      }
     }
-    int bytesSkipped = currentInput.skipBytes(bytesLeftToSkip);
-    return n - bytesLeftToSkip + bytesSkipped;
+    return n - bytesLeftToSkip;
   }
 
   @Override
@@ -187,10 +192,14 @@ public class BigDataInput implements ExtendedDataInput {
 
   @Override
   public int available() {
-    int available = 0;
-    for (int i = currentPositionInInputs; i < dataInputs.size(); i++) {
-      available += dataInputs.get(i).available();
-    }
-    return available;
+    throw new UnsupportedOperationException("available: " +
+        "Not supported with BigDataIO because overflow can happen");
+  }
+
+  @Override
+  public boolean endOfInput() {
+    return currentInput == EMPTY_INPUT ||
+        (dataInputs.get(currentPositionInInputs).endOfInput() &&
+            currentPositionInInputs == dataInputs.size() - 1);
   }
 }