You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ik...@apache.org on 2015/06/11 04:50:40 UTC

[1/5] git commit: updated refs/heads/trunk to 819d6d3

Repository: giraph
Updated Branches:
  refs/heads/trunk 06a1084af -> 819d6d38d


http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java
new file mode 100644
index 0000000..e82c3a8
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java
@@ -0,0 +1,463 @@
+/*
+ * 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 it.unimi.dsi.fastutil.bytes.ByteArrays;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+
+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.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.giraph.utils.EdgeIterables;
+import org.apache.giraph.utils.Trimmable;
+import org.apache.giraph.utils.Varint;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Implementation of {@link org.apache.giraph.edge.OutEdges} with long ids
+ * and null edge values, 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.
+ * Users of this class should explicitly call {@link #trim()} function
+ * to compact in-memory representation after all updates are done.
+ * Compacting object is expensive so should only be done once after bulk update.
+ * Compaction can also be caused by serialization attempt or
+ * by calling {@link #iterator()}
+ */
+@NotThreadSafe
+public class LongDiffNullArrayEdges
+    extends ConfigurableOutEdges<LongWritable, NullWritable>
+    implements ReuseObjectsOutEdges<LongWritable, NullWritable>,
+    MutableOutEdges<LongWritable, NullWritable>, Trimmable {
+
+  /**
+   * Array of target vertex ids.
+   */
+  private byte[] compressedData;
+  /**
+   * Number of edges stored in compressed array.
+   * There may be some extra edges in transientData or there may be some edges
+   * removed. These will not count here. To get real number of elements stored
+   * in this object @see {@link #size()}
+   */
+  private int size;
+
+  /**
+   * Last updates are stored here. We clear them out after object is compacted.
+   */
+  private TransientChanges transientData;
+
+  @Override
+  public void initialize(Iterable<Edge<LongWritable, NullWritable>> edges) {
+    reset();
+    EdgeIterables.initialize(this, edges);
+    trim();
+  }
+
+  @Override
+  public void initialize(int capacity) {
+    reset();
+    if (capacity > 0) {
+      transientData = new TransientChanges(capacity);
+    }
+  }
+
+  @Override
+  public void initialize() {
+    reset();
+  }
+
+  @Override
+  public void add(Edge<LongWritable, NullWritable> edge) {
+    checkTransientData();
+    transientData.add(edge.getTargetVertexId().get());
+  }
+
+
+  @Override
+  public void remove(LongWritable targetVertexId) {
+    checkTransientData();
+    long target = targetVertexId.get();
+
+    if (size > 0) {
+      LongsDiffReader reader = new LongsDiffReader(compressedData);
+      for (int i = 0; i < size; i++) {
+        long cur = reader.readNext();
+        if (cur == target) {
+          transientData.markRemoved(i);
+        } else if (cur > target) {
+          break;
+        }
+      }
+    }
+    transientData.removeAdded(target);
+  }
+
+  @Override
+  public int size() {
+    int result = size;
+    if (transientData != null) {
+      result += transientData.size();
+    }
+    return result;
+  }
+
+  @Override
+  public Iterator<Edge<LongWritable, 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<LongWritable, NullWritable>> mutableIterator() {
+    trim();
+    return new Iterator<MutableEdge<LongWritable, NullWritable>>() {
+      /** Current position in the array. */
+      private int position;
+      private final LongsDiffReader reader =
+          new LongsDiffReader(compressedData);
+
+      /** Representative edge object. */
+      private final MutableEdge<LongWritable, NullWritable> representativeEdge =
+          EdgeFactory.createReusable(new LongWritable());
+
+      @Override
+      public boolean hasNext() {
+        return position < size;
+      }
+
+      @Override
+      public MutableEdge<LongWritable, NullWritable> next() {
+        position++;
+        representativeEdge.getTargetVertexId().set(reader.readNext());
+        return representativeEdge;
+      }
+
+      @Override
+      public void remove() {
+        removeAt(position - 1);
+      }
+    };
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    trim();
+    Varint.writeUnsignedVarInt(compressedData.length, out);
+    Varint.writeUnsignedVarInt(size, out);
+    out.write(compressedData);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    reset();
+    compressedData = new byte[Varint.readUnsignedVarInt(in)];
+    // We can actually calculate size after data array is read,
+    // the trade-off is memory vs speed
+    size = Varint.readUnsignedVarInt(in);
+    in.readFully(compressedData);
+  }
+
+  /**
+   * This function takes all recent updates and stores them efficiently.
+   * It is safe to call this function multiple times.
+   */
+  @Override
+  public void trim() {
+    if (transientData == null) {
+      // We don't have any updates to this object. Return quickly.
+      return;
+    }
+
+    // Beware this array is longer than the number of elements we interested in
+    long[] transientValues = transientData.sortedValues();
+    int pCompressed = 0;
+    int pTransient = 0;
+
+    LongsDiffReader reader = new LongsDiffReader(compressedData);
+    LongsDiffWriter writer = new LongsDiffWriter();
+
+    long curValue = size > 0 ? reader.readNext() : Long.MAX_VALUE;
+
+    // Here we merge freshly added elements and old elements, we also want
+    // to prune removed elements. Both arrays are sorted so in order to merge
+    // them, we move to pointers and store result in the new array
+    while (pTransient < transientData.numberOfAddedElements() ||
+        pCompressed < size) {
+      if (pTransient < transientData.numberOfAddedElements() &&
+          curValue >= transientValues[pTransient]) {
+        writer.writeNext(transientValues[pTransient]);
+        pTransient++;
+      } else {
+        if (!transientData.isRemoved(pCompressed)) {
+          writer.writeNext(curValue);
+        }
+        pCompressed++;
+        if (pCompressed < size) {
+          curValue = reader.readNext();
+        } else {
+          curValue = Long.MAX_VALUE;
+        }
+      }
+    }
+
+    compressedData = writer.toByteArray();
+    size += transientData.size();
+    transientData = null;
+  }
+
+
+  /**
+   * Remove edge at position i.
+   *
+   * @param i Position of edge to be removed
+   */
+  private void removeAt(int i) {
+    checkTransientData();
+    if (i < size) {
+      transientData.markRemoved(i);
+    } else {
+      transientData.removeAddedAt(i - size);
+    }
+  }
+
+  /**
+   * Check if transient data needs to be created.
+   */
+  private void checkTransientData() {
+    if (transientData == null) {
+      transientData = new TransientChanges();
+    }
+  }
+
+  /**
+   * Reset object to completely empty state.
+   */
+  private void reset() {
+    compressedData = ByteArrays.EMPTY_ARRAY;
+    size = 0;
+    transientData = null;
+  }
+
+  /**
+   * Reading array of longs diff encoded from byte array.
+   */
+  private static class LongsDiffReader {
+    /** Input stream */
+    private final DataInput input;
+    /** last read value */
+    private long current;
+    /** True if we haven't read any numbers yet */
+    private boolean first = true;
+
+    /**
+     * Construct LongsDiffReader
+     * @param compressedData Input byte array
+     */
+    LongsDiffReader(byte[] compressedData) {
+      input = new DataInputStream(new ByteArrayInputStream(compressedData));
+    }
+
+    /**
+     * Read next value from reader
+     * @return next value
+     */
+    long readNext() {
+      try {
+        if (first) {
+          current = input.readLong();
+          first = false;
+        } else {
+          current += Varint.readUnsignedVarLong(input);
+        }
+        return current;
+      } catch (IOException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+  }
+
+  /**
+   * Writing array of longs diff encoded into the byte array.
+   */
+  private static class LongsDiffWriter {
+    /** Byte array stream containing result */
+    private final ByteArrayOutputStream resultStream =
+        new ByteArrayOutputStream();
+    /** Wrapping resultStream into DataOutputStream */
+    private final DataOutputStream out = new DataOutputStream(resultStream);
+    /** last value written */
+    private long lastWritten;
+    /** True if we haven't written any numbers yet */
+    private boolean first = true;
+
+    /**
+     * Write next value to writer
+     * @param value Value to be written
+     */
+    void writeNext(long value) {
+      try {
+        if (first) {
+          out.writeLong(value);
+          first = false;
+        } else {
+          Preconditions.checkState(value >= lastWritten,
+              "Values need to be in order");
+          Preconditions.checkState((value - lastWritten) >= 0,
+              "In order to use this class, difference of consecutive IDs " +
+              "cannot overflow longs");
+          Varint.writeUnsignedVarLong(value - lastWritten, out);
+        }
+        lastWritten = value;
+      } catch (IOException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+
+    /**
+     * Get resulting byte array
+     * @return resulting byte array
+     */
+    byte[] toByteArray() {
+      return resultStream.toByteArray();
+    }
+  }
+
+  /**
+   * Temporary storage for all updates.
+   * We don't want to update compressed array frequently so we only update it
+   * on request at the same time we allow temporary updates to persist in this
+   * class.
+   */
+  private static class TransientChanges {
+    /** Neighbors that were added since last flush */
+    private final LongArrayList neighborsAdded;
+    /** Removed indices in original array */
+    private final BitSet removed = new BitSet();
+    /** Number of values removed */
+    private int removedCount;
+
+    /**
+     * Construct transient changes with given capacity
+     * @param capacity capacity
+     */
+    private TransientChanges(int capacity) {
+      neighborsAdded = new LongArrayList(capacity);
+    }
+
+    /**
+     * Construct transient changes
+     */
+    private TransientChanges() {
+      neighborsAdded = new LongArrayList();
+    }
+
+    /**
+     * Add new value
+     * @param value value to add
+     */
+    private void add(long value) {
+      neighborsAdded.add(value);
+    }
+
+    /**
+     * Mark given index to remove
+     * @param index Index to remove
+     */
+    private void markRemoved(int index) {
+      if (!removed.get(index)) {
+        removedCount++;
+        removed.set(index);
+      }
+    }
+
+    /**
+     * Remove value from neighborsAdded
+     * @param index Position to remove from
+     */
+    private void removeAddedAt(int index) {
+      // The order of the edges is irrelevant, so we can simply replace
+      // the deleted edge with the rightmost element, thus achieving constant
+      // time.
+      if (index == neighborsAdded.size() - 1) {
+        neighborsAdded.popLong();
+      } else {
+        neighborsAdded.set(index, neighborsAdded.popLong());
+      }
+    }
+
+    /**
+     * Number of added elements
+     * @return number of added elements
+     */
+    private int numberOfAddedElements() {
+      return neighborsAdded.size();
+    }
+
+    /**
+     * Remove added value
+     * @param target value to remove
+     */
+    private void removeAdded(long target) {
+      neighborsAdded.rem(target);
+    }
+
+    /**
+     * Additional size in transient changes
+     * @return additional size
+     */
+    private int size() {
+      return neighborsAdded.size() - removedCount;
+    }
+
+    /**
+     * Sorted added values
+     * @return sorted added values
+     */
+    private long[] sortedValues() {
+      long[] ret = neighborsAdded.elements();
+      Arrays.sort(ret, 0, neighborsAdded.size());
+      return ret;
+    }
+
+    /**
+     * Check if index was removed
+     * @param i Index to check
+     * @return Whether it was removed
+     */
+    private boolean isRemoved(int i) {
+      return removed.get(i);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java
new file mode 100644
index 0000000..ae31bb2
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java
@@ -0,0 +1,118 @@
+/*
+ * 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.writable.kryo.KryoWritableWrapper;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Utility methods for dealing with Hadoop configuration
+ */
+public class ConfigurationObjectUtils {
+  /** Hide constructor */
+  private ConfigurationObjectUtils() {
+  }
+
+  /**
+   * Encode bytes to a hex String
+   *
+   * @param bytes byte[]
+   * @return encoded String
+   */
+  public static String encodeBytes(byte[] bytes) {
+    StringBuilder strBuf = new StringBuilder();
+    for (int i = 0; i < bytes.length; i++) {
+      strBuf.append((char) (((bytes[i] >> 4) & 0xF) + ('a')));
+      strBuf.append((char) (((bytes[i]) & 0xF) + ('a')));
+    }
+    return strBuf.toString();
+  }
+
+  /**
+   * Decode bytes from a hex String
+   *
+   * @param str String to decode
+   * @return decoded byte[]
+   */
+  public static byte[] decodeBytes(String str) {
+    byte[] bytes = new byte[str.length() / 2];
+    for (int i = 0; i < str.length(); i += 2) {
+      char c = str.charAt(i);
+      bytes[i / 2] = (byte) ((c - 'a') << 4);
+      c = str.charAt(i + 1);
+      bytes[i / 2] += c - 'a';
+    }
+    return bytes;
+  }
+
+  /**
+   * Set byte array to a conf option
+   *
+   * @param data Byte array
+   * @param confOption Conf option
+   * @param conf Configuration
+   */
+  public static void setByteArray(byte[] data, String confOption,
+      Configuration conf) {
+    conf.set(confOption, encodeBytes(data));
+  }
+
+  /**
+   * Get byte array from a conf option
+   *
+   * @param confOption Conf option
+   * @param conf Configuration
+   * @return Byte array
+   */
+  public static byte[] getByteArray(String confOption,
+      Configuration conf) {
+    return decodeBytes(conf.get(confOption));
+  }
+
+  /**
+   * Set object in a conf option using kryo
+   *
+   * @param object Object to set
+   * @param confOption Conf option
+   * @param conf Configuration
+   * @param <T> Type of the object
+   */
+  public static <T> void setObjectKryo(T object, String confOption,
+      Configuration conf) {
+    setByteArray(WritableUtils.toByteArrayUnsafe(
+            new KryoWritableWrapper<>(object)),
+        confOption, conf);
+  }
+
+  /**
+   * Get object from a conf option using kryo
+   *
+   * @param confOption Conf option
+   * @param conf Configuration
+   * @return Object from conf
+   * @param <T> Type of the object
+   */
+  public static <T> T getObjectKryo(String confOption,
+      Configuration conf) {
+    KryoWritableWrapper<T> wrapper = new KryoWritableWrapper<>();
+    WritableUtils.fromByteArrayUnsafe(
+        getByteArray(confOption, conf), wrapper,
+        new UnsafeReusableByteArrayInput());
+    return wrapper.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java b/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java
new file mode 100644
index 0000000..ef273b0
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.IOException;
+
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * Output committer which has abstract commit method
+ */
+public abstract class DefaultOutputCommitter extends OutputCommitter {
+  /**
+   * For cleaning up the job's output after job completion. Note that this
+   * is invoked for jobs with final run state as
+   * {@link org.apache.hadoop.mapreduce.JobStatus.State#SUCCEEDED}
+   *
+   * @param jobContext Context of the job whose output is being written.
+   */
+  public abstract void commit(JobContext jobContext) throws IOException;
+
+  @Override
+  public final void setupJob(JobContext jobContext) throws IOException {
+  }
+
+  @Override
+  public final void setupTask(TaskAttemptContext taskContext)
+      throws IOException {
+  }
+
+  @Override
+  public final void commitJob(JobContext jobContext)
+      throws IOException {
+    super.commitJob(jobContext);
+    commit(jobContext);
+  }
+
+  @Override
+  public final boolean needsTaskCommit(TaskAttemptContext taskContext)
+      throws IOException {
+    // Digraph does not require a task commit and there is a bug in Corona
+    // which triggers t5688706
+    // Avoiding the task commit should work around this.
+    return false;
+  }
+
+  @Override
+  public final void commitTask(TaskAttemptContext context) throws IOException {
+  }
+
+  @Override
+  public final void abortTask(TaskAttemptContext taskContext)
+      throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/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 68ed89a..2229c2f 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
@@ -34,6 +34,7 @@ import java.util.List;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.factories.ValueFactory;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.zk.ZooKeeperExt;
 import org.apache.giraph.zk.ZooKeeperExt.PathStat;
@@ -950,4 +951,190 @@ public class WritableUtils {
     }
     return copy;
   }
+
+  /**
+   * Create a copy of Writable object, by serializing and deserializing it.
+   *
+   * @param original Original value of which to make a copy
+   * @return Copy of the original value
+   * @param <T> Type of the object
+   */
+  public static final <T extends Writable> T createCopy(T original) {
+    return (T) createCopy(original, original.getClass(), null);
+  }
+
+  /**
+   * Create a copy of Writable object, by serializing and deserializing it.
+   *
+   * @param original Original value of which to make a copy
+   * @param outputClass Expected copy class, needs to match original
+   * @param conf Configuration
+   * @return Copy of the original value
+   * @param <T> Type of the object
+   */
+  public static final <T extends Writable>
+  T createCopy(T original, Class<? extends T> outputClass,
+      ImmutableClassesGiraphConfiguration conf) {
+    T result = WritableUtils.createWritable(outputClass, conf);
+    copyInto(original, result);
+    return result;
+  }
+
+  /**
+   * Create a copy of Writable object, by serializing and deserializing it.
+   *
+   * @param original Original value of which to make a copy
+   * @param classFactory Factory to create new empty object from
+   * @param conf Configuration
+   * @return Copy of the original value
+   * @param <T> Type of the object
+   */
+  public static final <T extends Writable>
+  T createCopy(T original, ValueFactory<T> classFactory,
+      ImmutableClassesGiraphConfiguration conf) {
+    T result = classFactory.newInstance();
+    copyInto(original, result);
+    return result;
+  }
+
+  /**
+   * Serialize given writable object, and return it's size.
+   *
+   * @param w Writable object
+   * @return it's size after serialization
+   */
+  public static int size(Writable w) {
+    try {
+      ExtendedByteArrayDataOutput out = new ExtendedByteArrayDataOutput();
+      w.write(out);
+      return out.getPos();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Serialize given writable to byte array,
+   * using new instance of ExtendedByteArrayDataOutput.
+   *
+   * @param w Writable object
+   * @return array of bytes
+   * @param <T> Type of the object
+   */
+  public static <T extends Writable> byte[] toByteArray(T w) {
+    try {
+      ExtendedByteArrayDataOutput out = new ExtendedByteArrayDataOutput();
+      w.write(out);
+      return out.toByteArray();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Deserialize from given byte array into given writable,
+   * using new instance of ExtendedByteArrayDataInput.
+   *
+   * @param data Byte array representing writable
+   * @param to Object to fill
+   * @param <T> Type of the object
+   */
+  public static <T extends Writable> void fromByteArray(byte[] data, T to) {
+    try {
+      ExtendedByteArrayDataInput in =
+          new ExtendedByteArrayDataInput(data, 0, data.length);
+      to.readFields(in);
+
+      if (in.available() != 0) {
+        throw new RuntimeException(
+            "Serialization encountered issues, " + in.available() +
+            " bytes left to be read");
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Serialize given writable to byte array,
+   * using new instance of UnsafeByteArrayOutputStream.
+   *
+   * @param w Writable object
+   * @return array of bytes
+   * @param <T> Type of the object
+   */
+  public static <T extends Writable> byte[] toByteArrayUnsafe(T w) {
+    try {
+      UnsafeByteArrayOutputStream out = new UnsafeByteArrayOutputStream();
+      w.write(out);
+      return out.toByteArray();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Deserialize from given byte array into given writable,
+   * using given reusable UnsafeReusableByteArrayInput.
+   *
+   * @param data Byte array representing writable
+   * @param to Object to fill
+   * @param reusableInput Reusable input to use
+   * @param <T> Type of the object
+   */
+  public static <T extends Writable> void fromByteArrayUnsafe(
+      byte[] data, T to, UnsafeReusableByteArrayInput reusableInput) {
+    try {
+      reusableInput.initialize(data, 0, data.length);
+      to.readFields(reusableInput);
+
+      if (reusableInput.available() != 0) {
+        throw new RuntimeException(
+            "Serialization encountered issues, " + reusableInput.available() +
+            " bytes left to be read");
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * First write a boolean saying whether an object is not null,
+   * and if it's not write the object
+   *
+   * @param object Object to write
+   * @param out DataOutput to write to
+   * @param <T> Object type
+   */
+  public static <T extends Writable> void writeIfNotNullAndObject(T object,
+      DataOutput out) throws IOException {
+    out.writeBoolean(object != null);
+    if (object != null) {
+      object.write(out);
+    }
+  }
+
+  /**
+   * First read a boolean saying whether an object is not null,
+   * and if it's not read the object
+   *
+   * @param reusableObject Reuse this object instance
+   * @param objectClass Class of the object, to create if reusableObject is null
+   * @param in DataInput to read from
+   * @param <T> Object type
+   * @return Object, or null
+   */
+  public static <T extends Writable> T readIfNotNullAndObject(T reusableObject,
+      Class<T> objectClass, DataInput in) throws IOException {
+    if (in.readBoolean()) {
+      if (reusableObject == null) {
+        reusableObject = ReflectionUtils.newInstance(objectClass);
+      }
+      reusableObject.readFields(in);
+      return reusableObject;
+    } else {
+      return null;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java
new file mode 100644
index 0000000..e23f592
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java
@@ -0,0 +1,260 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.UnsafeByteArrayInputStream;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+
+public class LongDiffNullArrayEdgesTest {
+  private static Edge<LongWritable, NullWritable> createEdge(long id) {
+    return EdgeFactory.create(new LongWritable(id));
+  }
+
+  private static void assertEdges(LongDiffNullArrayEdges edges, long... expected) {
+    int index = 0;
+    for (Edge<LongWritable, NullWritable> edge : edges) {
+      Assert.assertEquals(expected[index], edge.getTargetVertexId().get());
+      index++;
+    }
+    Assert.assertEquals(expected.length, index);
+  }
+
+  @Test
+  public void testEdges() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+        createEdge(1), createEdge(2), createEdge(4));
+
+    edges.initialize(initialEdges);
+    assertEdges(edges, 1, 2, 4);
+
+    edges.add(EdgeFactory.createReusable(new LongWritable(3)));
+    assertEdges(edges, 1, 2, 3, 4);
+
+    edges.remove(new LongWritable(2));
+    assertEdges(edges, 1, 3, 4);
+  }
+
+  @Test
+  public void testPositiveAndNegativeEdges() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+        createEdge(1), createEdge(-2), createEdge(3), createEdge(-4));
+
+    edges.initialize(initialEdges);
+    assertEdges(edges, -4, -2, 1, 3);
+
+    edges.add(EdgeFactory.createReusable(new LongWritable(5)));
+    assertEdges(edges, -4, -2, 1, 3, 5);
+
+    edges.remove(new LongWritable(-2));
+    assertEdges(edges, -4, 1, 3, 5);
+  }
+
+  @Test
+  public void testMutateEdges() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    edges.initialize();
+
+    // Add 10 edges with id i, for i = 0..9
+    for (int i = 0; i < 10; ++i) {
+      edges.add(createEdge(i));
+    }
+
+    // Use the mutable iterator to remove edges with even id
+    Iterator<MutableEdge<LongWritable, NullWritable>> edgeIt =
+        edges.mutableIterator();
+    while (edgeIt.hasNext()) {
+      if (edgeIt.next().getTargetVertexId().get() % 2 == 0) {
+        edgeIt.remove();
+      }
+    }
+
+    // We should now have 5 edges
+    assertEquals(5, edges.size());
+    // The edge ids should be all odd
+    for (Edge<LongWritable, NullWritable> edge : edges) {
+      assertEquals(1, edge.getTargetVertexId().get() % 2);
+    }
+  }
+
+  @Test
+  public void testSerialization() throws IOException {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    edges.initialize();
+
+    // Add 10 edges with id i, for i = 0..9
+    for (int i = 0; i < 10; ++i) {
+      edges.add(createEdge(i));
+    }
+
+    edges.trim();
+
+    // Use the mutable iterator to remove edges with even id
+    Iterator<MutableEdge<LongWritable, NullWritable>> edgeIt =
+        edges.mutableIterator();
+    while (edgeIt.hasNext()) {
+      if (edgeIt.next().getTargetVertexId().get() % 2 == 0) {
+        edgeIt.remove();
+      }
+    }
+
+    // We should now have 5 edges
+    assertEdges(edges, 1, 3, 5, 7, 9);
+
+    ByteArrayOutputStream arrayStream = new ByteArrayOutputStream();
+    DataOutputStream tempBuffer = new DataOutputStream(arrayStream);
+
+    edges.write(tempBuffer);
+
+    byte[] binary = arrayStream.toByteArray();
+
+    assertTrue("Serialized version should not be empty ", binary.length > 0);
+
+    edges = getEdges();
+    edges.readFields(new UnsafeByteArrayInputStream(binary));
+
+    assertEquals(5, edges.size());
+
+    for (Edge<LongWritable, NullWritable> edge : edges) {
+      assertEquals(1, edge.getTargetVertexId().get() % 2);
+    }
+  }
+
+  @Test
+  public void testParallelEdges() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+        createEdge(2), createEdge(2), createEdge(2));
+
+    edges.initialize(initialEdges);
+    assertEquals(3, edges.size());
+
+    edges.remove(new LongWritable(2));
+    assertEquals(0, edges.size());
+
+    edges.add(EdgeFactory.create(new LongWritable(2)));
+    assertEquals(1, edges.size());
+
+    edges.trim();
+    assertEquals(1, edges.size());
+  }
+
+  @Test
+  public void testEdgeValues() {
+    LongDiffNullArrayEdges edges = getEdges();
+    Set<Long> testValues = new HashSet<Long>();
+    testValues.add(0L);
+    testValues.add((long) Integer.MAX_VALUE);
+    testValues.add(Long.MAX_VALUE);
+
+    // shouldn't be working with negative IDs
+    // testValues.add((long) Integer.MIN_VALUE);
+    // testValues.add(Long.MIN_VALUE);
+
+    List<Edge<LongWritable, NullWritable>> initialEdges =
+        new ArrayList<Edge<LongWritable, NullWritable>>();
+    for(Long id : testValues) {
+      initialEdges.add(createEdge(id));
+    }
+
+    edges.initialize(initialEdges);
+    edges.trim();
+
+    Iterator<MutableEdge<LongWritable, NullWritable>> edgeIt =
+        edges.mutableIterator();
+    while (edgeIt.hasNext()) {
+      long value = edgeIt.next().getTargetVertexId().get();
+      assertTrue("Unknown edge found " + value, testValues.remove(value));
+    }
+  }
+
+  private LongDiffNullArrayEdges getEdges() {
+    GiraphConfiguration gc = new GiraphConfiguration();
+    ImmutableClassesGiraphConfiguration<LongWritable, Writable, NullWritable> conf =
+        new ImmutableClassesGiraphConfiguration<LongWritable, Writable, NullWritable>(gc);
+    LongDiffNullArrayEdges ret = new LongDiffNullArrayEdges();
+    ret.setConf(new ImmutableClassesGiraphConfiguration<LongWritable, Writable, NullWritable>(conf));
+    return ret;
+  }
+
+  @Test
+  public void testAddedSmalerValues() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+        createEdge(100));
+
+    edges.initialize(initialEdges);
+
+    edges.trim();
+
+    for (int i=0; i<16; i++) {
+      edges.add(createEdge(i));
+    }
+
+    edges.trim();
+
+    assertEquals(17, edges.size());
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testFailSafeOnPotentialOverflow() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+        createEdge(5223372036854775807L), createEdge(-4223372036854775807L));
+    edges.initialize(initialEdges);
+  }
+
+  @Test
+  public void testAvoidOverflowWithZero() {
+    LongDiffNullArrayEdges edges = getEdges();
+
+    List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+        createEdge(5223372036854775807L), createEdge(-4223372036854775807L), createEdge(0));
+    edges.initialize(initialEdges);
+    assertEdges(edges, -4223372036854775807L, 0, 5223372036854775807L);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 63a9bae..4198f13 100644
--- a/pom.xml
+++ b/pom.xml
@@ -275,6 +275,8 @@ under the License.
     <!-- This lets modules skip unit tests. More details: GIRAPH-957 --> 
     <surefire.skip>false</surefire.skip>
 
+    <checkstyle.config.path>${top.dir}/checkstyle.xml</checkstyle.config.path>
+
     <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>
@@ -541,7 +543,7 @@ under the License.
           <artifactId>maven-checkstyle-plugin</artifactId>
           <version>2.15</version>
           <configuration>
-            <configLocation>${top.dir}/checkstyle.xml</configLocation>
+            <configLocation>${checkstyle.config.path}</configLocation>
             <consoleOutput>true</consoleOutput>
             <enableRulesSummary>false</enableRulesSummary>
             <headerLocation>${top.dir}/license-header.txt</headerLocation>
@@ -837,7 +839,7 @@ under the License.
               <artifactId>maven-checkstyle-plugin</artifactId>
               <version>2.10</version>
               <configuration>
-                <configLocation>${top.dir}/checkstyle.xml</configLocation>
+                <configLocation>${checkstyle.config.path}</configLocation>
                 <consoleOutput>true</consoleOutput>
                 <enableRulesSummary>false</enableRulesSummary>
                 <headerLocation>${top.dir}/license-header.txt</headerLocation>
@@ -1713,6 +1715,11 @@ under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.giraph</groupId>
+        <artifactId>giraph-block-app</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.giraph</groupId>
         <artifactId>giraph-examples</artifactId>
         <version>${project.version}</version>
       </dependency>
@@ -2276,6 +2283,7 @@ under the License.
 
   <modules>
     <module>giraph-core</module>
+    <module>giraph-block-app</module>
     <module>giraph-examples</module>
   </modules>
 


[5/5] git commit: updated refs/heads/trunk to 819d6d3

Posted by ik...@apache.org.
[GIRAPH-1013] Add core of Blocks Framework

Summary:
Add all classes in the core of the Framework.

This is the full execution engine of the framework.

New module giraph-block-app is created for it, and all framework
classes are going into org.apache.giraph.block_app.framework,
and all non-framework-internal classes are going to go into
subpackages of org.apache.giraph.block_app (i.e. utilities for
writing applications, etc)

Test Plan: mvn clean install

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

Reviewed By: sergey.edunov

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


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

Branch: refs/heads/trunk
Commit: 819d6d38d6cb7073786f4b7ed11763fff5200ded
Parents: 06a1084
Author: Igor Kabiljo <ik...@fb.com>
Authored: Fri Jun 5 00:19:41 2015 -0700
Committer: Igor Kabiljo <ik...@fb.com>
Committed: Wed Jun 10 19:50:06 2015 -0700

----------------------------------------------------------------------
 checkstyle-relaxed.xml                          | 272 +++++++++++
 giraph-block-app/pom.xml                        | 124 +++++
 giraph-block-app/src/main/assembly/compile.xml  |  39 ++
 .../framework/AbstractBlockFactory.java         | 203 ++++++++
 .../block_app/framework/BlockFactory.java       |  83 ++++
 .../giraph/block_app/framework/BlockUtils.java  | 206 +++++++++
 .../block_app/framework/BulkConfigurator.java   |  32 ++
 .../block_app/framework/api/BlockApi.java       |  47 ++
 .../block_app/framework/api/BlockConfApi.java   |  32 ++
 .../block_app/framework/api/BlockMasterApi.java |  58 +++
 .../block_app/framework/api/BlockOutputApi.java |  31 ++
 .../api/BlockOutputHandleAccessor.java          |  33 ++
 .../block_app/framework/api/BlockWorkerApi.java |  35 ++
 .../framework/api/BlockWorkerContextApi.java    |  45 ++
 .../api/BlockWorkerContextReceiveApi.java       |  29 ++
 .../api/BlockWorkerContextSendApi.java          |  39 ++
 .../framework/api/BlockWorkerReceiveApi.java    |  33 ++
 .../framework/api/BlockWorkerSendApi.java       | 116 +++++
 .../framework/api/BlockWorkerValueAccessor.java |  36 ++
 .../giraph/block_app/framework/api/Counter.java |  33 ++
 .../framework/api/CreateReducersApi.java        |  83 ++++
 .../block_app/framework/api/StatusReporter.java |  31 ++
 .../framework/api/giraph/BlockComputation.java  |  54 +++
 .../api/giraph/BlockMasterApiWrapper.java       | 170 +++++++
 .../api/giraph/BlockMasterCompute.java          |  72 +++
 .../api/giraph/BlockWorkerApiWrapper.java       | 180 +++++++
 .../api/giraph/BlockWorkerContext.java          | 102 ++++
 .../giraph/BlockWorkerContextApiWrapper.java    |  84 ++++
 .../framework/api/giraph/package-info.java      |  22 +
 .../block_app/framework/api/package-info.java   |  36 ++
 .../giraph/block_app/framework/block/Block.java |  59 +++
 .../block_app/framework/block/EmptyBlock.java   |  39 ++
 .../framework/block/FilteringBlock.java         | 113 +++++
 .../block_app/framework/block/IfBlock.java      |  70 +++
 .../block_app/framework/block/RepeatBlock.java  |  87 ++++
 .../framework/block/RepeatUntilBlock.java       |  83 ++++
 .../framework/block/SequenceBlock.java          |  60 +++
 .../block_app/framework/block/package-info.java |  22 +
 .../framework/internal/BlockCounters.java       |  79 ++++
 .../framework/internal/BlockMasterLogic.java    | 173 +++++++
 .../internal/BlockWorkerContextLogic.java       |  91 ++++
 .../framework/internal/BlockWorkerLogic.java    |  68 +++
 .../framework/internal/BlockWorkerPieces.java   | 180 +++++++
 .../framework/internal/PairedPieceAndStage.java | 111 +++++
 .../framework/internal/package-info.java        |  22 +
 .../framework/output/BlockOutputDesc.java       |  50 ++
 .../framework/output/BlockOutputFormat.java     | 107 +++++
 .../framework/output/BlockOutputHandle.java     | 119 +++++
 .../framework/output/BlockOutputOption.java     |  52 +++
 .../framework/output/BlockOutputWriter.java     |  26 ++
 .../framework/output/package-info.java          |  21 +
 .../block_app/framework/package-info.java       |  25 +
 .../framework/piece/AbstractPiece.java          | 287 ++++++++++++
 .../framework/piece/DefaultParentPiece.java     | 311 +++++++++++++
 .../giraph/block_app/framework/piece/Piece.java |  59 +++
 .../framework/piece/PieceWithWorkerContext.java |  54 +++
 .../framework/piece/delegate/DelegatePiece.java | 277 +++++++++++
 .../piece/delegate/FilteringPiece.java          | 157 +++++++
 .../framework/piece/delegate/package-info.java  |  21 +
 .../piece/global_comm/BroadcastHandle.java      |  30 ++
 .../piece/global_comm/ReduceUtilsObject.java    |  62 +++
 .../ReducerAndBroadcastWrapperHandle.java       |  61 +++
 .../piece/global_comm/ReducerHandle.java        |  41 ++
 .../piece/global_comm/array/ArrayHandle.java    |  36 ++
 .../global_comm/array/BroadcastArrayHandle.java |  35 ++
 .../global_comm/array/ReducerArrayHandle.java   |  43 ++
 .../piece/global_comm/array/package-info.java   |  21 +
 .../internal/CreateReducersApiWrapper.java      |  73 +++
 .../internal/ReducersForPieceHandler.java       | 250 ++++++++++
 .../internal/VertexSenderObserver.java          |  28 ++
 .../global_comm/internal/package-info.java      |  22 +
 .../global_comm/map/BroadcastMapHandle.java     |  36 ++
 .../piece/global_comm/map/MapHandle.java        |  31 ++
 .../piece/global_comm/map/ReducerMapHandle.java |  43 ++
 .../piece/global_comm/map/package-info.java     |  21 +
 .../piece/global_comm/package-info.java         |  22 +
 .../piece/interfaces/VertexPostprocessor.java   |  34 ++
 .../piece/interfaces/VertexReceiver.java        |  54 +++
 .../piece/interfaces/VertexSender.java          |  45 ++
 .../piece/interfaces/package-info.java          |  21 +
 .../piece/messages/ObjectMessageClasses.java    | 119 +++++
 .../piece/messages/SupplierFromConf.java        |  71 +++
 .../framework/piece/messages/package-info.java  |  21 +
 .../block_app/framework/piece/package-info.java |  24 +
 .../org/apache/giraph/function/Consumer.java    |  34 ++
 .../org/apache/giraph/function/Function.java    |  38 ++
 .../apache/giraph/function/PairConsumer.java    |  34 ++
 .../apache/giraph/function/PairFunction.java    |  40 ++
 .../org/apache/giraph/function/Supplier.java    |  37 ++
 .../apache/giraph/function/package-info.java    |  26 ++
 .../giraph/function/primitive/IntSupplier.java  |  27 ++
 .../giraph/function/primitive/package-info.java |  22 +
 .../function/vertex/ConsumerWithVertex.java     |  48 ++
 .../function/vertex/FunctionWithVertex.java     |  52 +++
 .../function/vertex/SupplierFromVertex.java     |  49 ++
 .../giraph/function/vertex/package-info.java    |  22 +
 .../framework/block/BlockTestingUtils.java      | 158 +++++++
 .../block_app/framework/block/TestIfBlock.java  |  88 ++++
 .../framework/block/TestRepeatBlock.java        |  80 ++++
 .../framework/block/TestRepeatUntilBlock.java   | 103 +++++
 .../giraph/edge/LongDiffNullArrayEdges.java     | 463 +++++++++++++++++++
 .../giraph/utils/ConfigurationObjectUtils.java  | 118 +++++
 .../giraph/utils/DefaultOutputCommitter.java    |  72 +++
 .../org/apache/giraph/utils/WritableUtils.java  | 187 ++++++++
 .../giraph/edge/LongDiffNullArrayEdgesTest.java | 260 +++++++++++
 pom.xml                                         |  12 +-
 106 files changed, 8395 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/checkstyle-relaxed.xml
----------------------------------------------------------------------
diff --git a/checkstyle-relaxed.xml b/checkstyle-relaxed.xml
new file mode 100644
index 0000000..11bcfe0
--- /dev/null
+++ b/checkstyle-relaxed.xml
@@ -0,0 +1,272 @@
+<?xml version="1.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.
+-->
+
+<!--
+  This version of checkstyle is based on the Hadoop and common-math
+  checkstyle configurations.  It is a best effort attempt to try to match
+  the CODE_CONVENTIONS and Oracle "Code Conventions for the Java
+  Programming Language".  See the following link:
+  
+  http://www.oracle.com/technetwork/java/codeconvtoc-136057.html
+
+  The documentation for checkstyle is available at 
+
+  http://checkstyle.sourceforge.net 
+-->
+
+<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.1//EN" "http://www.puppycrawl.com/dtds/configuration_1_1.dtd">
+
+<!-- Apache giraph customization of default Checkstyle behavior -->
+<module name="Checker">
+  <property name="localeLanguage" value="en"/>
+
+  <!-- Checks for headers -->
+  <!-- See http://checkstyle.sf.net/config_header.html -->
+    <!-- Verify that EVERY source file has the appropriate license -->
+  <module name="Header">
+    <property name="headerFile" value="${checkstyle.header.file}"/>
+    <property name="fileExtensions" value="java"/>	  
+  </module>
+
+  <!-- Checks for Javadoc comments (checker).           -->
+  <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+    <!-- Require package javadoc -->
+  <module name="JavadocPackage"/>
+
+  <!-- Miscellaneous other checks (checker).         -->
+  <!-- See http://checkstyle.sf.net/config_misc.html -->
+    <!-- Require files to end with newline characters -->
+  <module name="NewlineAtEndOfFile"/>
+
+  <!-- Checks for whitespace (tree walker)                 -->
+  <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+    <!-- No tabs allowed! -->
+  <module name="FileTabCharacter"/>
+
+  <module name="TreeWalker">
+    <property name="cacheFile" value="target/checkstyle-cachefile"/>
+
+    <!-- Checks for blocks. You know, those {}'s         -->
+    <!-- See http://checkstyle.sf.net/config_blocks.html -->
+      <!-- No empty blocks (i.e. catch) -->
+    <module name="EmptyBlock"/>
+    <module name="AvoidNestedBlocks"/>
+      <!-- No if/else/do/for/while without braces -->
+    <module name="NeedBraces"/>
+    <module name="LeftCurly"/>
+    <module name="RightCurly"/>
+
+    <!-- Checks for class design                         -->
+    <!-- See http://checkstyle.sf.net/config_design.html -->
+      <!-- Utility class should not be instantiated, they must have a
+	   private constructor -->
+    <module name="HideUtilityClassConstructor"/>
+      <!-- Interfaces must be types (not just constants) -->
+    <module name="InterfaceIsType"/>
+      <!-- No public fields -->
+    <module name="VisibilityModifier">
+       <property name="protectedAllowed" value="true"/>
+       <property name="publicMemberPattern" value="^$"/>
+    </module>
+
+    <!-- Checks for common coding problems               -->
+    <!-- See http://checkstyle.sf.net/config_coding.html -->
+    <module name="EmptyStatement"/>
+      <!-- Require hash code override when equals is -->
+    <module name="EqualsHashCode"/>
+      <!-- Method parameters and local variables should not hide
+	   fields, except in constructors and setters -->
+    <module name="HiddenField">
+      <property name="ignoreConstructorParameter" value="true" />
+      <property name="ignoreSetter" value="true" />
+      <property name="tokens" value="VARIABLE_DEF"/>
+    </module>
+      <!-- Disallow unnecessary instantiation of Boolean, String -->
+    <module name="IllegalInstantiation">
+      <property name="classes" value="java.lang.Boolean, java.lang.String"/>
+    </module>
+    <module name="InnerAssignment"/>
+      <!-- Switch statements should be complete and with independent cases -->
+    <module name="FallThrough" />
+    <module name="MissingSwitchDefault" />
+    <module name="SimplifyBooleanExpression"/>
+    <module name="SimplifyBooleanReturn"/>
+      <!-- Only one statment per line allowed -->
+    <module name="OneStatementPerLine"/>
+      <!-- Use a consistent way to put declarations -->
+    <module name="DeclarationOrder" />
+      <!-- Don't add up parentheses when they are not required -->
+    <module name="UnnecessaryParentheses" />
+      <!-- Don't use too widespread catch (Exception, Throwable,
+	   RuntimeException) --> 
+    <module name="IllegalCatch" />
+      <!-- Don't use = or != for string comparisons -->
+    <module name="StringLiteralEquality" />
+      <!-- Don't declare multiple variables in the same statement -->
+    <module name="MultipleVariableDeclarations" />
+      <!-- String literals more than one character long should not be 
+	   repeated several times -->
+      <!-- the "unchecked" string is also accepted to allow
+	   @SuppressWarnings("unchecked") -->
+      <!-- Disabling for now until we have a better ignoreStringsRegexp -->
+      <!--
+    <module name="MultipleStringLiterals" >
+      <property name="ignoreStringsRegexp" value='^(("")|(".")|("unchecked"))$'/>
+    </module>
+      -->
+
+    <!-- Checks for imports                              -->
+    <!-- See http://checkstyle.sf.net/config_import.html -->
+    <module name="RedundantImport"/>
+      <!-- Import should be explicit, really needed and only from pure
+	   java packages --> 
+    <module name="AvoidStarImport" />
+    <module name="UnusedImports" />
+    <module name="IllegalImport" />
+
+    <!-- Checks for Javadoc comments (tree walker).       -->
+    <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+      <!-- Javadoc must be formatted correctly -->
+    <module name="JavadocStyle">
+      <property name="checkFirstSentence" value="false"/>
+    </module>
+      <!-- Must have class / interface header comments -->
+    <module name="JavadocType"/>
+      <!-- Require method javadocs, allow undeclared RTE, allow missing 
+	   javadoc on getters and setters  -->
+    <module name="JavadocMethod">
+      <property name="allowMissingJavadoc" value="true"/>
+      <property name="allowUndeclaredRTE" value="true"/>
+      <property name="allowMissingThrowsTags" value="true"/> 
+      <property name="allowMissingPropertyJavadoc" value="true"/>
+      <property name="allowMissingParamTags" value="true"/>
+      <property name="allowMissingReturnTag" value="true"/> 
+    </module>
+
+    <!-- Miscellaneous other checks (tree walker).     -->
+    <!-- See http://checkstyle.sf.net/config_misc.html -->
+      <!-- Java style arrays -->
+    <module name="ArrayTypeStyle"/>
+      <!-- Indentation -->
+    <module name="Indentation">
+      <property name="basicOffset" value="2"/>
+      <property name="braceAdjustment" value="0"/>
+      <property name="caseIndent" value="0"/>
+      <property name="throwsIndent" value="2"/>
+      <property name="lineWrappingIndentation" value="0"/>
+      <property name="arrayInitIndent" value="2"/>     
+    </module>
+      <!-- Turn this on to see what needs to be done
+    <module name="TodoComment"/>
+       -->
+    <module name="UpperEll"/>
+
+    <!-- Modifier Checks                                    -->
+    <!-- See http://checkstyle.sf.net/config_modifiers.html -->
+      <!-- Use a consistent way to put modifiers -->
+    <module name="ModifierOrder"/>
+    <module name="RedundantModifier"/>
+
+    <!-- Checks for Naming Conventions.                  -->
+    <!-- See http://checkstyle.sf.net/config_naming.html -->
+      <!-- Constant names should obey the traditional all uppercase 
+	   naming convention -->
+    <module name="ConstantName"/>
+    <module name="LocalFinalVariableName"/>
+    <module name="LocalVariableName"/>
+    <module name="MemberName"/>
+    <module name="MethodName"/>
+    <module name="PackageName"/>
+    <module name="ParameterName"/>
+    <module name="StaticVariableName">
+      <property name="format" value="^[A-Z0-9_]*$"/>
+    </module>
+    <module name="TypeName"/>
+
+    <!-- Checks for regexp expressions.                  -->
+    <!-- See http://checkstyle.sf.net/config_regexp.html -->    
+      <!-- No trailing whitespace -->
+    <module name="Regexp">
+      <property name="format" value="[ \t]+$"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="message" value="Trailing whitespace"/>
+    </module> 
+      <!-- No System.out.println() statements -->
+    <module name="Regexp">
+      <!-- No sysouts -->
+      <property name="format" value="System\.out\.println"/>
+      <property name="illegalPattern" value="true"/>
+    </module>
+      <!-- Authors should be in pom.xml file -->
+    <module name="Regexp">
+      <property name="format" value="@author"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="message" value="developers names should be in pom file"/>
+    </module>
+
+    <!-- Checks for Size Violations.                    -->
+    <!-- See http://checkstyle.sf.net/config_sizes.html -->
+      <!-- Lines cannot exceed 80 chars -->
+    <module name="LineLength">
+      <property name="max" value="80"/>
+      <property name="ignorePattern" value="^import"/>
+    </module>
+      <!-- Over time, we will revised this down -->
+    <module name="MethodLength">
+      <property name="max" value="200"/>
+    </module>
+    <module name="ParameterNumber">
+      <property name="max" value="8"/>
+    </module>
+
+    <!-- Checks for whitespace (tree walker)                 -->
+    <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+    <module name="EmptyForIteratorPad"/>
+      <!-- Spacing around methods -->
+    <module name="MethodParamPad">
+      <property name="option" value="nospace"/>
+      <property name="allowLineBreaks" value="true"/>
+     </module>
+      <!-- No whitespace before a token -->
+    <module name="NoWhitespaceBefore"/>
+      <!-- Whitespace after tokens is required -->
+    <module name="WhitespaceAfter"/>
+      <!-- Whitespace around tokens is required -->
+    <module name="WhitespaceAround"/>
+    <module name="ParenPad"/>
+    <module name="TypecastParenPad"/>
+      <!-- No extra whitespace around types -->
+    <module name="GenericWhitespace"/>
+    <!-- Operator must be at end of wrapped line -->
+    <module name="OperatorWrap">
+      <property name="option" value="eol"/>
+    </module>
+
+    <!-- Required for SuppressionCommentFilter below -->
+    <module name="FileContentsHolder"/>
+  </module>
+  
+  <!-- Setup special comments to suppress specific checks from source files -->
+  <module name="SuppressionCommentFilter">
+    <property name="offCommentFormat" value="CHECKSTYLE\: stop ([\w\|]+)"/>
+    <property name="onCommentFormat"  value="CHECKSTYLE\: resume ([\w\|]+)"/>
+    <property name="checkFormat"      value="$1"/>
+  </module>
+</module>
+

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-block-app/pom.xml b/giraph-block-app/pom.xml
new file mode 100644
index 0000000..1f653bb
--- /dev/null
+++ b/giraph-block-app/pom.xml
@@ -0,0 +1,124 @@
+<!--
+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>
+
+  <parent>
+    <groupId>org.apache.giraph</groupId>
+    <artifactId>giraph-parent</artifactId>
+    <version>1.2.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>giraph-block-app</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Apache Giraph Blocks Framework</name>
+  <url>http://giraph.apache.org/giraph-block-app/</url>
+  <description>Giraph Blocks Framework and utilities for writing applications</description>
+
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+    <checkstyle.config.path>${top.dir}/checkstyle-relaxed.xml</checkstyle.config.path>
+  </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>
+          <skip>${surefire.skip}</skip>
+          <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>it.unimi.dsi</groupId>
+      <artifactId>fastutil</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.giraph</groupId>
+      <artifactId>giraph-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.python</groupId>
+      <artifactId>jython</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </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>
+</project>

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/assembly/compile.xml
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/assembly/compile.xml b/giraph-block-app/src/main/assembly/compile.xml
new file mode 100644
index 0000000..fcaffa6
--- /dev/null
+++ b/giraph-block-app/src/main/assembly/compile.xml
@@ -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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>jar-with-dependencies</id>
+   <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <dependencySets>
+    <dependencySet>
+      <useProjectArtifact>true</useProjectArtifact>
+      <outputDirectory>/</outputDirectory>
+      <unpackOptions>
+          <excludes>
+              <exclude>META-INF/LICENSE</exclude>
+          </excludes>
+      </unpackOptions>
+      <unpack>true</unpack>
+      <scope>runtime</scope>
+    </dependencySet>
+  </dependencySets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java
new file mode 100644
index 0000000..66ad775
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java
@@ -0,0 +1,203 @@
+/*
+ * 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.block_app.framework;
+
+import java.util.List;
+
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.StrConfOption;
+import org.apache.giraph.edge.IdAndNullArrayEdges;
+import org.apache.giraph.edge.IdAndValueArrayEdges;
+import org.apache.giraph.edge.LongDiffNullArrayEdges;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.TypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Default block factory abstract class, providing default methods that need
+ * to be/can be overridden for specifying required/most common parameters,
+ * to simplify setting properties.
+ *
+ * @param <S> Execution stage type
+ */
+public abstract class AbstractBlockFactory<S> implements BlockFactory<S> {
+  /**
+   * Comma separated list of BulkConfigurators, that are going to be called
+   * to simplify specifying of large number of properties.
+   */
+  public static final StrConfOption CONFIGURATORS = new StrConfOption(
+      "digraph.block_factory_configurators", null, "");
+
+  @Override
+  public List<String> getGcJavaOpts(Configuration conf) {
+    return null;
+  }
+
+  @Override
+  public final void initConfig(GiraphConfiguration conf) {
+    initConfigurators(conf);
+    GiraphConstants.VERTEX_ID_CLASS.setIfUnset(conf, getVertexIDClass(conf));
+    GiraphConstants.VERTEX_VALUE_CLASS.setIfUnset(
+        conf, getVertexValueClass(conf));
+    GiraphConstants.EDGE_VALUE_CLASS.setIfUnset(conf, getEdgeValueClass(conf));
+    GiraphConstants.RESOLVER_CREATE_VERTEX_ON_MSGS.setIfUnset(
+        conf, shouldCreateVertexOnMsgs(conf));
+    if (shouldSendOneMessageToAll(conf)) {
+      GiraphConstants.MESSAGE_ENCODE_AND_STORE_TYPE.setIfUnset(
+          conf, MessageEncodeAndStoreType.EXTRACT_BYTEARRAY_PER_PARTITION);
+    }
+
+    BlockUtils.BLOCK_WORKER_CONTEXT_VALUE_CLASS.setIfUnset(
+        conf, getWorkerContextValueClass(conf));
+
+    // optimize edge structure, if available and not set already
+    if (!GiraphConstants.VERTEX_EDGES_CLASS.contains(conf)) {
+      @SuppressWarnings("rawtypes")
+      Class<? extends WritableComparable> vertexIDClass =
+          GiraphConstants.VERTEX_ID_CLASS.get(conf);
+      Class<? extends Writable> edgeValueClass =
+          GiraphConstants.EDGE_VALUE_CLASS.get(conf);
+
+
+      @SuppressWarnings("rawtypes")
+      PrimitiveIdTypeOps<? extends WritableComparable> idTypeOps =
+          TypeOpsUtils.getPrimitiveIdTypeOpsOrNull(vertexIDClass);
+      if (edgeValueClass.equals(NullWritable.class)) {
+        if (vertexIDClass.equals(LongWritable.class)) {
+          GiraphConstants.VERTEX_EDGES_CLASS.set(
+              conf, LongDiffNullArrayEdges.class);
+        } else if (idTypeOps != null) {
+          GiraphConstants.VERTEX_EDGES_CLASS.set(
+              conf, IdAndNullArrayEdges.class);
+        }
+      } else {
+        TypeOps<?> edgeValueTypeOps =
+            TypeOpsUtils.getTypeOpsOrNull(edgeValueClass);
+        if (edgeValueTypeOps != null) {
+          GiraphConstants.VERTEX_EDGES_CLASS.set(
+              conf, IdAndValueArrayEdges.class);
+        }
+      }
+    }
+
+    additionalInitConfig(conf);
+  }
+
+  @Override
+  public void registerOutputs(GiraphConfiguration conf) {
+  }
+
+  private void initConfigurators(GiraphConfiguration conf) {
+    String configurators = CONFIGURATORS.get(conf);
+    if (configurators != null) {
+      String[] split = configurators.split(",");
+      for (String configurator : split) {
+        runConfigurator(conf, configurator);
+      }
+    }
+  }
+
+  private void runConfigurator(GiraphConfiguration conf, String configurator) {
+    String[] packages = getConvenienceConfiguratorPackages();
+    String[] prefixes = new String[packages.length + 1];
+    prefixes[0] = "";
+    for (int i = 0; i < packages.length; i++) {
+      prefixes[i + 1] = packages[i] + ".";
+    }
+
+    for (String prefix : prefixes) {
+      try {
+        @SuppressWarnings({ "unchecked", "rawtypes" })
+        Class<BulkConfigurator> confClass =
+            (Class) Class.forName(prefix + configurator);
+        BulkConfigurator c = ReflectionUtils.newInstance(confClass);
+        c.configure(conf);
+        return;
+      // CHECKSTYLE: stop EmptyBlock
+      // ignore ClassNotFoundException, and continue the loop
+      } catch (ClassNotFoundException e) {
+      }
+      // CHECKSTYLE: resume EmptyBlock
+    }
+    throw new IllegalStateException(
+        "Configurator " + configurator + " not found");
+  }
+
+  /**
+   * Additional configuration initialization, other then overriding
+   * class specification.
+   */
+  protected void additionalInitConfig(GiraphConfiguration conf) {
+  }
+
+  /**
+   * Concrete vertex id class application will use.
+   */
+  @SuppressWarnings("rawtypes")
+  protected abstract Class<? extends WritableComparable> getVertexIDClass(
+      GiraphConfiguration conf);
+
+  /**
+   * Concrete vertex value class application will use.
+   */
+  protected abstract Class<? extends Writable> getVertexValueClass(
+      GiraphConfiguration conf);
+
+  /**
+   * Concrete edge value class application will use.
+   */
+  protected abstract Class<? extends Writable> getEdgeValueClass(
+      GiraphConfiguration conf);
+
+  /**
+   * Concrete worker context value class application will use, if overridden.
+   */
+  protected Class<?> getWorkerContextValueClass(GiraphConfiguration conf) {
+    return Object.class;
+  }
+
+  /**
+   * Override if vertices shouldn't be created by default, if message is sent
+   * to a vertex that doesn't exist.
+   */
+  protected boolean shouldCreateVertexOnMsgs(GiraphConfiguration conf) {
+    return true;
+  }
+
+  // TODO - see if it should be deprecated
+  protected boolean shouldSendOneMessageToAll(GiraphConfiguration conf) {
+    return false;
+  }
+
+  /**
+   * Provide list of strings representing packages where configurators will
+   * be searched for, allowing that full path is not required for
+   * CONFIGURATORS option.
+   */
+  protected String[] getConvenienceConfiguratorPackages() {
+    return new String[] { };
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java
new file mode 100644
index 0000000..652eb5e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.block_app.framework;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Class describing a particular application.
+ * Everything except input and output should be fully encapsulated within
+ * this class. For any application, it should be enough to only specify
+ * particular BlockFactory.
+ *
+ * Given configuration, it creates a block that represents a full Giraph job.
+ *
+ * Recommended is to extend AbstractBlockFactory directly for most cases.
+ *
+ * @param <S> Execution stage type
+ */
+public interface BlockFactory<S> {
+  /**
+   * Based on provided configuration, updates it, such that all necessary
+   * properties are initialized.
+   */
+  void initConfig(GiraphConfiguration conf);
+
+  /**
+   * Create a block (representing a full Giraph job), based on the given
+   * configuration. Configuration should be treated as immutable at this point.
+   *
+   * If there are issues in configuration, it is very cheap to throw
+   * from this method - as Giraph job will not even start.
+   * This function will be called two times - once before starting
+   * of the Giraph job, to fail early if anything is incorrectly configured.
+   * Second time will be on Master, which will return Block instance
+   * on which createIterator will be called once, which should return
+   * current application run.
+   * initConfig will be called only once, before starting Giraph job itself.
+   * Master will contain configuration already modified by initConfig.
+   */
+  Block createBlock(GiraphConfiguration conf);
+
+  /**
+   * Create an empty instance of execution stage object.
+   *
+   * Can be used by application to be aware of what was executed before.
+   * Most common example is counting iterations, or for having a boolean whether
+   * some important event happened.
+   *
+   * Execution stage should be immutable object, with creating a new
+   * object when different value is needed.
+   */
+  S createExecutionStage(GiraphConfiguration conf);
+
+  /**
+   * Get special GC Java options. If returns null, default options are used.
+   */
+  List<String> getGcJavaOpts(Configuration conf);
+
+  /**
+   * Register outputs to use during the application (vs output at the end of
+   * the application), based on provided configuration.
+   */
+  void registerOutputs(GiraphConfiguration conf);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java
new file mode 100644
index 0000000..df260f5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.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.block_app.framework;
+
+import static org.apache.giraph.utils.ReflectionUtils.getTypeArguments;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+
+import org.apache.giraph.block_app.framework.api.giraph.BlockComputation;
+import org.apache.giraph.block_app.framework.api.giraph.BlockMasterCompute;
+import org.apache.giraph.block_app.framework.api.giraph.BlockWorkerContext;
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.apache.giraph.conf.ClassConfOption;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.types.NoMessage;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Utility functions for block applications
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BlockUtils {
+  /** Property describing BlockFactory to use for current application run */
+  public static final ClassConfOption<BlockFactory> BLOCK_FACTORY_CLASS =
+      ClassConfOption.create("digraph.block_factory", null, BlockFactory.class,
+          "block factory describing giraph job");
+
+  /** Property describing BlockFactory to use for current application run */
+  public static final ClassConfOption<Object> BLOCK_WORKER_CONTEXT_VALUE_CLASS =
+      ClassConfOption.create(
+          "digraph.block_worker_context_value_class", null, Object.class,
+          "block worker context value class");
+
+  private static final Logger LOG = Logger.getLogger(BlockUtils.class);
+
+  /** Dissallow constructor */
+  private BlockUtils() { }
+
+  /**
+   * Create new BlockFactory that is specified in the configuration.
+   */
+  public static <S> BlockFactory<S> createBlockFactory(Configuration conf) {
+    return ReflectionUtils.newInstance(BLOCK_FACTORY_CLASS.get(conf));
+  }
+
+  /**
+   * Set which BlockFactory class to be used for the application.
+   * (generally useful within tests only)
+   */
+  public static void setBlockFactoryClass(Configuration conf,
+      Class<? extends BlockFactory<?>> clazz) {
+    BLOCK_FACTORY_CLASS.set(conf, clazz);
+  }
+
+  /**
+   * Set block factory, and initialize configs with it.
+   * Should be used only if there are no configuration options set after
+   * this method call.
+   */
+  public static void setAndInitBlockFactoryClass(GiraphConfiguration conf,
+      Class<? extends BlockFactory<?>> clazz) {
+    BLOCK_FACTORY_CLASS.set(conf, clazz);
+    initAndCheckConfig(conf);
+  }
+
+  /**
+   * Initializes configuration, such that running it executes block application.
+   *
+   * Additionally, checks types of all pieces with a block application.
+   */
+  public static void initAndCheckConfig(GiraphConfiguration conf) {
+    conf.setMasterComputeClass(BlockMasterCompute.class);
+    conf.setComputationClass(BlockComputation.class);
+    conf.setWorkerContextClass(BlockWorkerContext.class);
+
+    Preconditions.checkState(
+        GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.get(conf) == null,
+        "Message types should only be specified in Pieces, " +
+        "but outgoing was specified globally");
+    Preconditions.checkState(
+        GiraphConstants.OUTGOING_MESSAGE_VALUE_FACTORY_CLASS
+          .isDefaultValue(conf),
+        "Message types should only be specified in Pieces, " +
+        "but factory was specified globally");
+    Preconditions.checkState(
+        GiraphConstants.MESSAGE_COMBINER_CLASS.get(conf) == null,
+        "Message combiner should only be specified in Pieces, " +
+        "but was specified globally");
+
+    BlockFactory<?> blockFactory = createBlockFactory(conf);
+    blockFactory.initConfig(conf);
+
+    Preconditions.checkState(
+        GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.get(conf) == null,
+        "Outgoing message type was specified in blockFactory.initConfig");
+    Preconditions.checkState(
+        GiraphConstants.OUTGOING_MESSAGE_VALUE_FACTORY_CLASS
+          .isDefaultValue(conf),
+        "Outgoing message factory type was specified in " +
+        "blockFactory.initConfig");
+    Preconditions.checkState(
+        GiraphConstants.MESSAGE_COMBINER_CLASS.get(conf) == null,
+        "Message combiner type was specified in blockFactory.initConfig");
+
+    GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.set(conf, NoMessage.class);
+
+    final ImmutableClassesGiraphConfiguration immConf =
+        new ImmutableClassesGiraphConfiguration<>(conf);
+
+    // Create blocks to detect issues before creating a Giraph job
+    // They will not be used here
+    Block executionBlock = blockFactory.createBlock(immConf);
+    LOG.info("Executing application - " + executionBlock);
+
+    final Class<?> vertexIdClass = GiraphConstants.VERTEX_ID_CLASS.get(conf);
+    final Class<?> vertexValueClass =
+        GiraphConstants.VERTEX_VALUE_CLASS.get(conf);
+    final Class<?> edgeValueClass =
+        GiraphConstants.EDGE_VALUE_CLASS.get(conf);
+    final Class<?> workerContextValueClass =
+        BlockUtils.BLOCK_WORKER_CONTEXT_VALUE_CLASS.get(conf);
+    final Class<?> executionStageClass =
+        blockFactory.createExecutionStage(conf).getClass();
+
+    // Check for type inconsistencies
+    executionBlock.forAllPossiblePieces(new Consumer<AbstractPiece>() {
+      @Override
+      public void apply(AbstractPiece piece) {
+        if (!piece.getClass().equals(Piece.class)) {
+          Class<?>[] classList = getTypeArguments(
+              AbstractPiece.class, piece.getClass());
+          Preconditions.checkArgument(classList.length == 7);
+
+          ReflectionUtils.verifyTypes(
+              vertexIdClass, classList[0], "vertexId", piece.getClass());
+          ReflectionUtils.verifyTypes(
+              vertexValueClass, classList[1], "vertexValue", piece.getClass());
+          ReflectionUtils.verifyTypes(
+              edgeValueClass, classList[2], "edgeValue", piece.getClass());
+
+          MessageClasses classes = piece.getMessageClasses(immConf);
+          Class<?> messageType = classes.getMessageClass();
+          if (messageType == null) {
+            messageType = NoMessage.class;
+          }
+          ReflectionUtils.verifyTypes(
+              messageType, classList[3], "message", piece.getClass());
+
+          ReflectionUtils.verifyTypes(
+              workerContextValueClass, classList[4],
+              "workerContextValue", piece.getClass());
+          // No need to check worker context message class at all
+
+          ReflectionUtils.verifyTypes(
+              executionStageClass, classList[6],
+              "executionStage", piece.getClass());
+        }
+      }
+    });
+
+    // check for non 'static final' fields in BlockFactories
+    Class<?> bfClass = blockFactory.getClass();
+    while (!bfClass.equals(Object.class)) {
+      for (Field field : bfClass.getDeclaredFields()) {
+        if (!Modifier.isStatic(field.getModifiers()) ||
+            !Modifier.isFinal(field.getModifiers())) {
+          throw new IllegalStateException("BlockFactory (" + bfClass +
+              ") cannot have any mutable (non 'static final') fields as a " +
+              "safety measure, as createBlock function is called from a " +
+              "different context then all other functions, use conf argument " +
+              "instead, or make it 'static final'. Field present: " + field);
+        }
+      }
+      bfClass = bfClass.getSuperclass();
+    }
+
+    // Register outputs
+    blockFactory.registerOutputs(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java
new file mode 100644
index 0000000..348c907
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.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.block_app.framework;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+
+/**
+ * Function that modifies configuration.
+ *
+ * Allows for multi-option configuration to be specified in a common classes.
+ */
+public interface BulkConfigurator {
+  /**
+   * Modify given configuration.
+   */
+  void configure(GiraphConfiguration conf);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java
new file mode 100644
index 0000000..a0c92ad
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.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.block_app.framework.api;
+
+
+/**
+ * Basic block computation API for accessing items
+ * present on both workers and master.
+ */
+public interface BlockApi extends BlockConfApi {
+  /**
+   * Get the total (all workers) number of vertices that
+   * existed at the start of the current piece.
+   *
+   * Recommended to avoid it, as it introduces global dependencies,
+   * code will not be able to work on top of a subgraphs any more.
+   * This number should be easily computable via reducer or aggregator.
+   */
+  @Deprecated
+  long getTotalNumVertices();
+
+  /**
+   * Get the total (all workers) number of edges that
+   * existed at the start of the current piece.
+   *
+   * Recommended to avoid it, as it introduces global dependencies,
+   * code will not be able to work on top of a subgraphs any more.
+   * This number should be easily computable via reducer or aggregator.
+   */
+  @Deprecated
+  long getTotalNumEdges();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java
new file mode 100644
index 0000000..98e51f8
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.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.block_app.framework.api;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+
+/**
+ * Block computation API for accessing configuration.
+ */
+public interface BlockConfApi {
+  /**
+   * Return the configuration used by this object.
+   *
+   * @return Set configuration
+   */
+  ImmutableClassesGiraphConfiguration<?, ?, ?> getConf();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java
new file mode 100644
index 0000000..3d04584
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java
@@ -0,0 +1,58 @@
+/*
+ * 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.block_app.framework.api;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.master.MasterAggregatorUsage;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Block computation API available for the master methods.
+ *
+ * Interface to the MasterCompute methods.
+ */
+public interface BlockMasterApi extends MasterAggregatorUsage,
+    MasterGlobalCommUsage, StatusReporter, BlockApi, BlockOutputApi {
+  /**
+   * No need to use it, and introduces global dependencies.
+   *
+   * Store data locally within the piece, or use ObjectHolder.
+   */
+  @Deprecated
+  @Override
+  <A extends Writable>
+  boolean registerPersistentAggregator(
+      String name, Class<? extends Aggregator<A>> aggregatorClass
+  ) throws InstantiationException, IllegalAccessException;
+
+  /**
+   * Broadcast given value to all workers for next computation.
+   * @param value Value to broadcast
+   */
+  <T extends Writable> BroadcastHandle<T> broadcast(T 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
+   */
+  void logToCommandLine(String line);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java
new file mode 100644
index 0000000..8c33623
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.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.block_app.framework.api;
+
+import org.apache.giraph.block_app.framework.output.BlockOutputDesc;
+import org.apache.giraph.block_app.framework.output.BlockOutputWriter;
+
+/**
+ * Block worker output api
+ */
+public interface BlockOutputApi {
+  <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+  OD getOutputDesc(String confOption);
+
+  <OW extends BlockOutputWriter> OW getWriter(String confOption);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java
new file mode 100644
index 0000000..074637d
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.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.block_app.framework.api;
+
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+
+/**
+ * Function for accessing BlockOutputHandle.
+ *
+ * Intentionally hidden from APIs, to allow usage only
+ * within DefaultParentPiece.
+ */
+public interface BlockOutputHandleAccessor {
+  /**
+   * Get global block output handle.
+   */
+  BlockOutputHandle getBlockOutputHandle();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java
new file mode 100644
index 0000000..727bf08
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java
@@ -0,0 +1,35 @@
+/*
+ * 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.block_app.framework.api;
+
+import org.apache.giraph.aggregators.AggregatorUsage;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Block computation API available for worker methods.
+ *
+ * Interface to the Computation methods.
+ * @param <I> vertex Id type.
+ */
+@SuppressWarnings("rawtypes")
+public interface BlockWorkerApi<I extends WritableComparable>
+    extends AggregatorUsage, BlockApi {
+  @Override
+  ImmutableClassesGiraphConfiguration<I, ?, ?> getConf();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java
new file mode 100644
index 0000000..d5918b5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java
@@ -0,0 +1,45 @@
+/*
+ * 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.block_app.framework.api;
+
+import org.apache.giraph.aggregators.AggregatorUsage;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+
+/**
+ * Block computation API available for worker context methods.
+ *
+ * Interface to the WorkerContext methods.
+ */
+public interface BlockWorkerContextApi extends AggregatorUsage, BlockApi {
+  @Override
+  ImmutableClassesGiraphConfiguration<?, ?, ?> getConf();
+
+  /**
+   * Get number of workers
+   *
+   * @return Number of workers
+   */
+  int getWorkerCount();
+
+  /**
+   * Get index for this worker
+   *
+   * @return Index of this worker
+   */
+  int getMyWorkerIndex();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java
new file mode 100644
index 0000000..a8242b2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java
@@ -0,0 +1,29 @@
+/*
+ * 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.block_app.framework.api;
+
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Block computation API available for worker context receive methods.
+ *
+ * Interface to the WorkerContext methods.
+ */
+public interface BlockWorkerContextReceiveApi
+    extends BlockWorkerContextApi, WorkerBroadcastUsage {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java
new file mode 100644
index 0000000..769562d
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.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.block_app.framework.api;
+
+import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Block computation API available for worker send methods.
+ *
+ * Interface to the WorkerContext methods.
+ *
+ * @param <WM> Worker message type
+ */
+public interface BlockWorkerContextSendApi<WM extends Writable>
+    extends BlockWorkerContextApi, WorkerAggregatorUsage {
+  /**
+   * Send message to another worker
+   *
+   * @param message Message to send
+   * @param workerIndex Index of the worker to send the message to
+   */
+  void sendMessageToWorker(WM message, int workerIndex);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java
new file mode 100644
index 0000000..6db51bd
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.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.block_app.framework.api;
+
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Block computation API available for worker receive methods.
+ *
+ * Interface to the Computation methods.
+ *
+ * @param <I> vertex Id type.
+ */
+@SuppressWarnings("rawtypes")
+public interface BlockWorkerReceiveApi<I extends WritableComparable>
+    extends BlockWorkerApi<I>, WorkerBroadcastUsage, BlockOutputApi {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java
new file mode 100644
index 0000000..a72150f
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java
@@ -0,0 +1,116 @@
+/*
+ * 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.block_app.framework.api;
+
+import java.util.Iterator;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerReduceUsage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Block computation API available for worker send methods.
+ *
+ * Interface to the Computation methods.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ */
+@SuppressWarnings("rawtypes")
+public interface BlockWorkerSendApi<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable>
+    extends BlockWorkerApi<I>, WorkerAggregatorUsage, WorkerReduceUsage {
+  @Override
+  ImmutableClassesGiraphConfiguration<I, V, E> getConf();
+
+  /**
+   * Send a message to a vertex id.
+   *
+   * @param id Vertex id to send the message to
+   * @param message Message data to send
+   */
+  void sendMessage(I id, M message);
+
+  /**
+   * Send a message to all edges.
+   *
+   * @param vertex Vertex whose edges to send the message to.
+   * @param message Message sent to all edges.
+   */
+  void sendMessageToAllEdges(Vertex<I, V, E> vertex, M message);
+
+  /**
+   * Send a message to multiple target vertex ids in the iterator.
+   *
+   * @param vertexIdIterator An iterator to multiple target vertex ids.
+   * @param message Message sent to all targets in the iterator.
+   */
+  void sendMessageToMultipleEdges(Iterator<I> vertexIdIterator, M message);
+
+  /**
+   * Sends a request to create a vertex that will be available
+   * in the receive phase.
+   *
+   * @param id Vertex id
+   * @param value Vertex value
+   * @param edges Initial edges
+   */
+  void addVertexRequest(I id, V value, OutEdges<I, E> edges);
+
+  /**
+   * Sends a request to create a vertex that will be available
+   * in the receive phase.
+   *
+   * @param id Vertex id
+   * @param value Vertex value
+   */
+  void addVertexRequest(I id, V value);
+
+  /**
+   * Request to remove a vertex from the graph
+   * (applied just prior to the next receive phase).
+   *
+   * @param vertexId Id of the vertex to be removed.
+   */
+  void removeVertexRequest(I vertexId);
+
+  /**
+   * Request to add an edge of a vertex in the graph
+   * (processed just prior to the next receive phase)
+   *
+   * @param sourceVertexId Source vertex id of edge
+   * @param edge Edge to add
+   */
+  void addEdgeRequest(I sourceVertexId, Edge<I, E> edge);
+
+  /**
+   * Request to remove all edges from a given source vertex to a given target
+   * vertex (processed just prior to the next receive phase).
+   *
+   * @param sourceVertexId Source vertex id
+   * @param targetVertexId Target vertex id
+   */
+  void removeEdgesRequest(I sourceVertexId, I targetVertexId);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java
new file mode 100644
index 0000000..f7cddf3
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java
@@ -0,0 +1,36 @@
+/*
+ * 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.block_app.framework.api;
+
+/**
+ * Function for accessing WorkerValue.
+ *
+ * Intentionally hidden from APIs, to allow usage only
+ * within PieceWithWorkerContext.
+ */
+public interface BlockWorkerValueAccessor {
+  /**
+   * Get global worker value.
+   * Value returned can be accessed from may threads, and so all
+   * accesses to it should be done in a thread-safe manner!
+   *
+   * This is the only place in Blocks Framework where you need
+   * to take care of concurrency.
+   */
+  Object getWorkerValue();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java
new file mode 100644
index 0000000..730b0ab
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.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.block_app.framework.api;
+
+/** Interface wrapping around org.appache.hadoop.mapreduce.Counter */
+public interface Counter {
+  /**
+   * Set this counter by the given value
+   * @param value the value to set
+   */
+  void setValue(long value);
+
+  /**
+   * Increment this counter by the given value
+   * @param incr the value to increase this counter by
+   */
+  void increment(long incr);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java
new file mode 100644
index 0000000..77b3b1e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java
@@ -0,0 +1,83 @@
+/*
+ * 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.block_app.framework.api;
+
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Api for creating reducer handles.
+ */
+public interface CreateReducersApi extends BlockConfApi {
+
+  /**
+   * Create local reducer, returning a handle to it.
+   *
+   * Local reducer means that each worker thread has it's own local partially
+   * reduced value, which are at the end reduced all together.
+   * Preferable, unless it cannot be used, because all copies of the object
+   * do not fit the memory.
+   */
+  <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+      ReduceOperation<S, R> reduceOp);
+
+  /**
+   * Create local reducer, returning a handle to it.
+   *
+   * Local reducer means that each worker thread has it's own local partially
+   *  reduced value, which are at the end reduced all together.
+   * Preferable, unless it cannot be used, because all copies of the object
+   * do not fit the memory.
+   */
+  <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+      ReduceOperation<S, R> reduceOp, R globalInitialValue);
+
+  /**
+   * Create global reducer, returning a handle to it.
+   *
+   * Global reducer means that there is only one value for each worker,
+   * and each call to reduce will have to obtain a global lock, and incur
+   * synchronization costs.
+   * Use only when objects are so large, that having many copies cannot
+   * fit into memory.
+   */
+  <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+      ReduceOperation<S, R> reduceOp);
+
+  /**
+   * Create global reducer, returning a handle to it.
+   *
+   * Global reducer means that there is only one value for each worker,
+   * and each call to reduce will have to obtain a global lock, and incur
+   * synchronization costs.
+   * Use only when objects are so large, that having many copies cannot
+   * fit into memory.
+   */
+  <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+      ReduceOperation<S, R> reduceOp, R globalInitialValue);
+
+  /**
+   * Function that creates a reducer - abstracting away whether it is
+   * local or global reducer
+   */
+  public interface CreateReducerFunctionApi {
+    <S, R extends Writable> ReducerHandle<S, R> createReducer(
+        ReduceOperation<S, R> reduceOp);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java
new file mode 100644
index 0000000..3ce1862
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.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.block_app.framework.api;
+
+/**
+ * Interface wrapping around functions from
+ * org.appache.hadoop.mapreduce.StatusReporter
+ */
+public interface StatusReporter {
+  /** Get specified counter handler */
+  Counter getCounter(String group, String name);
+  /** Report progress to the Hadoop framework. */
+  void progress();
+  /** Set the current status of the task to the given string. */
+  void setStatus(String status);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java
new file mode 100644
index 0000000..930b62b
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.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.block_app.framework.api.giraph;
+
+import java.io.IOException;
+
+import org.apache.giraph.block_app.framework.internal.BlockWorkerLogic;
+import org.apache.giraph.block_app.framework.internal.BlockWorkerPieces;
+import org.apache.giraph.graph.AbstractComputation;
+import org.apache.giraph.graph.Vertex;
+
+
+/**
+ * Computation that executes receiver and sender blocks passed
+ * into BlockWorkerPieces.
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public final class BlockComputation extends AbstractComputation {
+  private BlockWorkerLogic workerLogic;
+
+  @Override
+  public void preSuperstep() {
+    BlockWorkerPieces workerPieces =
+        BlockWorkerPieces.getNextWorkerPieces(this);
+    workerLogic = new BlockWorkerLogic(workerPieces);
+    BlockWorkerApiWrapper workerApi = new BlockWorkerApiWrapper<>(this);
+    workerLogic.preSuperstep(workerApi, workerApi);
+  }
+
+  @Override
+  public void compute(Vertex vertex, Iterable messages) throws IOException {
+    workerLogic.compute(vertex, messages);
+  }
+
+  @Override
+  public void postSuperstep() {
+    workerLogic.postSuperstep();
+  }
+}


[3/5] git commit: updated refs/heads/trunk to 819d6d3

Posted by ik...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java
new file mode 100644
index 0000000..fd38520
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java
@@ -0,0 +1,119 @@
+/*
+ * 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.block_app.framework.output;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.utils.CallableFactory;
+import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Handler for blocks output - keeps track of outputs and writers created
+ */
+@SuppressWarnings("unchecked")
+public class BlockOutputHandle implements BlockOutputApi {
+  private transient Configuration conf;
+  private transient Progressable progressable;
+  private final Map<String, BlockOutputDesc> outputDescMap;
+  private final Map<String, Queue<BlockOutputWriter>> freeWriters;
+  private final Map<String, Queue<BlockOutputWriter>> occupiedWriters;
+
+  public BlockOutputHandle(String jobIdentifier, Configuration conf,
+      Progressable hadoopProgressable) {
+    outputDescMap = BlockOutputFormat.createInitAndCheckOutputDescsMap(
+        conf, jobIdentifier);
+    freeWriters = new HashMap<>();
+    occupiedWriters = new HashMap<>();
+    for (String confOption : outputDescMap.keySet()) {
+      freeWriters.put(confOption,
+          new ConcurrentLinkedQueue<BlockOutputWriter>());
+      occupiedWriters.put(confOption,
+          new ConcurrentLinkedQueue<BlockOutputWriter>());
+    }
+    initialize(conf, hadoopProgressable);
+  }
+
+  public void initialize(Configuration conf, Progressable progressable) {
+    this.conf = conf;
+    this.progressable = progressable;
+  }
+
+
+  @Override
+  public <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+  OD getOutputDesc(String confOption) {
+    return (OD) outputDescMap.get(confOption);
+  }
+
+  @Override
+  public <OW extends BlockOutputWriter> OW getWriter(String confOption) {
+    OW outputWriter = (OW) freeWriters.get(confOption).poll();
+    if (outputWriter == null) {
+      outputWriter = (OW) outputDescMap.get(confOption).createOutputWriter(
+          conf, progressable);
+    }
+    occupiedWriters.get(confOption).add(outputWriter);
+    return outputWriter;
+  }
+
+  public void returnAllWriters() {
+    for (Map.Entry<String, Queue<BlockOutputWriter>> entry :
+        occupiedWriters.entrySet()) {
+      freeWriters.get(entry.getKey()).addAll(entry.getValue());
+      entry.getValue().clear();
+    }
+  }
+
+  public void closeAllWriters() {
+    final Queue<BlockOutputWriter> allWriters = new ConcurrentLinkedQueue<>();
+    for (Queue<BlockOutputWriter> blockOutputWriters : freeWriters.values()) {
+      allWriters.addAll(blockOutputWriters);
+    }
+    if (allWriters.isEmpty()) {
+      return;
+    }
+    // Closing writers can take time - use multiple threads and call progress
+    CallableFactory<Void> callableFactory = new CallableFactory<Void>() {
+      @Override
+      public Callable<Void> newCallable(int callableId) {
+        return new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            BlockOutputWriter writer = allWriters.poll();
+            while (writer != null) {
+              writer.close();
+              writer = allWriters.poll();
+            }
+            return null;
+          }
+        };
+      }
+    };
+    ProgressableUtils.getResultsWithNCallables(callableFactory,
+        Math.min(GiraphConstants.NUM_OUTPUT_THREADS.get(conf),
+            allWriters.size()), "close-writers-%d", progressable);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java
new file mode 100644
index 0000000..5f82612
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.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.block_app.framework.output;
+
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.conf.GiraphConfiguration;
+
+/**
+ * Block output option, with apis to use from application code
+ *
+ * @param <OD> Output description type
+ * @param <OW> Output writer type
+ */
+public class BlockOutputOption<OD extends BlockOutputDesc<OW>,
+    OW extends BlockOutputWriter> {
+  private final String key;
+
+  public BlockOutputOption(String key) {
+    this.key = key;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public void register(OD outputDesc, GiraphConfiguration conf) {
+    BlockOutputFormat.addOutputDesc(outputDesc, key, conf);
+  }
+
+  public OD getOutputDesc(BlockOutputApi outputApi) {
+    return outputApi.<OW, OD>getOutputDesc(key);
+  }
+
+  public OW getWriter(BlockOutputApi outputApi) {
+    return outputApi.getWriter(key);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java
new file mode 100644
index 0000000..5574ab4
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.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.block_app.framework.output;
+
+import java.io.Closeable;
+
+/**
+ * Block output writer
+ */
+public interface BlockOutputWriter extends Closeable {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java
new file mode 100644
index 0000000..11193b1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/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.
+ */
+/**
+ * Multi-output support for Block Applications
+ */
+package org.apache.giraph.block_app.framework.output;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java
new file mode 100644
index 0000000..0b6934e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/**
+ * Block application abstraction package.
+ *
+ * Giraph application is represented as a collection of pieces,
+ * aggregated via blocks, ultimately into a single block, that
+ * represents complete application execution.
+ */
+package org.apache.giraph.block_app.framework;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java
new file mode 100644
index 0000000..882f4f1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java
@@ -0,0 +1,287 @@
+/*
+ * 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.block_app.framework.piece;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexSender;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.function.Consumer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.Iterators;
+
+/**
+ * Parent of all Pieces, contains comprehensive list of methods Piece
+ * can support. Specific subclasses should be extended directly,
+ * to simplify usage - most frequently for example Piece class.
+ *
+ * Single unit of execution, capturing:
+ * - sending and then receiving messages from vertices
+ * - sending data to be aggregated from workers to master
+ * - sending values from master, via aggregators, to workers
+ * - sending and receiving worker messages
+ *
+ *
+ * Order of execution is:
+ * - On master, once at the start of the application
+ * -- registerAggregators (deprecated, use registerReducers instead)
+ *
+ * - After masterCompute of previous piece, on master:
+ * -- registerReducers
+ *
+ * - Send logic on workers:
+ * -- getVertexSender per each worker thread, and on object returned by it:
+ * --- vertexSend on each vertex
+ * --- postprocess on each worker thread
+ * -- workerContextSend per worker
+ *
+ * - Logic on master:
+ * -- masterCompute
+ *
+ * - Receive logic on workers:
+ * -- workerContextReceive per worker
+ * -- getVertexReceiver per each worker thread, and on object returned by it:
+ * --- vertexReceive on each vertex
+ * --- postprocess on each worker thread
+ *
+ * And before everything, during initialization, registerAggregators.
+ *
+ * Only masterCompute and registerReducers/registerAggregators should modify
+ * the Piece, all of the worker methods should treat Piece as read-only.
+ *
+ * Each piece should be encapsulated unit of execution. Vertex value should be
+ * used as a single implicit "communication" channel between different pieces,
+ * all other dependencies should be explicitly defined and passed through
+ * constructor, via interfaces (as explained below).
+ * I.e. state of the vertex value is invariant that Pieces act upon.
+ * Best is not to depend on explicit vertex value class, but on interface that
+ * provides all needed functions, so that pieces can be freely combined,
+ * as long as vertex value implements appropriate ones.
+ * Similarly, use most abstract class you need - if Piece doesn't depend
+ * on edge value, don't use NullWritable, but Writable. Or if it doesn't
+ * depend on ExecutionStage, use Object for it.
+ *
+ * All other external dependencies should be explicitly passed through
+ * constructor, through interfaces.
+ *
+ * All Pieces will be created within one context - on the master.
+ * They are then going to be replicated across all workers, and across all
+ * threads within each worker, and will see everything that happens in global
+ * context (masterCompute) before them, including any state master has.
+ * Through ObjectHolder/ObjectTransfer, you can pass data between Pieces in
+ * global context, and from global context to worker functions of a Piece
+ * that happens in the future.
+ *
+ * VertexReceiver of previous Piece and VertexSender of next Piece live in
+ * the same context, and vertexReceive of the next Piece is executed
+ * immediately after vertexSend of the previous piece, before vertexSend is
+ * called on the next vertex.
+ * This detail allows you to have external dependency on each other through
+ * memory only mediator objects - like ObjectTransfer.
+ *
+ * All other logic going to live in different contexts,
+ * specifically VertexSender and VertexReceiver of the same Piece,
+ * or workerContextSend and VertexSender of the same Piece, and cannot interact
+ * with each other outside of changing the state of the graph or using
+ * global communication api.
+ *
+ * All methods on this class (or objects it returns) will be called serially,
+ * so there is no need for any Thread synchronization.
+ * Each Thread will have a complete deep copy of the Piece, to achieve that,
+ * so all static fields must be written to be Thread safe!
+ * (i.e. either immutable, or have synchronized/locked access to them)
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes" })
+public abstract class AbstractPiece<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable, WV,
+    WM extends Writable, S> implements Block {
+
+  // Overridable functions
+
+  // registerReducers(CreateReducersApi reduceApi, S executionStage)
+
+  /**
+   * Add automatic handling of reducers to registerReducers.
+   * Only for internal use.
+   */
+  public abstract void wrappedRegisterReducers(
+      BlockMasterApi masterApi, S executionStage);
+
+  // getVertexSender(BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage)
+
+  /**
+   * Add automatic handling of reducers to getVertexSender.
+   *
+   * Only for Framework internal use.
+   */
+  public abstract InnerVertexSender getWrappedVertexSender(
+      final BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage);
+
+  /**
+   * Override to have worker context send computation.
+   *
+   * Called once per worker, after all vertices have been processed with
+   * getVertexSender.
+   */
+  public void workerContextSend(
+      BlockWorkerContextSendApi<WM> workerContextApi, S executionStage,
+      WV workerValue) {
+  }
+
+  /**
+   * Function that is called on master, after send phase, before receive phase.
+   *
+   * It can:
+   * - read aggregators sent from worker
+   * - do global processing
+   * - send data to workers through aggregators
+   */
+  public void masterCompute(BlockMasterApi masterApi, S executionStage) {
+  }
+
+  /**
+   * Override to have worker context receive computation.
+   *
+   * Called once per worker, before all vertices are going to be processed
+   * with getVertexReceiver.
+   */
+  public void workerContextReceive(
+      BlockWorkerContextReceiveApi workerContextApi, S executionStage,
+      WV workerValue, List<WM> workerMessages) {
+  }
+
+  /**
+   * Override to do vertex receive processing.
+   *
+   * Creates handler that defines what should be executed on worker
+   * for each vertex during receive phase.
+   *
+   * This logic executed last.
+   * This function is called once on each worker on each thread, in parallel,
+   * on their copy of Piece object to create functions handler.
+   *
+   * If returned object implements Postprocessor interface, then corresponding
+   * postprocess() function is going to be called once, after all vertices
+   * corresponding thread needed to process are done.
+   */
+  public VertexReceiver<I, V, E, M> getVertexReceiver(
+      BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+    return null;
+  }
+
+  /**
+   * Returns MessageClasses definition for messages being sent by this Piece.
+   */
+  public abstract MessageClasses<I, M> getMessageClasses(
+      ImmutableClassesGiraphConfiguration conf);
+
+  /**
+   * Override to provide different next execution stage for
+   * Pieces that come after it.
+   *
+   * Execution stage should be immutable, and this function should be
+   * returning a new object, if it needs to return different value.
+   *
+   * It affects pieces that come after this piece,
+   * and isn't applied to execution stage this piece sees.
+   */
+  public S nextExecutionStage(S executionStage) {
+    return executionStage;
+  }
+
+  /**
+   * Override to register any potential aggregators used by this piece.
+   *
+   * @deprecated Use registerReducers instead.
+   */
+  @Deprecated
+  public void registerAggregators(BlockMasterApi masterApi)
+      throws InstantiationException, IllegalAccessException {
+  }
+
+  // Inner classes
+
+  /** Inner class to provide clean use without specifying types */
+  public abstract class InnerVertexSender
+      implements VertexSender<I, V, E>, VertexPostprocessor {
+    @Override
+    public void postprocess() { }
+  }
+
+  /** Inner class to provide clean use without specifying types */
+  public abstract class InnerVertexReceiver
+      implements VertexReceiver<I, V, E, M>, VertexPostprocessor {
+    @Override
+    public void postprocess() { }
+  }
+
+  // Internal implementation
+
+  @Override
+  public final Iterator<AbstractPiece> iterator() {
+    return Iterators.<AbstractPiece>singletonIterator(this);
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    consumer.apply(this);
+  }
+
+  @Override
+  public String toString() {
+    String name = getClass().getSimpleName();
+    if (name.isEmpty()) {
+      name = getClass().getName();
+    }
+    return name;
+  }
+
+
+  // make hashCode and equals final, forcing them to be based on
+  // reference identity.
+  @Override
+  public final int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public final boolean equals(Object obj) {
+    return super.equals(obj);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java
new file mode 100644
index 0000000..0963efb
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java
@@ -0,0 +1,311 @@
+/*
+ * 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.block_app.framework.piece;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.api.CreateReducersApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReduceUtilsObject;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.block_app.framework.piece.global_comm.internal.CreateReducersApiWrapper;
+import org.apache.giraph.block_app.framework.piece.global_comm.internal.ReducersForPieceHandler;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexSender;
+import org.apache.giraph.block_app.framework.piece.messages.ObjectMessageClasses;
+import org.apache.giraph.block_app.framework.piece.messages.SupplierFromConf;
+import org.apache.giraph.block_app.framework.piece.messages.SupplierFromConf.DefaultMessageFactorySupplierFromConf;
+import org.apache.giraph.block_app.framework.piece.messages.SupplierFromConf.SupplierFromConfByCopy;
+import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.EnumConfOption;
+import org.apache.giraph.conf.GiraphConfigurationSettable;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.types.NoMessage;
+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;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Additional abstract implementations for all pieces to be used.
+ * Code here is not in AbstractPiece only to allow for non-standard
+ * non-user-defined pieces. <br>
+ * Only logic used by the underlying framework directly is in AbstractPiece
+ * itself.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public abstract class DefaultParentPiece<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable, WV,
+    WM extends Writable, S> extends AbstractPiece<I, V, E, M, WV, WM, S> {
+  // TODO move to GiraphConstants
+  /**
+   * This option will tell which message encode & store enum to force, when
+   * combining is not enabled.
+   *
+   * MESSAGE_ENCODE_AND_STORE_TYPE and this property are basically upper
+   * and lower bound on message store type, when looking them in order from
+   * not doing anything special, to most advanced type:
+   * BYTEARRAY_PER_PARTITION,
+   * EXTRACT_BYTEARRAY_PER_PARTITION,
+   * POINTER_LIST_PER_VERTEX
+   * resulting encode type is going to be:
+   * pieceEncodingType = piece.allowOneMessageToManyIdsEncoding() ?
+   *    POINTER_LIST_PER_VERTEX : BYTEARRAY_PER_PARTITION)
+   * Math.max(index(minForce), Math.min(index(maxAllowed), index(pieceType);
+   *
+   * This is useful to force all pieces onto particular message store, even
+   * if they do not overrideallowOneMessageToManyIdsEncoding, though that might
+   * be rarely needed.
+   * This option might be more useful for fully local computation,
+   * where overall job behavior is quite different.
+   */
+  public static final EnumConfOption<MessageEncodeAndStoreType>
+  MESSAGE_ENCODE_AND_STORE_TYPE_MIN_FORCE =
+      EnumConfOption.create("giraph.messageEncodeAndStoreTypeMinForce",
+          MessageEncodeAndStoreType.class,
+          MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION,
+          "Select the message_encode_and_store_type min force to use");
+
+  private final ReduceUtilsObject reduceUtils = new ReduceUtilsObject();
+  private ReducersForPieceHandler reducersHandler;
+
+  // Overridable functions
+
+  /**
+   * Override to register any potential reducers used by this piece,
+   * through calls to {@code reduceApi}, which will return reducer handles
+   * for simple.
+   * <br/>
+   * Tip: Without defining a field, first write here name of the field and what
+   * you want to reduce, like:
+   * <br/>
+   * {@code totalSum = reduceApi.createLocalReducer(SumReduce.DOUBLE); }
+   * <br/>
+   * and then use tools your IDE provides to generate field signature itself,
+   * which might be slightly complex:
+   * <br/>
+   * {@code ReducerHandle<DoubleWritable, DoubleWritable> totalSum; }
+   */
+  public void registerReducers(CreateReducersApi reduceApi, S executionStage) {
+  }
+
+  /**
+   * Override to do vertex send processing.
+   *
+   * Creates handler that defines what should be executed on worker
+   * during send phase.
+   *
+   * This logic gets executed first.
+   * This function is called once on each worker on each thread, in parallel,
+   * on their copy of Piece object to create functions handler.
+   *
+   * If returned object implements Postprocessor interface, then corresponding
+   * postprocess() function is going to be called once, after all vertices
+   * corresponding thread needed to process are done.
+   */
+  public VertexSender<I, V, E> getVertexSender(
+      BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+    return null;
+  }
+
+  /**
+   * Override to specify type of the message this Piece sends, if it does
+   * send messages.
+   *
+   * If not overwritten, no messages can be sent.
+   */
+  protected Class<M> getMessageClass() {
+    return null;
+  }
+
+  /**
+   * Override to specify message value factory to be used,
+   * which creates objects into which messages will be deserialized.
+   *
+   * If not overwritten, or null is returned, DefaultMessageValueFactory
+   * will be used.
+   */
+  protected MessageValueFactory<M> getMessageFactory(
+      ImmutableClassesGiraphConfiguration conf) {
+    return null;
+  }
+
+  /**
+   * Override to specify message combiner to be used, if any.
+   *
+   * Message combiner itself should be immutable
+   * (i.e. it will be call simultanously from multiple threads)
+   */
+  protected MessageCombiner<? super I, M> getMessageCombiner(
+      ImmutableClassesGiraphConfiguration conf) {
+    return null;
+  }
+
+  /**
+   * Override to specify that this Piece allows one to many ids encoding to be
+   * used for messages.
+   * You should override this function, if you are sending identical message to
+   * all targets, and message itself is not extremely small.
+   */
+  protected boolean allowOneMessageToManyIdsEncoding() {
+    return false;
+  }
+
+  @Override
+  public MessageClasses<I, M> getMessageClasses(
+      ImmutableClassesGiraphConfiguration conf) {
+    Class<M> messageClass = null;
+    MessageValueFactory<M> messageFactory = getMessageFactory(conf);
+    MessageCombiner<? super I, M> messageCombiner = getMessageCombiner(conf);
+
+    if (messageFactory != null) {
+      messageClass = (Class) messageFactory.newInstance().getClass();
+    } else if (messageCombiner != null) {
+      messageClass = (Class) messageCombiner.createInitialMessage().getClass();
+    }
+
+    if (messageClass != null) {
+      Preconditions.checkState(getMessageClass() == null,
+          "Piece %s defines getMessageFactory or getMessageCombiner, " +
+          "so it doesn't need to define getMessageClass.",
+          toString());
+    } else {
+      messageClass = getMessageClass();
+      if (messageClass == null) {
+        messageClass = (Class) NoMessage.class;
+      }
+    }
+
+    SupplierFromConf<MessageValueFactory<M>> messageFactorySupplier;
+    if (messageFactory != null) {
+      messageFactorySupplier =
+          new SupplierFromConfByCopy<MessageValueFactory<M>>(messageFactory);
+    } else {
+      messageFactorySupplier =
+          new DefaultMessageFactorySupplierFromConf<>(messageClass);
+    }
+
+    SupplierFromConf<? extends MessageCombiner<? super I, M>>
+    messageCombinerSupplier;
+    if (messageCombiner != null) {
+      messageCombinerSupplier = new SupplierFromConfByCopy<>(messageCombiner);
+    } else {
+      messageCombinerSupplier = null;
+    }
+
+    int maxAllowed =
+        GiraphConstants.MESSAGE_ENCODE_AND_STORE_TYPE.get(conf).ordinal();
+    int minForce = MESSAGE_ENCODE_AND_STORE_TYPE_MIN_FORCE.get(conf).ordinal();
+    Preconditions.checkState(maxAllowed >= minForce);
+
+    int pieceEncodeType = (allowOneMessageToManyIdsEncoding() ?
+        MessageEncodeAndStoreType.POINTER_LIST_PER_VERTEX :
+        MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION).ordinal();
+    // bound piece type with boundaries:
+    pieceEncodeType = Math.max(minForce, Math.min(maxAllowed, pieceEncodeType));
+
+    MessageEncodeAndStoreType messageEncodeAndStoreType =
+        MessageEncodeAndStoreType.values()[pieceEncodeType];
+
+    if (messageFactory instanceof GiraphConfigurationSettable) {
+      throw new IllegalStateException(
+          messageFactory.getClass() + " MessageFactory in " + this +
+          " Piece implements GiraphConfigurationSettable");
+    }
+    if (messageCombiner instanceof GiraphConfigurationSettable) {
+      throw new IllegalStateException(
+          messageCombiner.getClass() + " MessageCombiner in " + this +
+          " Piece implements GiraphConfigurationSettable");
+    }
+
+    return new ObjectMessageClasses<>(
+        messageClass, messageFactorySupplier,
+        messageCombinerSupplier, messageEncodeAndStoreType);
+  }
+
+  // Internal implementation
+
+  @Override
+  public final InnerVertexSender getWrappedVertexSender(
+      final BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+    reducersHandler.vertexSenderWorkerPreprocess(workerApi);
+    final VertexSender<I, V, E> functions =
+        getVertexSender(workerApi, executionStage);
+    return new InnerVertexSender() {
+      @Override
+      public void vertexSend(Vertex<I, V, E> vertex) {
+        if (functions != null) {
+          functions.vertexSend(vertex);
+        }
+      }
+      @Override
+      public void postprocess() {
+        if (functions instanceof VertexPostprocessor) {
+          ((VertexPostprocessor) functions).postprocess();
+        }
+        reducersHandler.vertexSenderWorkerPostprocess(workerApi);
+      }
+    };
+  }
+
+  @Override
+  public final void wrappedRegisterReducers(
+      BlockMasterApi masterApi, S executionStage) {
+    reducersHandler = new ReducersForPieceHandler();
+    registerReducers(new CreateReducersApiWrapper(
+        masterApi, reducersHandler), executionStage);
+  }
+
+  // utility functions:
+  // TODO Java8 - move these as default functions to VertexSender interface
+  protected final void reduceDouble(
+      ReducerHandle<DoubleWritable, ?> reduceHandle, double value) {
+    reduceUtils.reduceDouble(reduceHandle, value);
+  }
+
+  protected final void reduceFloat(
+      ReducerHandle<FloatWritable, ?> reduceHandle, float value) {
+    reduceUtils.reduceFloat(reduceHandle, value);
+  }
+
+  protected final void reduceLong(
+      ReducerHandle<LongWritable, ?> reduceHandle, long value) {
+    reduceUtils.reduceLong(reduceHandle, value);
+  }
+
+  protected final void reduceInt(
+      ReducerHandle<IntWritable, ?> reduceHandle, int value) {
+    reduceUtils.reduceInt(reduceHandle, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java
new file mode 100644
index 0000000..3ad66d1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.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.block_app.framework.piece;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.types.NoMessage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Piece that should be extended in common usecases, when we want to be:
+ * - sending and then receiving messages from vertices
+ * - sending data to be aggregated from workers to master
+ * - sending values from master, via aggregators, to workers
+ *
+ * (basically - we don't want to use WorkerContext)
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class Piece<I extends WritableComparable, V extends Writable,
+    E extends Writable, M extends Writable, S>
+    extends DefaultParentPiece<I, V, E, M, Object, NoMessage, S> {
+
+  // Disallowing use of Worker Context functions:
+  @Override
+  public final void workerContextSend(
+      BlockWorkerContextSendApi<NoMessage> workerContextApi,
+      S executionStage, Object workerValue) {
+  }
+
+  @Override
+  public final void workerContextReceive(
+      BlockWorkerContextReceiveApi workerContextApi,
+      S executionStage, Object workerValue, List<NoMessage> workerMessages) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java
new file mode 100644
index 0000000..a5d0c8c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.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.block_app.framework.piece;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerValueAccessor;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Piece that should be extended when WorkerContext is used.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class PieceWithWorkerContext<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable, WV,
+    WM extends Writable, S>
+    extends DefaultParentPiece<I, V, E, M, WV, WM, S> {
+
+  /**
+   * Get global worker value.
+   * Value returned can be accessed from may threads, and so all
+   * accesses to it should be done in a thread-safe manner!
+   *
+   * This is the only place in Blocks Framework where you need
+   * to take care of concurrency.
+   */
+  @SuppressWarnings("unchecked")
+  public WV getWorkerValue(BlockWorkerApi<I> workerApi) {
+    return (WV) ((BlockWorkerValueAccessor) workerApi).getWorkerValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java
new file mode 100644
index 0000000..23c2d29
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java
@@ -0,0 +1,277 @@
+/*
+ * 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.block_app.framework.piece.delegate;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.types.NoMessage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Delegate Piece which allows combining multiple pieces in same iteration:
+ * new DelegatePiece(new LogicPiece(), new StatsPiece())
+ * You should be careful when doing so, since those pieces must not interact,
+ * and only one can send messages.
+ * Execution of any of the Piece methods by the framework is going to trigger
+ * sequential execution of that method on all of the pieces this DelegatePiece
+ * wraps. That means for example, getVertexSender is going to be called on all
+ * pieces before masterCompute is called on all pieces, which is called before
+ * getVertexReceiver on all pieces.
+ *
+ * Also, via overriding, it provides an abstract class for filtering. I.e. if
+ * you want piece that filters out calls to masterCompute, you can have:
+ * new FilterMasterPiece(new LogicPiece()),
+ * with FilterMasterPiece extends DelegatePiece, and only overrides getMaster
+ * function and DelegateMasterPiece class.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class DelegatePiece<I extends WritableComparable, V extends Writable,
+    E extends Writable, M extends Writable, WV, WM extends Writable, S>
+    extends AbstractPiece<I, V, E, M, WV, WM, S> {
+
+  private final List<AbstractPiece<I, V, E, M, WV, WM, S>> innerPieces;
+
+  @SafeVarargs
+  @SuppressWarnings("unchecked")
+  public DelegatePiece(AbstractPiece<? super I, ? super V, ? super E,
+      ? super M, ? super WV, ? super WM, ? super S>... innerPieces) {
+    // Pieces are contravariant, but Java generics cannot express that,
+    // so use unchecked cast inside to allow callers to be typesafe
+    this.innerPieces = new ArrayList(Arrays.asList(innerPieces));
+  }
+
+  @SuppressWarnings("unchecked")
+  public DelegatePiece(AbstractPiece<? super I, ? super V, ? super E,
+      ? super M, ? super WV, ? super WM, ? super S> innerPiece) {
+    // Pieces are contravariant, but Java generics cannot express that,
+    // so use unchecked cast inside to allow callers to be typesafe
+    this.innerPieces = new ArrayList(Arrays.asList(innerPiece));
+  }
+
+  protected DelegateWorkerSendFunctions delegateWorkerSendFunctions(
+      ArrayList<InnerVertexSender> workerSendFunctions,
+      BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+    return new DelegateWorkerSendFunctions(workerSendFunctions);
+  }
+
+  protected DelegateWorkerReceiveFunctions delegateWorkerReceiveFunctions(
+      ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions,
+      BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+    return new DelegateWorkerReceiveFunctions(workerReceiveFunctions);
+  }
+
+  @Override
+  public InnerVertexSender getWrappedVertexSender(
+      BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+    ArrayList<InnerVertexSender> workerSendFunctions =
+        new ArrayList<>(innerPieces.size());
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      workerSendFunctions.add(
+          innerPiece.getWrappedVertexSender(workerApi, executionStage));
+    }
+    return delegateWorkerSendFunctions(
+        workerSendFunctions, workerApi, executionStage);
+  }
+
+  @Override
+  public InnerVertexReceiver getVertexReceiver(
+      BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+    ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions =
+        new ArrayList<>(innerPieces.size());
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      workerReceiveFunctions.add(
+          innerPiece.getVertexReceiver(workerApi, executionStage));
+    }
+    return delegateWorkerReceiveFunctions(
+        workerReceiveFunctions, workerApi, executionStage);
+  }
+
+  /** Delegating WorkerSendPiece */
+  protected class DelegateWorkerSendFunctions extends InnerVertexSender {
+    private final ArrayList<InnerVertexSender> workerSendFunctions;
+
+    public DelegateWorkerSendFunctions(
+        ArrayList<InnerVertexSender> workerSendFunctions) {
+      this.workerSendFunctions = workerSendFunctions;
+    }
+
+    @Override
+    public void vertexSend(Vertex<I, V, E> vertex) {
+      for (InnerVertexSender functions : workerSendFunctions) {
+        if (functions != null) {
+          functions.vertexSend(vertex);
+        }
+      }
+    }
+
+    @Override
+    public void postprocess() {
+      for (InnerVertexSender functions : workerSendFunctions) {
+        if (functions != null) {
+          functions.postprocess();
+        }
+      }
+    }
+  }
+
+  /** Delegating WorkerReceivePiece */
+  protected class DelegateWorkerReceiveFunctions extends InnerVertexReceiver {
+    private final ArrayList<VertexReceiver<I, V, E, M>>
+    workerReceiveFunctions;
+
+    public DelegateWorkerReceiveFunctions(
+        ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions) {
+      this.workerReceiveFunctions = workerReceiveFunctions;
+    }
+
+    @Override
+    public void vertexReceive(Vertex<I, V, E> vertex, Iterable<M> messages) {
+      for (VertexReceiver<I, V, E, M> functions :
+            workerReceiveFunctions) {
+        if (functions != null) {
+          functions.vertexReceive(vertex, messages);
+        }
+      }
+    }
+
+    @Override
+    public void postprocess() {
+      for (VertexReceiver<I, V, E, M> functions :
+            workerReceiveFunctions) {
+        if (functions instanceof VertexPostprocessor) {
+          ((VertexPostprocessor) functions).postprocess();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void masterCompute(BlockMasterApi api, S executionStage) {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> piece : innerPieces) {
+      piece.masterCompute(api, executionStage);
+    }
+  }
+
+  @Override
+  public void workerContextSend(
+      BlockWorkerContextSendApi<WM> workerContextApi, S executionStage,
+      WV workerValue) {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> piece : innerPieces) {
+      piece.workerContextSend(workerContextApi, executionStage, workerValue);
+    }
+  }
+
+  @Override
+  public void workerContextReceive(
+      BlockWorkerContextReceiveApi workerContextApi, S executionStage,
+      WV workerValue, List<WM> workerMessages) {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> piece : innerPieces) {
+      piece.workerContextReceive(
+          workerContextApi, executionStage, workerValue, workerMessages);
+    }
+  }
+
+  @Override
+  public S nextExecutionStage(S executionStage) {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      executionStage = innerPiece.nextExecutionStage(executionStage);
+    }
+    return executionStage;
+  }
+
+  @Override
+  public MessageClasses<I, M> getMessageClasses(
+      ImmutableClassesGiraphConfiguration conf) {
+    MessageClasses<I, M> messageClasses = null;
+    MessageClasses<I, M> firstMessageClasses = null;
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      MessageClasses<I, M> cur = innerPiece.getMessageClasses(conf);
+      Preconditions.checkState(cur != null);
+      if (!cur.getMessageClass().equals(NoMessage.class)) {
+        if (messageClasses != null) {
+          throw new RuntimeException(
+              "Only one piece combined through delegate (" +
+              toString() + ") can send messages");
+        }
+        messageClasses = cur;
+      }
+      if (firstMessageClasses == null) {
+        firstMessageClasses = cur;
+      }
+    }
+    return messageClasses != null ? messageClasses : firstMessageClasses;
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      innerPiece.forAllPossiblePieces(consumer);
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public void registerAggregators(BlockMasterApi master)
+      throws InstantiationException, IllegalAccessException {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      innerPiece.registerAggregators(master);
+    }
+  }
+
+  @Override
+  public void wrappedRegisterReducers(
+      BlockMasterApi masterApi, S executionStage) {
+    for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+      innerPiece.wrappedRegisterReducers(masterApi, executionStage);
+    }
+  }
+
+  protected String delegationName() {
+    return "Delegate";
+  }
+
+  @Override
+  public String toString() {
+    return delegationName() + innerPieces.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java
new file mode 100644
index 0000000..5c702c5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.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.block_app.framework.piece.delegate;
+
+import java.util.ArrayList;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.function.vertex.SupplierFromVertex;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Piece which uses a provided suppliers to decide whether or not to run
+ * receive/send piece part on a certain vertex.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes" })
+public class FilteringPiece<I extends WritableComparable, V extends Writable,
+    E extends Writable, M extends Writable, WV, WM extends Writable, S>
+    extends DelegatePiece<I, V, E, M, WV, WM, S> {
+  private final SupplierFromVertex<I, V, E, Boolean> toCallSend;
+  private final SupplierFromVertex<I, V, E, Boolean> toCallReceive;
+
+  /**
+   * Creates filtering piece which uses passed {@code toCallSend} to filter
+   * calls to {@code vertexSend}, and passed {@code toCallReceive} to filter
+   * calls to {@code vertexReceive}, on passed {@code innerPiece}.
+   */
+  @SuppressWarnings("unchecked")
+  public FilteringPiece(
+      SupplierFromVertex<? super I, ? super V, ? super E, Boolean> toCallSend,
+      SupplierFromVertex<? super I, ? super V, ? super E, Boolean>
+        toCallReceive,
+      AbstractPiece<? super I, ? super V, ? super E, ? super M,
+        ? super WV, ? super WM, ? super S> innerPiece) {
+    super(innerPiece);
+    // Suppliers are contravariant on vertex types,
+    // but Java generics cannot express that,
+    // so use unchecked cast inside to allow callers to be typesafe
+    this.toCallSend = (SupplierFromVertex) toCallSend;
+    this.toCallReceive = (SupplierFromVertex) toCallReceive;
+    Preconditions.checkArgument(
+        toCallSend != null || toCallReceive != null,
+        "Both send and receive filter cannot be null");
+  }
+
+  /**
+   * Creates filtering piece, where both vertexSend and vertexReceive is
+   * filtered based on same supplier.
+   */
+  public FilteringPiece(
+      SupplierFromVertex<? super I, ? super V, ? super E, Boolean>
+        toCallSendAndReceive,
+      AbstractPiece<? super I, ? super V, ? super E, ? super M,
+        ? super WV, ? super WM, ? super S> innerPiece) {
+    this(toCallSendAndReceive, toCallSendAndReceive, innerPiece);
+  }
+
+  /**
+   * Creates filtering piece, that filters only vertexReceive function,
+   * and always calls vertexSend function.
+   */
+  public static <I extends WritableComparable, V extends Writable,
+  E extends Writable, M extends Writable, WV, WM extends Writable, S>
+  FilteringPiece<I, V, E, M, WV, WM, S> createReceiveFiltering(
+      SupplierFromVertex<? super I, ? super V, ? super E, Boolean>
+        toCallReceive,
+      AbstractPiece<? super I, ? super V, ? super E, ? super M,
+        ? super WV, ? super WM, ? super S> innerPiece) {
+    return new FilteringPiece<>(null, toCallReceive, innerPiece);
+  }
+
+  /**
+   * Creates filtering block, that filters only vertexSend function,
+   * and always calls vertexReceive function.
+   */
+  public static <I extends WritableComparable, V extends Writable,
+  E extends Writable, M extends Writable, WV, WM extends Writable, S>
+  FilteringPiece<I, V, E, M, WV, WM, S> createSendFiltering(
+      SupplierFromVertex<? super I, ? super V, ? super E, Boolean> toCallSend,
+      AbstractPiece<? super I, ? super V, ? super E, ? super M, ? super WV,
+        ? super WM, ? super S> innerPiece) {
+    return new FilteringPiece<>(toCallSend, null, innerPiece);
+  }
+
+  @Override
+  protected DelegateWorkerSendFunctions delegateWorkerSendFunctions(
+      ArrayList<InnerVertexSender> workerSendFunctions,
+      BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+    return new DelegateWorkerSendFunctions(workerSendFunctions) {
+      @Override
+      public void vertexSend(Vertex<I, V, E> vertex) {
+        if (toCallSend == null || toCallSend.get(vertex)) {
+          super.vertexSend(vertex);
+        }
+      }
+    };
+  }
+
+  @Override
+  protected DelegateWorkerReceiveFunctions delegateWorkerReceiveFunctions(
+      ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions,
+      BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+    return new DelegateWorkerReceiveFunctions(workerReceiveFunctions) {
+      @Override
+      public void vertexReceive(Vertex<I, V, E> vertex, Iterable<M> messages) {
+        if (toCallReceive == null || toCallReceive.get(vertex)) {
+          super.vertexReceive(vertex, messages);
+        }
+      }
+    };
+  }
+
+  @Override
+  protected String delegationName() {
+    if (toCallSend != null && toCallReceive != null) {
+      if (toCallSend != toCallReceive) {
+        return "AsymFilter";
+      }
+      return "Filter";
+    } else if (toCallSend != null) {
+      return "SendFilter";
+    } else if (toCallReceive != null) {
+      return "ReceiveFilter";
+    } else {
+      throw new IllegalStateException("Both Send and Receive filters are null");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java
new file mode 100644
index 0000000..f367e6c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/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.
+ */
+/**
+ * Pieces that delegate their work to a set of one or multiple other Pieces.
+ */
+package org.apache.giraph.block_app.framework.piece.delegate;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java
new file mode 100644
index 0000000..f18d1f4
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.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.block_app.framework.piece.global_comm;
+
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle to a broadcast.
+ *
+ * @param <T> Value type
+ */
+public interface BroadcastHandle<T> {
+  /** Get broadcasted value */
+  T getBroadcast(WorkerBroadcastUsage worker);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java
new file mode 100644
index 0000000..bbec1c6
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.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.block_app.framework.piece.global_comm;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Utility object with common primitive reduce operations,
+ * without need to create reusable objects within the piece.
+ */
+public class ReduceUtilsObject {
+  private final DoubleWritable reusableDouble = new DoubleWritable();
+  private final FloatWritable reusableFloat = new FloatWritable();
+  private final LongWritable reusableLong = new LongWritable();
+  private final IntWritable reusableInt = new IntWritable();
+
+  // utility functions:
+  public void reduceDouble(
+      ReducerHandle<DoubleWritable, ?> reduceHandle, double value) {
+    DoubleWritable tmp = reusableDouble;
+    tmp.set(value);
+    reduceHandle.reduce(tmp);
+  }
+
+  public void reduceFloat(
+      ReducerHandle<FloatWritable, ?> reduceHandle, float value) {
+    FloatWritable tmp = reusableFloat;
+    tmp.set(value);
+    reduceHandle.reduce(tmp);
+  }
+
+  public void reduceLong(
+      ReducerHandle<LongWritable, ?> reduceHandle, long value) {
+    LongWritable tmp = reusableLong;
+    tmp.set(value);
+    reduceHandle.reduce(tmp);
+  }
+
+  public void reduceInt(ReducerHandle<IntWritable, ?> reduceHandle, int value) {
+    IntWritable tmp = reusableInt;
+    tmp.set(value);
+    reduceHandle.reduce(tmp);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java
new file mode 100644
index 0000000..921c863
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.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.block_app.framework.piece.global_comm;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle that wraps both reducerHandle and broadcastHandle, so callers
+ * don't need to have two fields.
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public class ReducerAndBroadcastWrapperHandle<S, R> {
+  private ReducerHandle<S, R> reducerHandle;
+  private BroadcastHandle<R> broadcastHandle;
+
+  /** Set reducer handle to just registered handle */
+  public void registeredReducer(ReducerHandle<S, R> reducerHandle) {
+    this.reducerHandle = reducerHandle;
+  }
+
+  /** Reduce single value */
+  public void reduce(S valueToReduce) {
+    reducerHandle.reduce(valueToReduce);
+  }
+
+  /** Get reduced value */
+  public R getReducedValue(MasterGlobalCommUsage master) {
+    return reducerHandle.getReducedValue(master);
+  }
+
+  /**
+   * Broadcast reduced value from master
+   */
+  public void broadcastValue(BlockMasterApi master) {
+    broadcastHandle = reducerHandle.broadcastValue(master);
+  }
+
+  /** Get broadcasted value */
+  public R getBroadcast(WorkerBroadcastUsage worker) {
+    return broadcastHandle.getBroadcast(worker);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java
new file mode 100644
index 0000000..dae40f2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java
@@ -0,0 +1,41 @@
+/*
+ * 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.block_app.framework.piece.global_comm;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+
+/**
+ * Handle to a reducer.
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public interface ReducerHandle<S, R> {
+  /** Reduce single value */
+  void reduce(S valueToReduce);
+  /** Get reduced value */
+  R getReducedValue(MasterGlobalCommUsage master);
+
+  /**
+   * Broadcast reduced value from master
+   *
+   * @return Handle to the broadcasted value.
+   */
+  BroadcastHandle<R> broadcastValue(BlockMasterApi master);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java
new file mode 100644
index 0000000..7ee54cb
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java
@@ -0,0 +1,36 @@
+/*
+ * 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.block_app.framework.piece.global_comm.array;
+
+/**
+ * Handle to array of handles underneath
+ *
+ * @param <T> Value type
+ */
+public interface ArrayHandle<T> {
+  /**
+   * Get value at index.
+   */
+  T get(int index);
+
+  /**
+   * Size of this array if defined up front, or throws
+   * UnsupportedOperationException if size is dynamic
+   */
+  int getStaticSize();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java
new file mode 100644
index 0000000..bf0d333
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java
@@ -0,0 +1,35 @@
+/*
+ * 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.block_app.framework.piece.global_comm.array;
+
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle to array of broadcasts
+ *
+ * @param <T> Value type
+ */
+public interface BroadcastArrayHandle<T>
+  extends ArrayHandle<BroadcastHandle<T>> {
+
+  /**
+   * Number of elements that were broadcasted.
+   */
+  int getBroadcastedSize(WorkerBroadcastUsage worker);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java
new file mode 100644
index 0000000..a4b99ac
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java
@@ -0,0 +1,43 @@
+/*
+ * 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.block_app.framework.piece.global_comm.array;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+
+/**
+ * Handle to array of reducers
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public interface ReducerArrayHandle<S, R>
+    extends ArrayHandle<ReducerHandle<S, R>> {
+
+  /**
+   * Number of elements that were reduced.
+   */
+  int getReducedSize(BlockMasterApi master);
+
+  /**
+   * Broadcast whole array of reducers to master
+   *
+   * @return Handle to the broadcasted array.
+   */
+  BroadcastArrayHandle<R> broadcastValue(BlockMasterApi master);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java
new file mode 100644
index 0000000..a8beb85
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/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.
+ */
+/**
+ * Interfaces representing arrays of individual handles.
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm.array;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java
new file mode 100644
index 0000000..c2cc0f2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java
@@ -0,0 +1,73 @@
+/*
+ * 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.block_app.framework.piece.global_comm.internal;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.CreateReducersApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Wrapping masterApi and reducers handler into API for creating reducer
+ * handles.
+ */
+public class CreateReducersApiWrapper implements CreateReducersApi {
+  private final BlockMasterApi masterApi;
+  private final ReducersForPieceHandler reducersApi;
+
+  public CreateReducersApiWrapper(
+      BlockMasterApi masterApi, ReducersForPieceHandler reducersApi) {
+    this.masterApi = masterApi;
+    this.reducersApi = reducersApi;
+  }
+
+  @Override
+  public <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+      ReduceOperation<S, R> reduceOp) {
+    return reducersApi.createLocalReducer(
+        masterApi, reduceOp, reduceOp.createInitialValue());
+  }
+
+  @Override
+  public <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+      ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+    return reducersApi.createLocalReducer(
+        masterApi, reduceOp, globalInitialValue);
+  }
+
+  @Override
+  public <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+      ReduceOperation<S, R> reduceOp) {
+    return reducersApi.createGlobalReducer(
+        masterApi, reduceOp, reduceOp.createInitialValue());
+  }
+
+  @Override
+  public <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+      ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+    return reducersApi.createGlobalReducer(
+        masterApi, reduceOp, globalInitialValue);
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<?, ?, ?> getConf() {
+    return masterApi.getConf();
+  }
+}


[4/5] git commit: updated refs/heads/trunk to 819d6d3

Posted by ik...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java
new file mode 100644
index 0000000..ede6005
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java
@@ -0,0 +1,170 @@
+/*
+ * 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.block_app.framework.api.giraph;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.block_app.framework.api.BlockOutputHandleAccessor;
+import org.apache.giraph.block_app.framework.api.Counter;
+import org.apache.giraph.block_app.framework.output.BlockOutputDesc;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.giraph.block_app.framework.output.BlockOutputWriter;
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.block_app.framework.piece.global_comm.internal.ReducersForPieceHandler.BroadcastHandleImpl;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Giraph implementation of BlockMasterApi, that delegates all calls
+ * to MasterCompute.
+ */
+final class BlockMasterApiWrapper implements BlockMasterApi,
+    BlockOutputApi, BlockOutputHandleAccessor {
+  private final MasterCompute master;
+  private final BlockOutputHandle outputHandle;
+
+  public BlockMasterApiWrapper(MasterCompute master,
+                               BlockOutputHandle outputHandle) {
+    this.master = master;
+    this.outputHandle = outputHandle;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<?, ?, ?> getConf() {
+    return master.getConf();
+  }
+
+  @Override
+  public void setStatus(String status) {
+    master.getContext().setStatus(status);
+  }
+
+  @Override
+  public void progress() {
+    master.getContext().progress();
+  }
+
+  @Override
+  public Counter getCounter(String group, String name) {
+    final org.apache.hadoop.mapreduce.Counter counter =
+        master.getContext().getCounter(group, name);
+    return new Counter() {
+      @Override
+      public void increment(long incr) {
+        counter.increment(incr);
+      }
+
+      @Override
+      public void setValue(long value) {
+        counter.setValue(value);
+      }
+    };
+  }
+
+  @Override
+  public <R extends Writable> R getReduced(String name) {
+    return master.getReduced(name);
+  }
+
+  @Override
+  public void broadcast(String name, Writable value) {
+    master.broadcast(name, value);
+  }
+
+  @Override
+  public <S, R extends Writable> void registerReducer(
+      String name, ReduceOperation<S, R> reduceOp) {
+    master.registerReducer(name, reduceOp);
+  }
+
+  @Override
+  public <S, R extends Writable> void registerReducer(
+      String name, ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+    master.registerReducer(name, reduceOp, globalInitialValue);
+  }
+
+  @Override
+  public <A extends Writable> A getAggregatedValue(String name) {
+    return master.getAggregatedValue(name);
+  }
+
+  @Override
+  public <A extends Writable>
+  boolean registerAggregator(
+      String name, Class<? extends Aggregator<A>> aggregatorClass
+  ) throws InstantiationException, IllegalAccessException {
+    return master.registerAggregator(name, aggregatorClass);
+  }
+
+  @Override
+  public <A extends Writable>
+  boolean registerPersistentAggregator(
+      String name, Class<? extends Aggregator<A>> aggregatorClass
+  ) throws InstantiationException,
+      IllegalAccessException {
+    return master.registerPersistentAggregator(name, aggregatorClass);
+  }
+
+  @Override
+  public <A extends Writable> void setAggregatedValue(String name, A value) {
+    master.setAggregatedValue(name, value);
+  }
+
+  @Override
+  public <T extends Writable> BroadcastHandle<T> broadcast(T object) {
+    BroadcastHandleImpl<T> handle = new BroadcastHandleImpl<>();
+    master.broadcast(handle.getName(), object);
+    return handle;
+  }
+
+  @Override
+  @Deprecated
+  public long getTotalNumEdges() {
+    return master.getTotalNumEdges();
+  }
+
+  @Override
+  @Deprecated
+  public long getTotalNumVertices() {
+    return master.getTotalNumVertices();
+  }
+
+  @Override
+  public void logToCommandLine(String line) {
+    master.logToCommandLine(line);
+  }
+
+  @Override
+  public <OW extends BlockOutputWriter,
+      OD extends BlockOutputDesc<OW>> OD getOutputDesc(String confOption) {
+    return outputHandle.<OW, OD>getOutputDesc(confOption);
+  }
+
+  @Override
+  public <OW extends BlockOutputWriter> OW getWriter(String confOption) {
+    return outputHandle.getWriter(confOption);
+  }
+
+  @Override
+  public BlockOutputHandle getBlockOutputHandle() {
+    return outputHandle;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java
new file mode 100644
index 0000000..69cf9f8
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java
@@ -0,0 +1,72 @@
+/*
+ * 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.block_app.framework.api.giraph;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.block_app.framework.internal.BlockMasterLogic;
+import org.apache.giraph.block_app.framework.internal.BlockWorkerPieces;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.writable.kryo.KryoWritableWrapper;
+
+/**
+ * MasterCompute class which executes block computation.
+ *
+ * @param <S> Execution stage type
+ */
+public final class BlockMasterCompute<S> extends MasterCompute {
+  private BlockMasterLogic<S> blockMasterLogic = new BlockMasterLogic<>();
+
+  @Override
+  public void initialize() throws InstantiationException,
+      IllegalAccessException {
+    blockMasterLogic.initialize(getConf(), new BlockMasterApiWrapper(this,
+        new BlockOutputHandle(getContext().getJobID().toString(),
+        getConf(), getContext())));
+  }
+
+  @Override
+  public void compute() {
+    BlockWorkerPieces<S> workerPieces =
+        blockMasterLogic.computeNext(getSuperstep());
+    if (workerPieces == null) {
+      haltComputation();
+    } else {
+      BlockWorkerPieces.setNextWorkerPieces(this, workerPieces);
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    new KryoWritableWrapper<>(blockMasterLogic).write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    KryoWritableWrapper<BlockMasterLogic<S>> object =
+        new KryoWritableWrapper<>();
+    object.readFields(in);
+    blockMasterLogic = object.get();
+    blockMasterLogic.initializeAfterRead(new BlockMasterApiWrapper(this,
+        new BlockOutputHandle(getContext().getJobID().toString(),
+        getConf(), getContext())));
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java
new file mode 100644
index 0000000..6e839f9
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java
@@ -0,0 +1,180 @@
+/*
+ * 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.block_app.framework.api.giraph;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerValueAccessor;
+import org.apache.giraph.block_app.framework.output.BlockOutputDesc;
+import org.apache.giraph.block_app.framework.output.BlockOutputWriter;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.graph.Computation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.types.NoMessage;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Giraph implementation of BlockWorkerReceiveApi and BlockWorkerSendAPI,
+ * passing all calls to Computation.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ */
+@SuppressWarnings("rawtypes")
+final class BlockWorkerApiWrapper<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable>
+    implements BlockWorkerReceiveApi<I>, BlockWorkerSendApi<I, V, E, M>,
+    BlockWorkerValueAccessor, WorkerGlobalCommUsage, BlockOutputApi {
+  private final Computation<I, V, E, NoMessage, M> worker;
+
+  public BlockWorkerApiWrapper(Computation<I, V, E, NoMessage, M> worker) {
+    this.worker = worker;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
+    return worker.getConf();
+  }
+
+  @Override
+  public <A extends Writable> void aggregate(String name, A value) {
+    worker.aggregate(name, value);
+  }
+
+  @Override
+  public <A extends Writable> A getAggregatedValue(String name) {
+    return worker.getAggregatedValue(name);
+  }
+
+  @Override
+  public <B extends Writable> B getBroadcast(String name) {
+    return worker.getBroadcast(name);
+  }
+
+  @Override
+  public void reduce(String name, Object value) {
+    worker.reduce(name, value);
+  }
+
+  @Override
+  public void reduceMerge(String name, Writable value) {
+    worker.reduceMerge(name, value);
+  }
+
+  @Override
+  public void sendMessage(I id, M message) {
+    worker.sendMessage(id, message);
+  }
+
+  @Override
+  public void sendMessageToAllEdges(Vertex<I, V, E> vertex, M message) {
+    worker.sendMessageToAllEdges(vertex, message);
+  }
+
+  @Override
+  public void sendMessageToMultipleEdges(
+      Iterator<I> vertexIdIterator, M message) {
+    worker.sendMessageToMultipleEdges(vertexIdIterator, message);
+  }
+
+  @Override
+  public void addVertexRequest(I id, V value) {
+    try {
+      worker.addVertexRequest(id, value);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void addVertexRequest(I id, V value, OutEdges<I, E> edges) {
+    try {
+      worker.addVertexRequest(id, value, edges);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void removeVertexRequest(I vertexId) {
+    try {
+      worker.removeVertexRequest(vertexId);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void addEdgeRequest(I sourceVertexId, Edge<I, E> edge) {
+    try {
+      worker.addEdgeRequest(sourceVertexId, edge);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void removeEdgesRequest(I sourceVertexId, I targetVertexId) {
+    try {
+      worker.removeEdgesRequest(sourceVertexId, targetVertexId);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private BlockWorkerContext getBlockWorkerContext() {
+    return (BlockWorkerContext) worker.getWorkerContext();
+  }
+
+  @Override
+  public Object getWorkerValue() {
+    return getBlockWorkerContext().getWorkerValue();
+  }
+
+  @Override
+  public long getTotalNumEdges() {
+    return worker.getTotalNumEdges();
+  }
+
+  @Override
+  public long getTotalNumVertices() {
+    return worker.getTotalNumVertices();
+  }
+
+  @Override
+  public <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+  OD getOutputDesc(String confOption) {
+    return getBlockWorkerContext().getOutputHandle().<OW, OD>getOutputDesc(
+        confOption);
+  }
+
+  @Override
+  public <OW extends BlockOutputWriter> OW getWriter(String confOption) {
+    return getBlockWorkerContext().getOutputHandle().getWriter(confOption);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java
new file mode 100644
index 0000000..1a4f8d8
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.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.block_app.framework.api.giraph;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.internal.BlockWorkerContextLogic;
+import org.apache.giraph.block_app.framework.internal.BlockWorkerPieces;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.writable.kryo.HadoopKryo;
+import org.apache.giraph.writable.kryo.markers.KryoIgnoreWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+/**
+ * WorkerContext that executes receiver and sender blocks passed
+ * into BlockWorkerPieces.
+ */
+public final class BlockWorkerContext extends WorkerContext
+    implements KryoIgnoreWritable {
+  public static final Logger LOG = Logger.getLogger(BlockWorkerContext.class);
+
+  private BlockWorkerContextLogic workerLogic;
+
+  @Override
+  public void preApplication()
+      throws InstantiationException, IllegalAccessException {
+    workerLogic = new BlockWorkerContextLogic();
+    workerLogic.preApplication(new BlockWorkerContextApiWrapper<>(this),
+        new BlockOutputHandle(getContext().getJobID().toString(),
+            getConf(), getContext()));
+  }
+
+  @Override
+  public void preSuperstep() {
+    List<Writable> messages = getAndClearMessagesFromOtherWorkers();
+    BlockWorkerContextApiWrapper<Writable> workerApi =
+        new BlockWorkerContextApiWrapper<>(this);
+    BlockWorkerPieces<Object> workerPieces =
+        BlockWorkerPieces.getNextWorkerPieces(this);
+
+    LOG.info("PassedComputation in " + getSuperstep() +
+        " superstep executing " + workerPieces);
+
+    workerLogic.preSuperstep(
+        workerApi, workerApi, workerPieces, getSuperstep(), messages);
+  }
+
+  @Override
+  public void postSuperstep() {
+    workerLogic.postSuperstep();
+  }
+
+  @Override
+  public void postApplication() {
+    workerLogic.postApplication();
+  }
+
+  public Object getWorkerValue() {
+    return workerLogic.getWorkerValue();
+  }
+
+  public BlockOutputHandle getOutputHandle() {
+    return workerLogic.getOutputHandle();
+  }
+
+  // Cannot extend KryoWritable directly, since WorkerContext is
+  // abstract class, not interface... Additionally conf in parent
+  // class cannot be made transient.
+  // So just add serialization of two individual fields.
+  // (and adding KryoIgnoreWritable to avoid wrapping it twice)
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    HadoopKryo.writeClassAndObject(out, workerLogic);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    workerLogic = HadoopKryo.readClassAndObject(in);
+    workerLogic.getOutputHandle().initialize(getConf(), getContext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java
new file mode 100644
index 0000000..c52b6a5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java
@@ -0,0 +1,84 @@
+/*
+ * 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.block_app.framework.api.giraph;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Giraph implementation of BlockWorkerContextReceiveApi and
+ * BlockWorkerContextSendApi, passing all calls to WorkerContext.
+ *
+ * @param <WM> Worker message type
+ */
+final class BlockWorkerContextApiWrapper<WM extends Writable>
+    implements BlockWorkerContextReceiveApi, BlockWorkerContextSendApi<WM> {
+  private final WorkerContext workerContext;
+
+  public BlockWorkerContextApiWrapper(WorkerContext workerContext) {
+    this.workerContext = workerContext;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<?, ?, ?> getConf() {
+    return workerContext.getConf();
+  }
+
+  @Override
+  public int getWorkerCount() {
+    return workerContext.getWorkerCount();
+  }
+
+  @Override
+  public int getMyWorkerIndex() {
+    return workerContext.getMyWorkerIndex();
+  }
+
+  @Override
+  public <A extends Writable> A getAggregatedValue(String name) {
+    return workerContext.getAggregatedValue(name);
+  }
+
+  @Override
+  public <A extends Writable> void aggregate(String name, A value) {
+    workerContext.aggregate(name, value);
+  }
+
+  @Override
+  public void sendMessageToWorker(WM message, int workerIndex) {
+    workerContext.sendMessageToWorker(message, workerIndex);
+  }
+
+  @Override
+  public <B extends Writable> B getBroadcast(String name) {
+    return workerContext.getBroadcast(name);
+  }
+
+  @Override
+  public long getTotalNumEdges() {
+    return workerContext.getTotalNumEdges();
+  }
+
+  @Override
+  public long getTotalNumVertices() {
+    return workerContext.getTotalNumVertices();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java
new file mode 100644
index 0000000..e20fb8e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/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.
+ */
+/**
+ * Giraph implementation of graph processing system API used by
+ * Blocks Framework.
+ */
+package org.apache.giraph.block_app.framework.api.giraph;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java
new file mode 100644
index 0000000..c10e5d2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+/**
+ * Interfaces representing full API to the underlying graph processing system.
+ *
+ * Framework implementation is fully contained within package
+ * org.apache.giraph.block_app.framework, given implementation of interfaces
+ * defined here.
+ *
+ * He have two such implementations:
+ * - one relying on Giraph, distributed graph processing system,
+ *   connecting all methods to it's internals
+ * - one having a fully contained local implementation, executing applications
+ *   on a single machine. Avoiding overheads of Giraph being distributed,
+ *   it allows very efficient evaluation on small graphs, especially useful for
+ *   fast unit tests.
+ *
+ * You could potentially use a different graph processing system, to execute
+ * any Block Application, by implementing these interfaces.
+ */
+package org.apache.giraph.block_app.framework.api;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java
new file mode 100644
index 0000000..6d5287c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.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.block_app.framework.block;
+
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+
+/**
+ * Composable unit of execution. Used to combine other Blocks into
+ * bigger units. Each Piece represents a Block itself.
+ *
+ * Execution is represented as an iterator across Pieces.
+ *
+ * The whole application run is represented by a single block at the end.
+ */
+@SuppressWarnings("rawtypes")
+public interface Block extends Iterable<AbstractPiece> {
+  /**
+   * Create iterator representing all pieces needed to be executed
+   * in this block.
+   *
+   * After Iterator.next call returns, master compute of returned Piece is
+   * guaranteed to be called before calling hasNext/next on the iterator.
+   * (allows for iterators logic to depend on the execution dynamically,
+   * and not be only static)
+   */
+  @Override
+  Iterator<AbstractPiece> iterator();
+
+  /**
+   * Calls consumer for each Piece:
+   * - in no particular order
+   * - potentially calling multiple times on same Piece
+   * - even if Piece might never be returned in the iterator
+   * - it will be called at least once for every piece that is
+   *   going to be returned by iterator
+   *
+   * Can be used for static analysis/introspection of the block,
+   * without actually executing them.
+   */
+  void forAllPossiblePieces(Consumer<AbstractPiece> consumer);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java
new file mode 100644
index 0000000..1a57402
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.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.block_app.framework.block;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+
+/**
+ * Block without any pieces
+ */
+@SuppressWarnings("rawtypes")
+public final class EmptyBlock implements Block {
+  @Override
+  public Iterator<AbstractPiece> iterator() {
+    return Collections.emptyIterator();
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java
new file mode 100644
index 0000000..5631417
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java
@@ -0,0 +1,113 @@
+/*
+ * 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.block_app.framework.block;
+
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.delegate.FilteringPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.vertex.SupplierFromVertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
+
+/**
+ * Block which filters out calls to vertexSend/vertexReceive functions
+ * of all pieces in a given block.
+ * Filtering happens based on toCallSend and toCallReceive suppliers
+ * that are passed in, as every piece is just wrapped with FilteringPiece.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public final class FilteringBlock<I extends WritableComparable,
+        V extends Writable, E extends Writable>
+    implements Block {
+  private final SupplierFromVertex<I, V, E, Boolean> toCallSend;
+  private final SupplierFromVertex<I, V, E, Boolean> toCallReceive;
+  private final Block block;
+
+  /**
+   * Creates filtering block which uses passed {@code toCallSend} to filter
+   * calls to {@code vertexSend}, and passed {@code toCallReceive} to filter
+   * calls to {@code vertexReceive}, on all pieces within passed {@code block}.
+   */
+  public FilteringBlock(
+      SupplierFromVertex<I, V, E, Boolean> toCallSend,
+      SupplierFromVertex<I, V, E, Boolean> toCallReceive,
+      Block block) {
+    this.toCallSend = toCallSend;
+    this.toCallReceive = toCallReceive;
+    this.block = block;
+  }
+
+  /**
+   * Creates filtering block, where both vertexSend and vertexReceive is
+   * filtered based on same supplier.
+   */
+  public FilteringBlock(
+      SupplierFromVertex<I, V, E, Boolean> toCallSendAndReceive, Block block) {
+    this(toCallSendAndReceive, toCallSendAndReceive, block);
+  }
+
+  /**
+   * Creates filtering block, that filters only vertexReceive function,
+   * and always calls vertexSend function.
+   */
+  public static
+  <I extends WritableComparable, V extends Writable, E extends Writable>
+  Block createReceiveFiltering(
+      SupplierFromVertex<I, V, E, Boolean> toCallReceive,
+      Block innerBlock) {
+    return new FilteringBlock<>(null, toCallReceive, innerBlock);
+  }
+
+  /**
+   * Creates filtering block, that filters only vertexSend function,
+   * and always calls vertexReceive function.
+   */
+  public static
+  <I extends WritableComparable, V extends Writable, E extends Writable>
+  Block createSendFiltering(
+      SupplierFromVertex<I, V, E, Boolean> toCallSend,
+      Block innerBlock) {
+    return new FilteringBlock<>(toCallSend, null, innerBlock);
+  }
+
+  @Override
+  public Iterator<AbstractPiece> iterator() {
+    return Iterators.transform(
+        block.iterator(),
+        new Function<AbstractPiece, AbstractPiece>() {
+          @Override
+          public AbstractPiece apply(AbstractPiece input) {
+            return new FilteringPiece<>(toCallSend, toCallReceive, input);
+          }
+        });
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    block.forAllPossiblePieces(consumer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java
new file mode 100644
index 0000000..e73392d
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.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.block_app.framework.block;
+
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.Supplier;
+
+/**
+ * Block that executes one of two branches based on a condition
+ */
+@SuppressWarnings("rawtypes")
+public final class IfBlock implements Block {
+  private final Block thenBlock;
+  private final Block elseBlock;
+  private final Supplier<Boolean> condition;
+
+  public IfBlock(
+      Supplier<Boolean> condition, Block thenBlock, Block elseBlock) {
+    this.condition = condition;
+    this.thenBlock = thenBlock;
+    this.elseBlock = elseBlock;
+  }
+
+  public IfBlock(Supplier<Boolean> condition, Block thenBlock) {
+    this.condition = condition;
+    this.thenBlock = thenBlock;
+    this.elseBlock = new EmptyBlock();
+  }
+
+  @Override
+  public Iterator<AbstractPiece> iterator() {
+    if (Boolean.TRUE.equals(condition.get())) {
+      return thenBlock.iterator();
+    } else {
+      return elseBlock.iterator();
+    }
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    thenBlock.forAllPossiblePieces(consumer);
+    elseBlock.forAllPossiblePieces(consumer);
+  }
+
+  @Override
+  public String toString() {
+    if (elseBlock instanceof EmptyBlock) {
+      return "IfBlock(" + thenBlock + ")";
+    }
+    return "IfBlock(" + thenBlock + " , " + elseBlock + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java
new file mode 100644
index 0000000..9f4f4a0
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.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.block_app.framework.block;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.primitive.IntSupplier;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Block that repeats another block given number of times.
+ */
+@SuppressWarnings("rawtypes")
+public final class RepeatBlock implements Block {
+  private final Block block;
+  private final IntSupplier repeatTimes;
+
+  public RepeatBlock(final int repeatTimes, Block block) {
+    this.block = block;
+    this.repeatTimes = new IntSupplier() {
+      @Override
+      public int get() {
+        return repeatTimes;
+      }
+    };
+  }
+
+  /**
+   * Creates a repeat block, that before starting execution takes number of
+   * iterations from the given supplier.
+   *
+   * This allows number of iterations to be dynamic, and depend on
+   * execution that happens before.
+   * Note - it doesn't allow for number of repetitions to change during the
+   * loop itself - as it is supplier is called only when this block gets
+   * its turn.
+   */
+  public RepeatBlock(IntSupplier repeatTimes, Block block) {
+    this.block = block;
+    this.repeatTimes = repeatTimes;
+  }
+
+  /**
+   * Create a repeat block that executes unlimited number of times.
+   *
+   * Should rarely be used, as it will cause application never to finish,
+   * unless other unconventional ways of termination are used.
+   */
+  public static Block unlimited(Block block) {
+    return new RepeatBlock(Integer.MAX_VALUE, block);
+  }
+
+  @Override
+  public Iterator<AbstractPiece> iterator() {
+    return Iterables.concat(
+        Collections.nCopies(repeatTimes.get(), block)).iterator();
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    block.forAllPossiblePieces(consumer);
+  }
+
+  @Override
+  public String toString() {
+    return "RepeatBlock(" + repeatTimes + " * " + block + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java
new file mode 100644
index 0000000..13e8833
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java
@@ -0,0 +1,83 @@
+/*
+ * 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.block_app.framework.block;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.Supplier;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+
+/**
+ * Block that repeats another block until toQuit supplier returns true,
+ * but at most given number of times.
+ *
+ * If toQuit returns true on first run, block is not going
+ * to be executed at all.
+ */
+@SuppressWarnings("rawtypes")
+public final class RepeatUntilBlock implements Block {
+  private final Block block;
+  private final int repeatTimes;
+  private final Supplier<Boolean> toQuit;
+
+  public RepeatUntilBlock(
+      int repeatTimes, Block block, Supplier<Boolean> toQuit) {
+    this.block = block;
+    this.repeatTimes = repeatTimes;
+    this.toQuit = toQuit;
+  }
+
+  /**
+   * Repeat unlimited number of times, until toQuit supplier returns true.
+   */
+  public static Block unlimited(Block block, Supplier<Boolean> toQuit) {
+    return new RepeatUntilBlock(Integer.MAX_VALUE, block, toQuit);
+  }
+
+  @Override
+  public Iterator<AbstractPiece> iterator() {
+    // nCopies uses constant memory, creating a looped list with single element
+    final Iterator<AbstractPiece> repeatIterator =
+        Iterables.concat(Collections.nCopies(repeatTimes, block)).iterator();
+    return new AbstractIterator<AbstractPiece>() {
+      @Override
+      protected AbstractPiece computeNext() {
+        if (Boolean.TRUE.equals(toQuit.get()) || !repeatIterator.hasNext()) {
+          return endOfData();
+        }
+
+        return repeatIterator.next();
+      }
+    };
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    block.forAllPossiblePieces(consumer);
+  }
+
+  @Override
+  public String toString() {
+    return "RepeatUntilBlock(" + repeatTimes + " * " + block + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java
new file mode 100644
index 0000000..d768f0b
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java
@@ -0,0 +1,60 @@
+/*
+ * 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.block_app.framework.block;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Block that executes provided blocks sequentially.
+ */
+@SuppressWarnings("rawtypes")
+public final class SequenceBlock implements Block {
+  private final Block[] blocks;
+
+  public SequenceBlock(Block... blocks) {
+    this.blocks = blocks.clone();
+  }
+
+  public SequenceBlock(List<? extends Block> blocks) {
+    this.blocks = blocks.toArray(new Block[blocks.size()]);
+  }
+
+  @Override
+  public Iterator<AbstractPiece> iterator() {
+    return Iterables.concat(Arrays.asList(blocks)).iterator();
+  }
+
+  @Override
+  public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+    for (Block block : blocks) {
+      block.forAllPossiblePieces(consumer);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "SequenceBlock" + Arrays.toString(blocks);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java
new file mode 100644
index 0000000..64adc35
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/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.
+ */
+/**
+ * Block interface as a composable unit of execution, and its common
+ * implementations.
+ */
+package org.apache.giraph.block_app.framework.block;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java
new file mode 100644
index 0000000..6a2fb39
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java
@@ -0,0 +1,79 @@
+/*
+ * 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.block_app.framework.internal;
+
+import java.lang.reflect.Field;
+
+import org.apache.giraph.block_app.framework.api.StatusReporter;
+
+/** Utility class for Blocks Framework related counters */
+public class BlockCounters {
+  public static final String GROUP = "Blocks Framework";
+
+  private BlockCounters() { }
+
+  /**
+   * Takes all fields from stage object, and puts them into counters,
+   * if possible.
+   * Only fields that are convertible to long via widening are set
+   * (i.e. long/int/short/byte)
+   */
+  public static void setStageCounters(
+      String prefix, Object stage, StatusReporter reporter) {
+    if (stage != null && reporter != null) {
+      Class<?> clazz = stage.getClass();
+
+      while (clazz != null) {
+        Field[] fields = clazz.getDeclaredFields();
+
+        Field.setAccessible(fields, true);
+        for (Field field : fields) {
+          try {
+            long value = field.getLong(stage);
+            reporter.getCounter(
+                GROUP, prefix + field.getName()).setValue(value);
+
+          // CHECKSTYLE: stop EmptyBlock - ignore any exceptions
+          } catch (IllegalArgumentException | IllegalAccessException e) {
+          }
+          // CHECKSTYLE: resume EmptyBlock
+        }
+        clazz = clazz.getSuperclass();
+      }
+    }
+  }
+
+  public static void setMasterTimeCounter(
+      PairedPieceAndStage<?> masterPiece, long superstep,
+      long millis, StatusReporter reporter) {
+    reporter.getCounter(
+        GROUP + " Master Timers",
+        String.format(
+            "In %6.1f %s (s)", superstep - 0.5, masterPiece.getPiece())
+    ).setValue(millis / 1000);
+  }
+
+  public static void setWorkerTimeCounter(
+      BlockWorkerPieces<?> workerPieces, long superstep,
+      long millis, StatusReporter reporter) {
+    reporter.getCounter(
+        GROUP + " Worker Timers",
+        String.format("In %6d %s (s)", superstep, workerPieces.toStringShort())
+    ).setValue(millis / 1000);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java
new file mode 100644
index 0000000..3b87372
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java
@@ -0,0 +1,173 @@
+/*
+ * 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.block_app.framework.internal;
+
+import java.util.HashSet;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.BlockFactory;
+import org.apache.giraph.block_app.framework.BlockUtils;
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockOutputHandleAccessor;
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.function.Consumer;
+import org.apache.log4j.Logger;
+import org.python.google.common.base.Preconditions;
+
+/**
+ * Block execution logic on master, iterating over Pieces of the
+ * application Block, executing master logic, and providing what needs to be
+ * executed on the workers.
+ *
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class BlockMasterLogic<S> {
+  private static final Logger LOG = Logger.getLogger(BlockMasterLogic.class);
+
+  private Iterator<AbstractPiece> pieceIterator;
+  private PairedPieceAndStage<S> previousPiece;
+  private transient BlockMasterApi masterApi;
+  private long lastTimestamp = -1;
+  private BlockWorkerPieces previousWorkerPieces;
+  private boolean computationDone;
+
+  public void initialize(
+      GiraphConfiguration conf, final BlockMasterApi masterApi)
+    throws InstantiationException, IllegalAccessException {
+    this.masterApi = masterApi;
+    this.computationDone = false;
+
+    BlockFactory<S> factory = BlockUtils.createBlockFactory(conf);
+    Block executionBlock = factory.createBlock(conf);
+    LOG.info("Executing application - " + executionBlock);
+
+    // We register all possible aggregators at the beginning
+    executionBlock.forAllPossiblePieces(new Consumer<AbstractPiece>() {
+      private final HashSet<AbstractPiece> registeredPieces = new HashSet<>();
+      @SuppressWarnings("deprecation")
+      @Override
+      public void apply(AbstractPiece piece) {
+        // no need to regiser the same piece twice.
+        if (registeredPieces.add(piece)) {
+          try {
+            piece.registerAggregators(masterApi);
+          } catch (InstantiationException | IllegalAccessException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      }
+    });
+
+    pieceIterator = executionBlock.iterator();
+    // Invariant is that ReceiveWorkerPiece of previousPiece has already been
+    // executed and that previousPiece.nextExecutionStage() should be used for
+    // iterating. So passing piece as null, and initial state as current state,
+    // so that nothing get's executed in first half, and calculateNextState
+    // returns initial state.
+    previousPiece = new PairedPieceAndStage<>(
+        null, factory.createExecutionStage(conf));
+  }
+
+  /**
+   * Initialize object after deserializing it.
+   * BlockMasterApi is not serializable, so it is transient, and set via this
+   * method afterwards.
+   */
+  public void initializeAfterRead(BlockMasterApi masterApi) {
+    this.masterApi = masterApi;
+  }
+
+  /**
+   * Executes operations on master (master compute and registering reducers),
+   * and calculates next pieces to be exectued on workers.
+   *
+   * @param superstep Current superstep
+   * @return Next BlockWorkerPieces to be executed on workers, or null
+   *         if computation should be halted.
+   */
+  public BlockWorkerPieces<S> computeNext(long superstep) {
+    long beforeMaster = System.currentTimeMillis();
+    if (lastTimestamp != -1) {
+      BlockCounters.setWorkerTimeCounter(
+          previousWorkerPieces, superstep - 1,
+          beforeMaster - lastTimestamp, masterApi);
+    }
+
+    if (previousPiece == null) {
+      postApplication();
+      return null;
+    } else {
+      LOG.info(
+          "Master executing " + previousPiece + ", in superstep " + superstep);
+      previousPiece.masterCompute(masterApi);
+      ((BlockOutputHandleAccessor) masterApi).getBlockOutputHandle().
+          returnAllWriters();
+      long afterMaster = System.currentTimeMillis();
+
+      if (previousPiece.getPiece() != null) {
+        BlockCounters.setMasterTimeCounter(
+            previousPiece, superstep, afterMaster - beforeMaster, masterApi);
+      }
+
+      PairedPieceAndStage<S> nextPiece;
+      if (pieceIterator.hasNext()) {
+        nextPiece = new PairedPieceAndStage<S>(
+            pieceIterator.next(), previousPiece.nextExecutionStage());
+        nextPiece.registerReducers(masterApi);
+      } else {
+        nextPiece = null;
+      }
+      BlockCounters.setStageCounters(
+          "Master finished stage: ", previousPiece.getExecutionStage(),
+          masterApi);
+      LOG.info(
+          "Master passing next " + nextPiece + ", in superstep " + superstep);
+
+      // if there is nothing more to compute, no need for additional superstep
+      // this can only happen if application uses no pieces.
+      BlockWorkerPieces<S> result;
+      if (previousPiece.getPiece() == null && nextPiece == null) {
+        postApplication();
+        result = null;
+      } else {
+        result = new BlockWorkerPieces<>(previousPiece, nextPiece);
+        LOG.info("Master in " + superstep + " superstep passing " +
+            result + " to be executed");
+      }
+
+      previousPiece = nextPiece;
+      lastTimestamp = afterMaster;
+      previousWorkerPieces = result;
+      return result;
+    }
+  }
+
+  /**
+   * Clean up any master state, after application has finished.
+   */
+  private void postApplication() {
+    ((BlockOutputHandleAccessor) masterApi).getBlockOutputHandle().
+        closeAllWriters();
+    Preconditions.checkState(!computationDone);
+    computationDone = true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java
new file mode 100644
index 0000000..8b8e174
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java
@@ -0,0 +1,91 @@
+/*
+ * 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.block_app.framework.internal;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.BlockUtils;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+/**
+ * Block execution logic on WorkerContext.
+ */
+@SuppressWarnings({ "rawtypes" })
+public class BlockWorkerContextLogic {
+  public static final Logger LOG =
+      Logger.getLogger(BlockWorkerContextLogic.class);
+
+  private Object workerValue;
+  private BlockWorkerPieces workerPieces;
+  private BlockOutputHandle outputHandle;
+
+  private transient BlockWorkerContextSendApi sendApi;
+
+  public BlockWorkerContextLogic() {
+  }
+
+  public void preApplication(BlockWorkerContextApi api,
+      BlockOutputHandle outputHandle) {
+    workerValue =
+        BlockUtils.BLOCK_WORKER_CONTEXT_VALUE_CLASS.newInstance(api.getConf());
+    this.outputHandle = outputHandle;
+  }
+
+  public Object getWorkerValue() {
+    return workerValue;
+  }
+
+  public BlockOutputHandle getOutputHandle() {
+    return outputHandle;
+  }
+
+  @SuppressWarnings("unchecked")
+  public void preSuperstep(
+      BlockWorkerContextReceiveApi receiveApi,
+      BlockWorkerContextSendApi sendApi,
+      BlockWorkerPieces workerPieces, long superstep,
+      List<Writable> messages) {
+    LOG.info("Worker executing " + workerPieces + " in " + superstep +
+        " superstep");
+    this.sendApi = sendApi;
+    this.workerPieces = workerPieces;
+    if (workerPieces.getReceiver() != null) {
+      workerPieces.getReceiver().workerContextReceive(
+          receiveApi, workerValue, messages);
+    }
+  }
+
+  public void postSuperstep() {
+    if (workerPieces.getSender() != null) {
+      workerPieces.getSender().workerContextSend(sendApi, workerValue);
+    }
+    workerPieces = null;
+    sendApi = null;
+    outputHandle.returnAllWriters();
+  }
+
+  public void postApplication() {
+    outputHandle.closeAllWriters();
+    // TODO add support through conf for postApplication, if needed.
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java
new file mode 100644
index 0000000..844160c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.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.block_app.framework.internal;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece.InnerVertexSender;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.graph.Vertex;
+
+/**
+ * Block execution logic on workers.
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BlockWorkerLogic {
+  private final BlockWorkerPieces pieces;
+
+  private transient VertexReceiver receiveFunctions;
+  private transient InnerVertexSender sendFunctions;
+
+  public BlockWorkerLogic(BlockWorkerPieces pieces) {
+    this.pieces = pieces;
+  }
+
+  public void preSuperstep(
+      BlockWorkerReceiveApi receiveApi, BlockWorkerSendApi sendApi) {
+    if (pieces.getReceiver() != null) {
+      receiveFunctions = pieces.getReceiver().getVertexReceiver(receiveApi);
+    }
+    if (pieces.getSender() != null) {
+      sendFunctions = pieces.getSender().getVertexSender(sendApi);
+    }
+  }
+
+  public void compute(Vertex vertex, Iterable messages) {
+    if (receiveFunctions != null) {
+      receiveFunctions.vertexReceive(vertex, messages);
+    }
+    if (sendFunctions != null) {
+      sendFunctions.vertexSend(vertex);
+    }
+  }
+
+  public void postSuperstep() {
+    if (receiveFunctions instanceof VertexPostprocessor) {
+      ((VertexPostprocessor) receiveFunctions).postprocess();
+    }
+    if (sendFunctions != null) {
+      sendFunctions.postprocess();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java
new file mode 100644
index 0000000..3b38cfa
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java
@@ -0,0 +1,180 @@
+/*
+ * 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.block_app.framework.internal;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Objects;
+
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.DefaultMessageClasses;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.factories.DefaultMessageValueFactory;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.types.NoMessage;
+import org.apache.giraph.utils.UnsafeReusableByteArrayInput;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
+import org.apache.giraph.writable.kryo.KryoWritableWrapper;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+/**
+ * Pair of pieces to be executed on workers in a superstep
+ *
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BlockWorkerPieces<S> {
+  private static final Logger LOG = Logger.getLogger(BlockWorkerPieces.class);
+
+  /** Aggregator holding next worker computation */
+  private static final
+  String NEXT_WORKER_PIECES = "giraph.blocks.next_worker_pieces";
+
+  private final PairedPieceAndStage<S> receiver;
+  private final PairedPieceAndStage<S> sender;
+
+  public BlockWorkerPieces(
+      PairedPieceAndStage<S> receiver, PairedPieceAndStage<S> sender) {
+    this.receiver = receiver;
+    this.sender = sender;
+  }
+
+  public PairedPieceAndStage<S> getReceiver() {
+    return receiver;
+  }
+
+  public PairedPieceAndStage<S> getSender() {
+    return sender;
+  }
+
+  public MessageClasses getOutgoingMessageClasses(
+      ImmutableClassesGiraphConfiguration conf) {
+    MessageClasses messageClasses;
+    if (sender == null || sender.getPiece() == null) {
+      messageClasses = new DefaultMessageClasses(
+          NoMessage.class,
+          DefaultMessageValueFactory.class,
+          null,
+          MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION);
+    } else {
+      messageClasses = sender.getPiece().getMessageClasses(conf);
+    }
+
+    messageClasses.verifyConsistent(conf);
+    return messageClasses;
+  }
+
+  @Override
+  public String toString() {
+    return "[receiver=" + receiver + ",sender=" + sender + "]";
+  }
+
+  public String toStringShort() {
+    String receiverString =
+        Objects.toString(receiver != null ? receiver.getPiece() : null);
+    String senderString =
+        Objects.toString(sender != null ? sender.getPiece() : null);
+    if (receiverString.equals(senderString)) {
+      return "[receiver&sender=" + receiverString + "]";
+    } else {
+      return "[receiver=" + receiverString + ",sender=" + senderString + "]";
+    }
+  }
+
+  /**
+   * Sets which WorkerComputation is going to be executed in the next superstep.
+   */
+  public static <S> void setNextWorkerPieces(
+      MasterCompute master, BlockWorkerPieces<S> nextWorkerPieces) {
+    Writable toBroadcast = new KryoWritableWrapper<>(nextWorkerPieces);
+    byte[] data = WritableUtils.toByteArrayUnsafe(toBroadcast);
+
+    // TODO: extract splitting logic into common utility
+    int overhead = 4096;
+    int singleSize = Math.max(
+        overhead,
+        GiraphConstants.MAX_MSG_REQUEST_SIZE.get(master.getConf()) - overhead);
+
+    ArrayList<byte[]> splittedData = new ArrayList<>();
+    if (data.length < singleSize) {
+      splittedData.add(data);
+    } else {
+      for (int start = 0; start < data.length; start += singleSize) {
+        splittedData.add(Arrays.copyOfRange(
+            data, start, Math.min(data.length, start + singleSize)));
+      }
+    }
+
+    LOG.info("Next worker piece - total serialized size: " + data.length +
+        ", split into " + splittedData.size());
+    master.getContext().getCounter(
+        "PassedWorker Stats", "total serialized size")
+        .increment(data.length);
+    master.getContext().getCounter(
+        "PassedWorker Stats", "split parts")
+        .increment(splittedData.size());
+
+    master.broadcast(NEXT_WORKER_PIECES, new IntWritable(splittedData.size()));
+
+    for (int i = 0; i < splittedData.size(); i++) {
+      master.broadcast(NEXT_WORKER_PIECES + "_part_" + i,
+          KryoWritableWrapper.wrapIfNeeded(splittedData.get(i)));
+    }
+
+    master.setOutgoingMessageClasses(
+        nextWorkerPieces.getOutgoingMessageClasses(master.getConf()));
+  }
+
+  public static <S> BlockWorkerPieces<S> getNextWorkerPieces(
+      WorkerGlobalCommUsage worker) {
+    int splits = worker.<IntWritable>getBroadcast(NEXT_WORKER_PIECES).get();
+
+    int totalLength = 0;
+    ArrayList<byte[]> splittedData = new ArrayList<>();
+    for (int i = 0; i < splits; i++) {
+      byte[] cur = KryoWritableWrapper.<byte[]>unwrapIfNeeded(
+          worker.getBroadcast(NEXT_WORKER_PIECES + "_part_" + i));
+      splittedData.add(cur);
+      totalLength += cur.length;
+    }
+
+    byte[] merged;
+    if (splits == 1) {
+      merged = splittedData.get(0);
+    } else {
+      merged = new byte[totalLength];
+      int index = 0;
+      for (int i = 0; i < splits; i++) {
+        System.arraycopy(
+            splittedData.get(i), 0, merged, index, splittedData.get(i).length);
+        index += splittedData.get(i).length;
+      }
+    }
+
+    KryoWritableWrapper<BlockWorkerPieces<S>> wrapper =
+        new KryoWritableWrapper<>();
+    WritableUtils.fromByteArrayUnsafe(
+        merged, wrapper, new UnsafeReusableByteArrayInput());
+    return wrapper.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java
new file mode 100644
index 0000000..90fe83e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java
@@ -0,0 +1,111 @@
+/*
+ * 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.block_app.framework.internal;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece.InnerVertexSender;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Object holding piece with it's corresponding execution stage.
+ *
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class PairedPieceAndStage<S> {
+  private final AbstractPiece piece;
+  private final S executionStage;
+
+  public PairedPieceAndStage(AbstractPiece piece, S executionStage) {
+    this.piece = piece;
+    this.executionStage = executionStage;
+  }
+
+  public S nextExecutionStage() {
+    // if piece is null, then it cannot change the execution stage
+    return piece != null ?
+      (S) piece.nextExecutionStage(executionStage) : executionStage;
+  }
+
+  public S getExecutionStage() {
+    return executionStage;
+  }
+
+  public void registerReducers(BlockMasterApi masterApi) {
+    if (piece != null) {
+      piece.wrappedRegisterReducers(masterApi, executionStage);
+    }
+  }
+
+  public InnerVertexSender getVertexSender(BlockWorkerSendApi sendApi) {
+    if (piece != null) {
+      return piece.getWrappedVertexSender(sendApi, executionStage);
+    }
+    return null;
+  }
+
+  public void masterCompute(BlockMasterApi masterApi) {
+    if (piece != null) {
+      piece.masterCompute(masterApi, executionStage);
+    }
+  }
+
+  public VertexReceiver getVertexReceiver(
+      BlockWorkerReceiveApi receiveApi) {
+    if (piece != null) {
+      return piece.getVertexReceiver(receiveApi, executionStage);
+    }
+    return null;
+  }
+
+  public void workerContextSend(
+      BlockWorkerContextSendApi workerContextApi, Object workerValue) {
+    if (piece != null) {
+      piece.workerContextSend(workerContextApi, executionStage, workerValue);
+    }
+  }
+
+  public void workerContextReceive(
+      BlockWorkerContextReceiveApi workerContextApi,
+      Object workerValue, List<Writable> workerMessages) {
+    if (piece != null) {
+      piece.workerContextReceive(
+          workerContextApi, executionStage, workerValue, workerMessages);
+    }
+  }
+
+  /**
+   * @return the piece
+   */
+  public AbstractPiece getPiece() {
+    return piece;
+  }
+
+  @Override
+  public String toString() {
+    return "Piece " + piece + " in stage " + executionStage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java
new file mode 100644
index 0000000..3ebe8f7
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/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.
+ */
+/**
+ * Implementation of execution logic, guiding internal execution of
+ * Block Application.
+ */
+package org.apache.giraph.block_app.framework.internal;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java
new file mode 100644
index 0000000..6f2a3dd
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.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.block_app.framework.output;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Output description
+ *
+ * @param <OW> Writer type
+ */
+public interface BlockOutputDesc<OW extends BlockOutputWriter> {
+  /**
+   * Initialize output and perform any necessary checks
+   *
+   * @param jobIdentifier Unique identifier of the job
+   * @param conf Configuration
+   */
+  void initializeAndCheck(String jobIdentifier, Configuration conf);
+
+  /**
+   * Create writer
+   *
+   * @param conf Configuration
+   * @param hadoopProgressable Progressable to call progress on
+   * @return Writer
+   */
+  OW createOutputWriter(Configuration conf, Progressable hadoopProgressable);
+
+  /**
+   * Commit everything
+   */
+  void commit();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java
new file mode 100644
index 0000000..818a311
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java
@@ -0,0 +1,107 @@
+/*
+ * 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.block_app.framework.output;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.giraph.bsp.BspOutputFormat;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.StrConfOption;
+import org.apache.giraph.utils.ConfigurationObjectUtils;
+import org.apache.giraph.utils.DefaultOutputCommitter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * Hadoop output format to use with block output.
+ * It keeps track of all registered outputs, and knows how to create them.
+ */
+public class BlockOutputFormat extends BspOutputFormat {
+  private static final StrConfOption OUTPUT_CONF_OPTIONS = new StrConfOption(
+      "digraph.outputConfOptions", "",
+      "List of conf options for outputs used");
+
+  public static <OD> void addOutputDesc(OD outputDesc, String confOption,
+      GiraphConfiguration conf) {
+    GiraphConstants.HADOOP_OUTPUT_FORMAT_CLASS.set(conf,
+        BlockOutputFormat.class);
+    String currentOutputs = OUTPUT_CONF_OPTIONS.get(conf);
+    if (!currentOutputs.isEmpty()) {
+      currentOutputs = currentOutputs + ",";
+    }
+    OUTPUT_CONF_OPTIONS.set(conf, currentOutputs + confOption);
+    ConfigurationObjectUtils.setObjectKryo(outputDesc, confOption, conf);
+  }
+
+  private static String[] getOutputConfOptions(Configuration conf) {
+    String outputConfOptions = OUTPUT_CONF_OPTIONS.get(conf);
+    return outputConfOptions.isEmpty() ?
+        new String[0] : outputConfOptions.split(",");
+  }
+
+  public static <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+  OD createInitAndCheckOutputDesc(String confOption, Configuration conf,
+      String jobIdentifier) {
+    OD outputDesc = ConfigurationObjectUtils.getObjectKryo(confOption, conf);
+    outputDesc.initializeAndCheck(jobIdentifier, conf);
+    return outputDesc;
+  }
+
+  public static Map<String, BlockOutputDesc>
+  createInitAndCheckOutputDescsMap(Configuration conf, String jobIdentifier) {
+    String[] outputConfOptions = getOutputConfOptions(conf);
+    Map<String, BlockOutputDesc> ret = new HashMap<>(outputConfOptions.length);
+    for (String outputConfOption : outputConfOptions) {
+      ret.put(outputConfOption,
+          createInitAndCheckOutputDesc(outputConfOption, conf, jobIdentifier));
+    }
+    return ret;
+  }
+
+  public static Map<String, BlockOutputDesc> createInitAndCheckOutputDescsMap(
+      JobContext jobContext) {
+    return createInitAndCheckOutputDescsMap(jobContext.getConfiguration(),
+        jobContext.getJobID().toString());
+  }
+
+  @Override
+  public void checkOutputSpecs(JobContext jobContext)
+      throws IOException, InterruptedException {
+    createInitAndCheckOutputDescsMap(jobContext);
+  }
+
+  @Override
+  public OutputCommitter getOutputCommitter(
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    return new DefaultOutputCommitter() {
+      @Override
+      public void commit(JobContext jobContext) throws IOException {
+        Map<String, BlockOutputDesc> map =
+            createInitAndCheckOutputDescsMap(jobContext);
+        for (BlockOutputDesc outputDesc : map.values()) {
+          outputDesc.commit();
+        }
+      }
+    };
+  }
+}


[2/5] git commit: updated refs/heads/trunk to 819d6d3

Posted by ik...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java
new file mode 100644
index 0000000..f06dd89
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java
@@ -0,0 +1,250 @@
+/*
+ * 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.block_app.framework.piece.global_comm.internal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.reducers.Reducer;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+import org.apache.giraph.worker.WorkerReduceUsage;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * All logic for transforming Giraph's reducer API to reducer handles.
+ * Contains state of active reducers, and is kept within a Piece.
+ */
+public class ReducersForPieceHandler implements VertexSenderObserver {
+  private static final AtomicInteger HANDLER_COUNTER = new AtomicInteger();
+  private static final AtomicInteger BROADCAST_COUNTER = new AtomicInteger();
+
+  private final int handleIndex = HANDLER_COUNTER.incrementAndGet();
+  private final AtomicInteger reduceCounter = new AtomicInteger();
+
+  private final ArrayList<VertexSenderObserver> observers = new ArrayList<>();
+
+  @Override
+  public void vertexSenderWorkerPreprocess(WorkerReduceUsage usage) {
+    for (VertexSenderObserver observer : observers) {
+      observer.vertexSenderWorkerPreprocess(usage);
+    }
+  }
+
+  @Override
+  public void vertexSenderWorkerPostprocess(WorkerReduceUsage usage) {
+    for (VertexSenderObserver observer : observers) {
+      observer.vertexSenderWorkerPostprocess(usage);
+    }
+  }
+
+  public <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+      MasterGlobalCommUsage master,  ReduceOperation<S, R> reduceOp,
+      R globalInitialValue) {
+    LocalReduceHandle<S, R> handle = new LocalReduceHandle<>(reduceOp);
+    master.registerReducer(handle.getName(), reduceOp, globalInitialValue);
+    observers.add(handle);
+    return handle;
+  }
+
+  public <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+      MasterGlobalCommUsage master,  ReduceOperation<S, R> reduceOp,
+      R globalInitialValue) {
+    ReduceHandleImpl<S, R> handle = new GlobalReduceHandle<>(reduceOp);
+    master.registerReducer(handle.getName(), reduceOp, globalInitialValue);
+    observers.add(handle);
+    return handle;
+  }
+
+  /**
+   * Implementation of BroadcastHandle
+   *
+   * @param <T> Value type
+   */
+  public static class BroadcastHandleImpl<T> implements BroadcastHandle<T> {
+    private final String name;
+
+    public BroadcastHandleImpl() {
+      this.name = "_utils.broadcast." + BROADCAST_COUNTER.incrementAndGet();
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public T getBroadcast(WorkerBroadcastUsage worker) {
+      return worker.getBroadcast(name);
+    }
+  }
+
+  /**
+   * Parent implementation of ReducerHandle
+   *
+   * @param <S> Single value type
+   * @param <R> Reduced value type
+   */
+  public abstract class ReduceHandleImpl<S, R extends Writable>
+      implements ReducerHandle<S, R>, VertexSenderObserver {
+    protected final ReduceOperation<S, R> reduceOp;
+    private final String name;
+
+    private ReduceHandleImpl(ReduceOperation<S, R> reduceOp) {
+      this.reduceOp = reduceOp;
+      name = "_utils." + handleIndex +
+          ".reduce." + reduceCounter.incrementAndGet();
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public R getReducedValue(MasterGlobalCommUsage master) {
+      return master.getReduced(name);
+    }
+
+    @Override
+    public BroadcastHandle<R> broadcastValue(BlockMasterApi master) {
+      return unwrapHandle(master.broadcast(
+          new WrappedReducedValue<>(reduceOp, getReducedValue(master))));
+    }
+  }
+
+  private static <R extends Writable> BroadcastHandle<R> unwrapHandle(
+      final BroadcastHandle<WrappedReducedValue<R>> handle) {
+    return new BroadcastHandle<R>() {
+      @Override
+      public R getBroadcast(WorkerBroadcastUsage worker) {
+        return handle.getBroadcast(worker).getValue();
+      }
+    };
+  }
+
+  /**
+   * Wrapper that makes reduced values self-serializable,
+   * and allows them to be broadcasted.
+   *
+   * @param <R> Reduced value type
+   */
+  public static class WrappedReducedValue<R extends Writable>
+      implements Writable {
+    private ReduceOperation<?, R> reduceOp;
+    private R value;
+
+    public WrappedReducedValue() {
+    }
+
+    public WrappedReducedValue(ReduceOperation<?, R> reduceOp, R value) {
+      this.reduceOp = reduceOp;
+      this.value = value;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      WritableUtils.writeWritableObject(reduceOp, out);
+      value.write(out);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      reduceOp = WritableUtils.readWritableObject(in, null);
+      value = reduceOp.createInitialValue();
+      value.readFields(in);
+    }
+
+    public R getValue() {
+      return value;
+    }
+  }
+
+  /**
+   * Global Reduce Handle is implementation of ReducerHandle, that will keep
+   * only one value for each worker, and each call to reduce will have
+   * to obtain a global lock, and incur synchronization costs.
+   * Use only when objects are so large, that having many copies cannot fit
+   * into memory.
+   *
+   * @param <S> Single value type
+   * @param <R> Reduced value type
+   */
+  public class GlobalReduceHandle<S, R extends Writable>
+      extends ReduceHandleImpl<S, R> {
+    private transient WorkerReduceUsage usage;
+
+    public GlobalReduceHandle(ReduceOperation<S, R> reduceOp) {
+      super(reduceOp);
+    }
+
+    @Override
+    public void vertexSenderWorkerPreprocess(WorkerReduceUsage usage) {
+      this.usage = usage;
+    }
+
+    @Override
+    public void reduce(S valueToReduce) {
+      usage.reduce(getName(), valueToReduce);
+    }
+
+    @Override
+    public void vertexSenderWorkerPostprocess(WorkerReduceUsage usage) {
+    }
+  }
+
+  /**
+   * Local Reduce Handle is implementation of ReducerHandle, that will make a
+   * partially reduced value on each worker thread, which are at the end
+   * reduced all together.
+   * This is preferred implementation, unless it cannot be used due to memory
+   * overhead, because all partially reduced values will not fit the memory.
+   *
+   * @param <S> Single value type
+   * @param <R> Reduced value type
+   */
+  public class LocalReduceHandle<S, R extends Writable>
+      extends ReduceHandleImpl<S, R> {
+    private transient Reducer<S, R> reducer;
+
+    public LocalReduceHandle(ReduceOperation<S, R> reduceOp) {
+      super(reduceOp);
+    }
+
+    @Override
+    public void vertexSenderWorkerPreprocess(WorkerReduceUsage usage) {
+      this.reducer = new Reducer<>(reduceOp);
+    }
+
+    @Override
+    public void reduce(S valueToReduce) {
+      reducer.reduce(valueToReduce);
+    }
+
+    @Override
+    public void vertexSenderWorkerPostprocess(WorkerReduceUsage usage) {
+      usage.reduceMerge(getName(), reducer.getCurrentValue());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java
new file mode 100644
index 0000000..5b3485f
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.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.block_app.framework.piece.global_comm.internal;
+
+import org.apache.giraph.worker.WorkerReduceUsage;
+
+/**
+ * Observer able to hook into vertex sender pre/post processing
+ */
+public interface VertexSenderObserver {
+  void vertexSenderWorkerPreprocess(WorkerReduceUsage usage);
+  void vertexSenderWorkerPostprocess(WorkerReduceUsage usage);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java
new file mode 100644
index 0000000..1ba7c8f
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/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.
+ */
+/**
+ * Reducer and Broadcast Handles internal implementation for automatic handling
+ * of global communication within Pieces, hiding a lot of it's complexities.
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm.internal;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java
new file mode 100644
index 0000000..50d7818
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java
@@ -0,0 +1,36 @@
+/*
+ * 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.block_app.framework.piece.global_comm.map;
+
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle to map of broadcasts
+ *
+ * @param <K> Key type
+ * @param <V> Value type
+ */
+public interface BroadcastMapHandle<K, V>
+  extends MapHandle<K, BroadcastHandle<V>> {
+
+  /**
+   * Number of elements that were broadcasted.
+   */
+  int getBroadcastedSize(WorkerBroadcastUsage worker);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java
new file mode 100644
index 0000000..db01e77
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.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.block_app.framework.piece.global_comm.map;
+
+/**
+ * Handle to map of handles underneath
+ *
+ * @param <K> Key type
+ * @param <V> Value type
+ */
+public interface MapHandle<K, V> {
+  /**
+   * Get value for key.
+   */
+  V get(K key);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java
new file mode 100644
index 0000000..5c31179
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java
@@ -0,0 +1,43 @@
+/*
+ * 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.block_app.framework.piece.global_comm.map;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+
+/**
+ * Handle to array of reducers
+ *
+ * @param <I> Key type
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public interface ReducerMapHandle<I, S, R>
+    extends MapHandle<I, ReducerHandle<S, R>> {
+  /**
+   * Number of elements that were reduced.
+   */
+  int getReducedSize(BlockMasterApi master);
+
+  /**
+   * Broadcast whole map of reducers to master
+   *
+   * @return Handle to the broadcasted map.
+   */
+  BroadcastMapHandle<I, R> broadcastValue(BlockMasterApi master);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java
new file mode 100644
index 0000000..092f864
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/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.
+ */
+/**
+ * Interfaces representing map of individual handles
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm.map;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java
new file mode 100644
index 0000000..0d40741
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/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.
+ */
+/**
+ * Interfaces for Reducer and Broadcast Handles for automatic handling
+ * of global communication within Pieces, hiding a lot of it's complexities.
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java
new file mode 100644
index 0000000..b6cc749
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.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.block_app.framework.piece.interfaces;
+
+import org.apache.giraph.writable.kryo.markers.NonKryoWritable;
+
+/**
+ * Interface containing a single function - postprocess.
+ *
+ * Marked to not allow seriazliation, as it should be created on the worker,
+ * so should never be serialiized, disallow only for catching problems early.
+ */
+public interface VertexPostprocessor extends NonKryoWritable {
+  /**
+   * Override to finish computation. This method is executed exactly once
+   * after computation for all vertices in the partition is complete.
+   */
+  void postprocess();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java
new file mode 100644
index 0000000..26912ee
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.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.block_app.framework.piece.interfaces;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.writable.kryo.markers.NonKryoWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface representing actions that happen on worker, for each vertex,
+ * during receive phase:
+ * <ul>
+ * <li> to receive messages from vertices </li>
+ * <li> to receive data from master through aggregators </li>
+ * </ul>
+ *
+ * Marked to not allow seriazliation, as it should be created on the worker,
+ * so should never be serialiized, disallow only for catching problems early.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ */
+@SuppressWarnings("rawtypes")
+public interface VertexReceiver<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable>
+    extends NonKryoWritable {
+  /**
+   * Must be defined by user to do computation on a single Vertex.
+   *
+   * @param vertex   Vertex
+   * @param messages Messages that were sent to this vertex in the previous
+   *                 superstep.  Each message is only guaranteed to have
+   *                 a life expectancy as long as next() is not called.
+   */
+  void vertexReceive(Vertex<I, V, E> vertex, Iterable<M> messages);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java
new file mode 100644
index 0000000..0587032
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java
@@ -0,0 +1,45 @@
+/*
+ * 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.block_app.framework.piece.interfaces;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.writable.kryo.markers.NonKryoWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface representing actions that happen on worker, for each vertex,
+ * during send phase:
+ * <ul>
+ * <li> to send messages to vertices </li>
+ * <li> to send data for aggregation on master </li>
+ * </ul>
+ *
+ * Marked to not allow seriazliation, as it should be created on the worker,
+ * so should never be serialiized, disallow only for catching problems early.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ */
+@SuppressWarnings("rawtypes")
+public interface VertexSender<I extends WritableComparable,
+    V extends Writable, E extends Writable> extends NonKryoWritable {
+  /** Must be defined by user to do computation on a single Vertex. */
+  void vertexSend(Vertex<I, V, E> vertex);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java
new file mode 100644
index 0000000..db05e78
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/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.
+ */
+/**
+ * Vertex processing functions for Pieces
+ */
+package org.apache.giraph.block_app.framework.piece.interfaces;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java
new file mode 100644
index 0000000..dd977e6
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java
@@ -0,0 +1,119 @@
+/*
+ * 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.block_app.framework.piece.messages;
+
+import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.giraph.writable.kryo.KryoWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * MessageClasses implementation that provides factory and combiner instances
+ * through a provided supplier.
+ *
+ * @param <I> Vertex id type
+ * @param <M> Message type
+ */
+public class ObjectMessageClasses<I extends WritableComparable,
+    M extends Writable> extends KryoWritable implements MessageClasses<I, M> {
+  private final Class<M> messageClass;
+  private final SupplierFromConf<MessageValueFactory<M>>
+  messageValueFactorySupplier;
+  private final SupplierFromConf<? extends MessageCombiner<? super I, M>>
+  messageCombinerSupplier;
+  private final MessageEncodeAndStoreType messageEncodeAndStoreType;
+
+  public ObjectMessageClasses() {
+    this(null, null, null, null);
+  }
+
+  public ObjectMessageClasses(Class<M> messageClass,
+      SupplierFromConf<MessageValueFactory<M>> messageValueFactorySupplier,
+      SupplierFromConf<? extends MessageCombiner<? super I, M>>
+        messageCombinerSupplier,
+      MessageEncodeAndStoreType messageEncodeAndStoreType) {
+    this.messageClass = messageClass;
+    this.messageValueFactorySupplier = messageValueFactorySupplier;
+    this.messageCombinerSupplier = messageCombinerSupplier;
+    this.messageEncodeAndStoreType = messageEncodeAndStoreType;
+  }
+
+  @Override
+  public Class<M> getMessageClass() {
+    return messageClass;
+  }
+
+  @Override
+  public MessageValueFactory<M> createMessageValueFactory(
+      ImmutableClassesGiraphConfiguration conf) {
+    return Preconditions.checkNotNull(messageValueFactorySupplier.apply(conf));
+  }
+
+  @Override
+  public MessageCombiner<? super I, M> createMessageCombiner(
+      ImmutableClassesGiraphConfiguration<I, ? extends Writable,
+        ? extends Writable> conf) {
+    return messageCombinerSupplier != null ?
+      Preconditions.checkNotNull(messageCombinerSupplier.apply(conf)) : null;
+  }
+
+  @Override
+  public boolean useMessageCombiner() {
+    return messageCombinerSupplier != null;
+  }
+
+  @Override
+  public MessageEncodeAndStoreType getMessageEncodeAndStoreType() {
+    return messageEncodeAndStoreType;
+  }
+
+  @Override
+  public MessageClasses<I, M> createCopyForNewSuperstep() {
+    return new ObjectMessageClasses<>(
+        messageClass, messageValueFactorySupplier,
+        messageCombinerSupplier, messageEncodeAndStoreType);
+  }
+
+  @Override
+  public void verifyConsistent(ImmutableClassesGiraphConfiguration conf) {
+    MessageValueFactory<M> messageValueFactory =
+        messageValueFactorySupplier.apply(conf);
+    Preconditions.checkState(
+        messageValueFactory.newInstance().getClass().equals(messageClass));
+
+    if (messageCombinerSupplier != null) {
+      MessageCombiner<? super I, M> messageCombiner =
+          messageCombinerSupplier.apply(conf);
+      Preconditions.checkState(messageCombiner.createInitialMessage()
+          .getClass().equals(messageClass));
+      Class<?>[] combinerTypes = ReflectionUtils.getTypeArguments(
+          MessageCombiner.class, messageCombiner.getClass());
+      ReflectionUtils.verifyTypes(conf.getVertexIdClass(), combinerTypes[0],
+          "Vertex id", messageCombiner.getClass());
+      ReflectionUtils.verifyTypes(messageClass, combinerTypes[1],
+          "Outgoing message", messageCombiner.getClass());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java
new file mode 100644
index 0000000..00c86cd
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.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.block_app.framework.piece.messages;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.DefaultMessageValueFactory;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.function.Function;
+import org.apache.giraph.writable.kryo.HadoopKryo;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Supplier from configuration
+ * @param <T> Type of object returned
+ */
+public interface SupplierFromConf<T>
+    extends Function<ImmutableClassesGiraphConfiguration, T> {
+
+  /**
+   * Supplier from configuration, by copying given instance every time.
+   *
+   * @param <T> Type of object returned
+   */
+  public static class SupplierFromConfByCopy<T> implements SupplierFromConf<T> {
+    private final T value;
+
+    public SupplierFromConfByCopy(T value) {
+      this.value = value;
+    }
+
+    @Override
+    public T apply(ImmutableClassesGiraphConfiguration conf) {
+      return HadoopKryo.createCopy(value);
+    }
+  }
+
+  /**
+   * Supplier from configuration returning DefaultMessageValueFactory instances.
+   *
+   * @param <M> Message type
+   */
+  public static class DefaultMessageFactorySupplierFromConf<M extends Writable>
+      implements SupplierFromConf<MessageValueFactory<M>> {
+    private final Class<M> messageClass;
+
+    public DefaultMessageFactorySupplierFromConf(Class<M> messageClass) {
+      this.messageClass = messageClass;
+    }
+
+    @Override
+    public MessageValueFactory<M> apply(
+        ImmutableClassesGiraphConfiguration conf) {
+      return new DefaultMessageValueFactory<>(messageClass, conf);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java
new file mode 100644
index 0000000..ba3014c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/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.
+ */
+/**
+ * Utility classes for handling of messages within Pieces
+ */
+package org.apache.giraph.block_app.framework.piece.messages;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java
new file mode 100644
index 0000000..fbc6e92
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+/**
+ * Single execution object - Piece, and related classes.
+ *
+ * AbstractPiece is parent class of all Pieces. Most frequentlly
+ * users should extend Piece class itself
+ */
+package org.apache.giraph.block_app.framework.piece;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java b/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java
new file mode 100644
index 0000000..2a0e36a
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.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.function;
+
+import java.io.Serializable;
+
+
+/**
+ * Function:
+ * (T) -> void
+ *
+ * @param <T> Argument type
+ */
+public interface Consumer<T> extends Serializable {
+  /**
+   * Applies this function to {@code input}
+   */
+  void apply(T input);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java b/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java
new file mode 100644
index 0000000..41046ba
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java
@@ -0,0 +1,38 @@
+/*
+ * 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.function;
+
+import java.io.Serializable;
+
+
+/**
+ * Function:
+ * (F) -> T
+ *
+ * @param <F> Argument type
+ * @param <T> Result type
+ */
+public interface Function<F, T> extends Serializable {
+  /**
+   * Returns the result of applying this function to given {@code input}.
+   *
+   * The returned object may or may not be a new instance,
+   * depending on the implementation.
+   */
+  T apply(F input);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java b/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java
new file mode 100644
index 0000000..012ec82
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.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.function;
+
+import java.io.Serializable;
+
+/**
+ * Function:
+ * (T1, T2) -> void
+ *
+ * @param <T1> First argument type
+ * @param <T2> Second argument type
+ */
+public interface PairConsumer<T1, T2> extends Serializable {
+  /**
+   * Applies this function to {@code input1} and {@code input2}
+   */
+  void apply(T1 input1, T2 input2);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java b/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java
new file mode 100644
index 0000000..bfff400
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.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.function;
+
+import java.io.Serializable;
+
+
+/**
+ * Function:
+ * (F1, F2) -> T
+ *
+ * @param <F1> First argument type
+ * @param <F2> Second argument type
+ * @param <T> Result type
+ */
+public interface PairFunction<F1, F2, T> extends Serializable {
+  /**
+   * Returns the result of applying this function to given
+   * {@code input1} and {@code input2}.
+   *
+   * The returned object may or may not be a new instance,
+   * depending on the implementation.
+   */
+  T apply(F1 input1, F2 input2);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java b/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java
new file mode 100644
index 0000000..1813b54
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java
@@ -0,0 +1,37 @@
+/*
+ * 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.function;
+
+import java.io.Serializable;
+
+/**
+ * Function:
+ * () -> T
+ * <br>
+ * Specialization of com.google.common.base.Supplier, that is also
+ * Serializable.
+ *
+ * @param <T> Result type
+ */
+public interface Supplier<T> extends Serializable {
+  /**
+   * Retrieves an instance of the appropriate type. The returned object may or
+   * may not be a new instance, depending on the implementation.
+   */
+  T get();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java
new file mode 100644
index 0000000..b089da6
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.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 for interfaces representing different functions,
+ * that all extends Serializable in order for Kryo to be able
+ * to serialize them.
+ *
+ * Even when same interface is present in Guava, we do not extend it
+ * due to @Nullable annotations adding requirement of handling nulls.
+ */
+package org.apache.giraph.function;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java
new file mode 100644
index 0000000..2cf74e1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java
@@ -0,0 +1,27 @@
+/*
+ * 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.function.primitive;
+
+/**
+ * Primitive specialization of Function:
+ * () -> int
+ */
+public interface IntSupplier {
+  /** Retrieves an int value. */
+  int get();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java
new file mode 100644
index 0000000..fdf40ff
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/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.
+ */
+/**
+ * Primitive specializations of interfaces from org.apache.giraph.function
+ * package.
+ */
+package org.apache.giraph.function.primitive;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java
new file mode 100644
index 0000000..87e7f9b
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java
@@ -0,0 +1,48 @@
+/*
+ * 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.function.vertex;
+
+import java.io.Serializable;
+
+import org.apache.giraph.function.PairConsumer;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+
+/**
+ * Function:
+ * (vertex, T) -> void
+ *
+ * A class that can consume objects of a single type, when given a vertex.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <T> Argument type
+ */
+@SuppressWarnings("rawtypes")
+public interface ConsumerWithVertex<I extends WritableComparable,
+    V extends Writable, E extends Writable, T>
+    extends PairConsumer<Vertex<I, V, E>, T>, Serializable  {
+  /**
+   * Applies this function to {@code vertex} and {@code input}
+   */
+  @Override
+  void apply(Vertex<I, V, E> vertex, T value);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java
new file mode 100644
index 0000000..fdab5de
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.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.function.vertex;
+
+import java.io.Serializable;
+
+import org.apache.giraph.function.PairFunction;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Function:
+ * (vertex, F) -> T
+ *
+ * Determines an output value based on a vertex and an input value.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <F> Argument type
+ * @param <T> Result type
+ */
+@SuppressWarnings("rawtypes")
+public interface FunctionWithVertex<I extends WritableComparable,
+    V extends Writable, E extends Writable, F, T>
+    extends PairFunction<Vertex<I, V, E>, F, T>, Serializable {
+  /**
+   * Returns the result of applying this function to given
+   * {@code vertex} and {@code input}.
+   *
+   * The returned object may or may not be a new instance,
+   * depending on the implementation.
+   */
+  @Override
+  T apply(Vertex<I, V, E> vertex, F input);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java
new file mode 100644
index 0000000..bc0f9c1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java
@@ -0,0 +1,49 @@
+/*
+ * 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.function.vertex;
+
+import java.io.Serializable;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Function:
+ * (vertex) -> T
+ *
+ * A class that can supply objects of a single type, when given a vertex.
+ *
+ * (doesn't extend Function<Vertex<I, V, E>, T>, because of different
+ * method names)
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <T> Result type
+ */
+@SuppressWarnings("rawtypes")
+public interface SupplierFromVertex<I extends WritableComparable,
+    V extends Writable, E extends Writable, T> extends Serializable {
+  /**
+   * Retrieves an instance of the appropriate type, given a vertex.
+   * The returned object may or may not be a new instance,
+   * depending on the implementation.
+   */
+  T get(Vertex<I, V, E> vertex);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java
new file mode 100644
index 0000000..bd5b019
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/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 interfaces representing functions additionally
+ * performed on vertex values.
+ */
+package org.apache.giraph.function.vertex;

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java
new file mode 100644
index 0000000..6487d95
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java
@@ -0,0 +1,158 @@
+/*
+ * 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.block_app.framework.block;
+
+import static org.junit.Assert.assertEquals;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class BlockTestingUtils {
+
+  BlockTestingUtils() { }
+
+  private static final int NUM_TRIALS = 10;
+  private static final int REPEAT_TIMES = 10;
+
+  private static int testSequential(Iterator<? extends AbstractPiece> referenceImpl,
+                                    Iterator<? extends AbstractPiece> testImpl) {
+    int length = 0;
+
+    CheckIterator checkIterator = new CheckIterator(referenceImpl, testImpl);
+    while (checkIterator.hasNext()) {
+      checkIterator.next();
+      length++;
+    }
+
+    System.out.println("Length is : " + length);
+    return length;
+  }
+
+  private static boolean anyHasNext(ArrayList<? extends Iterator> arr) {
+    for (Iterator t : arr) {
+      if (t.hasNext()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static void testRandom(int length,
+                                 Iterable<? extends AbstractPiece> referenceImpl,
+                                 Iterable<? extends AbstractPiece> testImpl) {
+    Random rand = new Random();
+
+    ArrayList<CheckIterator<AbstractPiece>> arr = new ArrayList<>();
+    IntArrayList lengths = new IntArrayList(NUM_TRIALS);
+    for (int i = 0; i < NUM_TRIALS; i++) {
+      lengths.add(0);
+    }
+    for (int i = 0; i < NUM_TRIALS; i++) {
+      arr.add(new CheckIterator(referenceImpl.iterator(), testImpl.iterator()));
+    }
+
+    int totalCount = 0;
+    while (anyHasNext(arr)) {
+      int index = rand.nextInt(NUM_TRIALS);
+      while (!arr.get(index).hasNext()) {
+        index = rand.nextInt(NUM_TRIALS);
+      }
+      CheckIterator it = arr.get(index);
+      it.next();
+      int itLength = lengths.getInt(index);
+      lengths.set(index, itLength + 1);
+      totalCount++;
+    }
+    assertEquals("TotalCount should be length * NUM_TRIALS", length * NUM_TRIALS, totalCount);
+    System.out.println("Final count is : " + totalCount);
+  }
+
+  /**
+   * Tests both the length of the iterator returned by the block, as-well as the deterministic behavior
+   * expected by calling .iterator() against the referenceImpl.
+   * @param referenceImpl : A list of pieces in the expected order
+   * @param testImpl : A list of pieces to test against (the Block)
+   */
+  public static void testIndependence(Iterable<? extends AbstractPiece> referenceImpl,
+                                      Iterable<? extends AbstractPiece> testImpl) {
+    int length = testSequential(referenceImpl.iterator(), testImpl.iterator());
+    testRandom(length, referenceImpl, testImpl);
+  }
+
+  /**
+   * Test how the block interacts with a repeatBlock. The expected result is to
+   * see the pieces in referenceImpl show up REPEAT_TIMES many times.
+   * @param referenceImpl : A list of pieces in the expected order
+   * @param block : The block to test
+   */
+  public static void testNestedRepeatBlock(Iterable<? extends AbstractPiece> referenceImpl, Block block) {
+    Block repeatBlock = new RepeatBlock(
+      REPEAT_TIMES,
+      block
+    );
+    testIndependence(
+            Iterables.concat(Collections.nCopies(REPEAT_TIMES, referenceImpl)),
+            repeatBlock
+    );
+  }
+
+  public static class CheckIterator<T> implements Iterator {
+
+    private final Iterator<T> fst;
+    private final Iterator<T> snd;
+
+    public CheckIterator(Iterator<T> fst, Iterator<T> snd) {
+      this.fst = fst;
+      this.snd = snd;
+    }
+
+    @Override
+    public boolean hasNext() {
+      boolean fstHasNxt = fst.hasNext();
+      boolean sndHasNxt = snd.hasNext();
+      Preconditions.checkArgument(fstHasNxt == sndHasNxt, "Expect hasNext() on " +
+              "both iterators to be identical. Got: " + fst.hasNext() + " and " + snd.hasNext());
+      return fstHasNxt;
+    }
+
+    @Override
+    public Object next() {
+      T fstNxt = fst.next();
+      T sndNxt = snd.next();
+      Preconditions.checkArgument(fstNxt == sndNxt, "Expect objs returned by " +
+              "both iterators to be identical. Got: " + fstNxt + " and " + sndNxt);
+      return fstNxt;
+    }
+
+    @Override
+    public void remove() {
+      throw new RuntimeException("Not implemented");
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java
new file mode 100644
index 0000000..0dacae1
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java
@@ -0,0 +1,88 @@
+/*
+ * 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.block_app.framework.block;
+
+
+import java.util.Arrays;
+
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.apache.giraph.function.Supplier;
+import org.junit.Test;
+
+public class TestIfBlock {
+
+  private static final Supplier<Boolean> TRUE_SUPPLIER = new Supplier<Boolean>() {
+    @Override
+    public Boolean get() {
+      return true;
+    }
+  };
+
+  private static final Supplier<Boolean> FALSE_SUPPLIER = new Supplier<Boolean>() {
+    @Override
+    public Boolean get() {
+      return false;
+    }
+  };
+
+  @Test
+  // Test short-circuiting the if -> then
+  public void testIfBlockThen() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block ifBlock = new IfBlock(
+      TRUE_SUPPLIER,
+      new SequenceBlock(piece1, piece2)
+    );
+
+    BlockTestingUtils.testIndependence(
+        Arrays.asList(piece1, piece2),
+        ifBlock);
+  }
+
+  @Test
+  // Test short-circuiting the if -> else
+  public void testIfBlockElse() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block ifBlock = new IfBlock(
+      FALSE_SUPPLIER,
+      new EmptyBlock(),
+      new SequenceBlock(piece1, piece2)
+    );
+
+    BlockTestingUtils.testIndependence(
+        Arrays.asList(piece1, piece2),
+        ifBlock);
+  }
+
+  @Test
+  public void testIfNestedInRepeat() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block ifBlock = new IfBlock(
+      TRUE_SUPPLIER,
+      new SequenceBlock(piece1, piece2)
+    );
+
+    BlockTestingUtils.testNestedRepeatBlock(
+            Arrays.asList(piece1, piece2),
+            ifBlock);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java
new file mode 100644
index 0000000..1e096ba
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.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.block_app.framework.block;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Tests repeatBlock's correctness
+ */
+public class TestRepeatBlock {
+
+  public static final int REPEAT_TIMES = 5;
+
+  @Test
+  public void testRepeatBlockBasic() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block innerBlock = new SequenceBlock(piece1, piece2);
+    Block repeatBlock = new RepeatBlock(
+            REPEAT_TIMES,
+            innerBlock
+    );
+    BlockTestingUtils.testIndependence(
+            Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+            repeatBlock);
+  }
+
+  @Test
+  public void testNestedRepeatBlock() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block innerBlock = new SequenceBlock(piece1, piece2);
+    Block repeatBlock = new RepeatBlock(
+            REPEAT_TIMES,
+            innerBlock
+    );
+    BlockTestingUtils.testNestedRepeatBlock(
+            Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+            repeatBlock);
+  }
+
+  @Test
+  public void testRepeatBlockEmpty() throws Exception {
+    Block innerBlock = new EmptyBlock();
+    Block repeatBlock = new RepeatBlock(
+            REPEAT_TIMES,
+            innerBlock
+    );
+    List<? extends AbstractPiece> referenceImpl = Collections.emptyList();
+    BlockTestingUtils.testIndependence(
+            // Concatenating EmptyIterator = just EmptyIterator. No obj's to
+            // compare against either
+            referenceImpl,
+            repeatBlock);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java
new file mode 100644
index 0000000..242d376
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.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.block_app.framework.block;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.apache.giraph.function.Supplier;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Tests repeatUntilBlock's correctness
+ */
+public class TestRepeatUntilBlock {
+
+  public static final int REPEAT_TIMES = 5;
+
+  private static final Supplier<Boolean> falseSupplier = new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return false;
+      }
+  };
+
+  @Test
+  public void testRepeatUntilBlockBasic() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block innerBlock = new SequenceBlock(piece1, piece2);
+    Block repeatBlock = new RepeatUntilBlock(
+      REPEAT_TIMES,
+      innerBlock,
+      falseSupplier
+    );
+    BlockTestingUtils.testIndependence(
+      Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+      repeatBlock);
+  }
+
+  @Test
+  public void testNestedRepeatUntilBlock() throws Exception {
+    Piece piece1 = new Piece();
+    Piece piece2 = new Piece();
+    Block innerBlock = new SequenceBlock(piece1, piece2);
+    Block repeatBlock = new RepeatUntilBlock(
+      REPEAT_TIMES,
+      innerBlock,
+      falseSupplier
+    );
+    BlockTestingUtils.testNestedRepeatBlock(
+      Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+      repeatBlock);
+  }
+
+  @Test
+  public void testRepeatUntilBlockUnlimited() throws Exception {
+    Block innerBlock = new SequenceBlock(new Piece());
+    // Can't test with testIndependence - spin up our own test inline
+    Supplier<Boolean> countingSupplier = new Supplier<Boolean>() {
+      private int i = 0;
+
+      @Override
+      public Boolean get() {
+        i++;
+        return i > REPEAT_TIMES;
+      }
+    };
+    Block repeatBlock = RepeatUntilBlock.unlimited(
+      innerBlock,
+      countingSupplier
+    );
+    int count = 0;
+    Iterator<AbstractPiece> it = repeatBlock.iterator();
+    while (it.hasNext()) {
+      it.next();
+      count++;
+    }
+    assertEquals("Count must be equal to REPEAT_TIMES", REPEAT_TIMES, count);
+  }
+
+}