You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by we...@apache.org on 2015/03/06 02:55:31 UTC

[1/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Repository: incubator-reef
Updated Branches:
  refs/heads/master 0911c0832 -> 6c6ad3367


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
new file mode 100644
index 0000000..e3edb01
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyStructImpl.java
@@ -0,0 +1,579 @@
+/**
+ * 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.reef.io.network.group.impl.task;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.group.api.task.NodeStruct;
+import org.apache.reef.io.network.group.api.task.OperatorTopologyStruct;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.operators.Sender;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+
+import java.util.*;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Logger;
+
+/**
+ *
+ */
+public class OperatorTopologyStructImpl implements OperatorTopologyStruct {
+
+  private static final int SMALL_MSG_LENGTH = 1 << 20;
+
+  private static final Logger LOG = Logger.getLogger(OperatorTopologyStructImpl.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String selfId;
+  private final String driverId;
+  private final Sender sender;
+
+  private boolean changes = true;
+  private NodeStruct parent;
+  private final List<NodeStruct> children = new ArrayList<>();
+
+  private final BlockingQueue<NodeStruct> nodesWithData = new LinkedBlockingQueue<>();
+  private final Set<String> childrenToRcvFrom = new HashSet<>();
+
+  private final ConcurrentMap<String, Set<Integer>> deadMsgs = new ConcurrentHashMap<>();
+
+  private final int version;
+
+  public OperatorTopologyStructImpl(final Class<? extends Name<String>> groupName,
+                                    final Class<? extends Name<String>> operName, final String selfId,
+                                    final String driverId, final Sender sender, final int version) {
+    super();
+    this.groupName = groupName;
+    this.operName = operName;
+    this.selfId = selfId;
+    this.driverId = driverId;
+    this.sender = sender;
+    this.version = version;
+  }
+
+  public OperatorTopologyStructImpl(final OperatorTopologyStruct topology) {
+    super();
+    this.groupName = topology.getGroupName();
+    this.operName = topology.getOperName();
+    this.selfId = topology.getSelfId();
+    this.driverId = topology.getDriverId();
+    this.sender = topology.getSender();
+    this.changes = topology.hasChanges();
+    this.parent = topology.getParent();
+    this.children.addAll(topology.getChildren());
+    this.version = topology.getVersion();
+  }
+
+  @Override
+  public String toString() {
+    return "OperatorTopologyStruct - " + Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + "(" + selfId + "," + version + ")";
+  }
+
+  @Override
+  public NodeStruct getParent() {
+    return parent;
+  }
+
+  @Override
+  public Collection<? extends NodeStruct> getChildren() {
+    return children;
+  }
+
+  @Override
+  public Class<? extends Name<String>> getGroupName() {
+    return groupName;
+  }
+
+  @Override
+  public Class<? extends Name<String>> getOperName() {
+    return operName;
+  }
+
+  @Override
+  public String getSelfId() {
+    return selfId;
+  }
+
+  @Override
+  public String getDriverId() {
+    return driverId;
+  }
+
+  @Override
+  public Sender getSender() {
+    return sender;
+  }
+
+  @Override
+  public boolean hasChanges() {
+    LOG.entering("OperatorTopologyStructImpl", "hasChanges", getQualifiedName());
+    LOG.exiting("OperatorTopologyStructImpl", "hasChanges", Arrays.toString(new Object[]{this.changes, getQualifiedName()}));
+    return this.changes;
+  }
+
+  @Override
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void addAsData(final GroupCommunicationMessage msg) {
+    LOG.entering("OperatorTopologyStructImpl", "addAsData", new Object[]{getQualifiedName(), msg});
+    final String srcId = msg.getSrcid();
+    final NodeStruct node = findNode(srcId);
+    if (node != null) {
+      try {
+        nodesWithData.put(node);
+        LOG.finest(getQualifiedName() + "Added node " + srcId + " to nodesWithData queue");
+      } catch (final InterruptedException e) {
+        throw new RuntimeException("InterruptedException while adding to childrenWithData queue", e);
+      }
+      node.addData(msg);
+    } else {
+      LOG.fine("Unable to find node " + srcId + " to send " + msg.getType() + " to");
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "addAsData", Arrays.toString(new Object[]{getQualifiedName(), msg}));
+  }
+
+  private NodeStruct findNode(final String srcId) {
+    LOG.entering("OperatorTopologyStructImpl", "findNode", new Object[]{getQualifiedName(), srcId});
+    final NodeStruct retVal;
+    if (parent != null && parent.getId().equals(srcId)) {
+      retVal = parent;
+    } else {
+      retVal = findChild(srcId);
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "findNode", Arrays.toString(new Object[]{retVal, getQualifiedName(), srcId}));
+    return retVal;
+  }
+
+  private void sendToNode(final byte[] data, final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType, final NodeStruct node) {
+    LOG.entering("OperatorTopologyStructImpl", "sendToNode", new Object[]{getQualifiedName(), data, msgType, node});
+    final String nodeId = node.getId();
+    try {
+
+      if (data.length > SMALL_MSG_LENGTH) {
+        LOG.finest(getQualifiedName() + "Msg too big. Sending readiness to send " + msgType + " msg to " + nodeId);
+        sender.send(Utils.bldVersionedGCM(groupName, operName, msgType, selfId, version, nodeId, node.getVersion(),
+            Utils.EmptyByteArr));
+        final byte[] tmpVal = receiveFromNode(node, true);
+        if (tmpVal != null) {
+          LOG.finest(getQualifiedName() + "Got readiness to accept " + msgType + " msg from " + nodeId
+              + ". Will send actual msg now");
+        } else {
+          LOG.exiting("OperatorTopologyStructImpl", "sendToNode", Arrays.toString(new Object[]{getQualifiedName(),
+              data, msgType, node}));
+          return;
+        }
+      }
+
+      sender.send(Utils.bldVersionedGCM(groupName, operName, msgType, selfId, version, nodeId, node.getVersion(), data));
+
+      if (data.length > SMALL_MSG_LENGTH) {
+        LOG.finest(getQualifiedName() + "Msg too big. Will wait for ACK before queing up one more msg");
+        final byte[] tmpVal = receiveFromNode(node, true);
+        if (tmpVal != null) {
+          LOG.finest(getQualifiedName() + "Got " + msgType + " msg received ACK from " + nodeId
+              + ". Will move to next msg if it exists");
+        } else {
+          LOG.exiting("OperatorTopologyStructImpl", "sendToNode", Arrays.toString(new Object[]{getQualifiedName(),
+              data, msgType, node}));
+          return;
+        }
+      }
+    } catch (final NetworkException e) {
+      throw new RuntimeException(
+          "NetworkException while sending " + msgType + " data from " + selfId + " to " + nodeId,
+          e);
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "sendToNode", Arrays.toString(new Object[]{getQualifiedName(), data,
+        msgType, node}));
+  }
+
+  /**
+   * @param childNode
+   * @return
+   */
+  private byte[] receiveFromNode(final NodeStruct node, final boolean remove) {
+    LOG.entering("OperatorTopologyStructImpl", "receiveFromNode", new Object[]{getQualifiedName(), node, remove});
+    final byte[] retVal = node.getData();
+    if (remove) {
+      final boolean removed = nodesWithData.remove(node);
+      final String msg = getQualifiedName() + "Removed(" + removed + ") node " + node.getId()
+          + " from nodesWithData queue";
+      if (removed) {
+        LOG.finest(msg);
+      } else {
+        LOG.fine(msg);
+      }
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "receiveFromNode", Arrays.toString(new Object[]{retVal, getQualifiedName(),
+        node, remove}));
+    return retVal;
+  }
+
+  @Override
+  public void sendToParent(final byte[] data, final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) {
+    LOG.entering("OperatorTopologyStructImpl", "sendToParent", new Object[]{getQualifiedName(), data, msgType});
+    if (parent != null) {
+      sendToNode(data, msgType, parent);
+    } else {
+      LOG.fine(getQualifiedName() + "Perhaps parent has died or has not been configured");
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "sendToParent", Arrays.toString(new Object[]{getQualifiedName(), data,
+        msgType}));
+  }
+
+  @Override
+  public void sendToChildren(final byte[] data, final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) {
+    LOG.entering("OperatorTopologyStructImpl", "sendToChildren", new Object[]{getQualifiedName(), data, msgType});
+    for (final NodeStruct child : children) {
+      sendToNode(data, msgType, child);
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "sendToChildren", Arrays.toString(new Object[]{getQualifiedName(),
+        data, msgType}));
+  }
+
+  @Override
+  public byte[] recvFromParent() {
+    LOG.entering("OperatorTopologyStructImpl", "recvFromParent", getQualifiedName());
+    LOG.finest(getQualifiedName() + "Waiting for " + parent.getId() + " to send data");
+    byte[] retVal = receiveFromNode(parent, true);
+    if (retVal != null && retVal.length == 0) {
+      LOG.finest(getQualifiedName() + "Got msg that parent " + parent.getId()
+          + " has large data and is ready to send data. Sending Ack to receive data");
+      sendToNode(Utils.EmptyByteArr, ReefNetworkGroupCommProtos.GroupCommMessage.Type.Broadcast, parent);
+      retVal = receiveFromNode(parent, true);
+      if (retVal != null) {
+        LOG.finest(getQualifiedName() + "Received large msg from Parent " + parent.getId()
+            + ". Will return it after ACKing it");
+        sendToNode(Utils.EmptyByteArr, ReefNetworkGroupCommProtos.GroupCommMessage.Type.Broadcast, parent);
+      }
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "recvFromParent",
+        Arrays.toString(new Object[]{retVal, getQualifiedName()}));
+    return retVal;
+  }
+
+  @Override
+  public <T> T recvFromChildren(final ReduceFunction<T> redFunc, final Codec<T> dataCodec) {
+    LOG.entering("OperatorTopologyStructImpl", "recvFromChildren", new Object[]{getQualifiedName(), redFunc,
+        dataCodec});
+    final List<T> retLst = new ArrayList<>(2);
+    for (final NodeStruct child : children) {
+      childrenToRcvFrom.add(child.getId());
+    }
+
+    while (!childrenToRcvFrom.isEmpty()) {
+      LOG.finest(getQualifiedName() + "Waiting for some child to send data");
+      NodeStruct child;
+      try {
+        child = nodesWithData.take();
+      } catch (final InterruptedException e) {
+        throw new RuntimeException("InterruptedException while waiting to take data from nodesWithData queue", e);
+      }
+      byte[] retVal = receiveFromNode(child, false);
+      if (retVal != null && retVal.length == 0) {
+        LOG.finest(getQualifiedName() + "Got msg that child " + child.getId()
+            + " has large data and is ready to send data. Sending Ack to receive data");
+        sendToNode(Utils.EmptyByteArr, ReefNetworkGroupCommProtos.GroupCommMessage.Type.Reduce, child);
+        retVal = receiveFromNode(child, true);
+        if (retVal != null) {
+          LOG.finest(getQualifiedName() + "Received large msg from child " + child.getId()
+              + ". Will reduce it after ACKing it");
+          sendToNode(Utils.EmptyByteArr, ReefNetworkGroupCommProtos.GroupCommMessage.Type.Reduce, child);
+        } else {
+          LOG.finest(getQualifiedName() + "Will not reduce it");
+        }
+      }
+      if (retVal != null) {
+        retLst.add(dataCodec.decode(retVal));
+        if (retLst.size() == 2) {
+          final T redVal = redFunc.apply(retLst);
+          retLst.clear();
+          retLst.add(redVal);
+        }
+      }
+      childrenToRcvFrom.remove(child.getId());
+    }
+    final T retVal = retLst.isEmpty() ? null : retLst.get(0);
+    LOG.exiting("OperatorTopologyStructImpl", "recvFromChildren", Arrays.toString(new Object[]{retVal, getQualifiedName(),
+        redFunc, dataCodec}));
+    return retVal;
+  }
+
+  private boolean removedDeadMsg(final String msgSrcId, final int msgSrcVersion) {
+    LOG.entering("OperatorTopologyStructImpl", "removedDeadMsg", new Object[]{getQualifiedName(), msgSrcId,
+        msgSrcVersion});
+    boolean retVal = false;
+    final Set<Integer> msgVersions = deadMsgs.get(msgSrcId);
+    if (msgVersions != null) {
+      LOG.fine(getQualifiedName() + "Found dead msgs " + msgVersions + " waiting for add");
+      if (msgVersions.remove(msgSrcVersion)) {
+        LOG.fine(getQualifiedName() + "Found dead msg with same version as srcVer-" + msgSrcVersion);
+        retVal = true;
+      } else {
+        LOG.finest(getQualifiedName() + "No dead msg with same version as srcVer-" + msgSrcVersion);
+      }
+    } else {
+      LOG.finest(getQualifiedName() + "No dead msgs waiting for add.");
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "removedDeadMsg",
+        Arrays.toString(new Object[]{retVal, getQualifiedName(), msgSrcId, msgSrcVersion}));
+    return retVal;
+  }
+
+  private void addToDeadMsgs(final String srcId, final int version) {
+    LOG.entering("OperatorTopologyStructImpl", "addToDeadMsgs", new Object[]{getQualifiedName(), srcId, version});
+    deadMsgs.putIfAbsent(srcId, new HashSet<Integer>());
+    deadMsgs.get(srcId).add(version);
+    LOG.exiting("OperatorTopologyStructImpl", "addToDeadMsgs", Arrays.toString(new Object[]{getQualifiedName(),
+        srcId, version}));
+  }
+
+  private boolean addedToDeadMsgs(final NodeStruct node, final String msgSrcId, final int msgSrcVersion) {
+    LOG.entering("OperatorTopologyStructImpl", "addedToDeadMsgs", new Object[]{getQualifiedName(), node, msgSrcId,
+        msgSrcVersion});
+    if (node == null) {
+      LOG.warning(getQualifiedName() + "Got dead msg when no node existed. OOS Queing up for add to handle");
+      addToDeadMsgs(msgSrcId, msgSrcVersion);
+      LOG.exiting("OperatorTopologyStructImpl", "addedToDeadMsgs", Arrays.toString(new Object[]{true, getQualifiedName(),
+          node, msgSrcId,
+          msgSrcVersion}));
+      return true;
+    }
+    final int nodeVersion = node.getVersion();
+    if (msgSrcVersion > nodeVersion) {
+      LOG.warning(getQualifiedName() + "Got an OOS dead msg. " + "Has HIGHER ver-" + msgSrcVersion + " than node ver-"
+          + nodeVersion + ". Queing up for add to handle");
+      addToDeadMsgs(msgSrcId, msgSrcVersion);
+      LOG.exiting("OperatorTopologyStructImpl", "addedToDeadMsgs", Arrays.toString(new Object[]{true, getQualifiedName(),
+          node, msgSrcId,
+          msgSrcVersion}));
+      return true;
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "addedToDeadMsgs", Arrays.toString(new Object[]{false, getQualifiedName(),
+        node, msgSrcId,
+        msgSrcVersion}));
+    return false;
+  }
+
+  /**
+   * Updates the topology structure with the received
+   * message. Does not make assumptions about msg order
+   * Tries to handle OOS msgs
+   * <p/>
+   * Expects only control messages
+   */
+  @Override
+  public void update(final GroupCommunicationMessage msg) {
+    if (msg.hasSrcVersion()) {
+      final String srcId = msg.getSrcid();
+      final int srcVersion = msg.getSrcVersion();
+      LOG.finest(getQualifiedName() + "Updating " + msg.getType() + " msg from " + srcId);
+      LOG.finest(getQualifiedName() + "Before update: parent=" + ((parent != null) ? parent.getId() : "NULL"));
+      LOG.finest(getQualifiedName() + "Before update: children=" + children);
+      switch (msg.getType()) {
+        case ParentAdd:
+          updateParentAdd(srcId, srcVersion);
+          break;
+        case ParentDead:
+          updateParentDead(srcId, srcVersion);
+          break;
+        case ChildAdd:
+          updateChildAdd(srcId, srcVersion);
+          break;
+        case ChildDead:
+          updateChildDead(srcId, srcVersion);
+          break;
+        default:
+          throw new RuntimeException("Received a non control message in update");
+      }
+      LOG.finest(getQualifiedName() + "After update: parent=" + ((parent != null) ? parent.getId() : "NULL"));
+      LOG.finest(getQualifiedName() + "After update: children=" + children);
+    } else {
+      throw new RuntimeException(getQualifiedName() + "can only deal with msgs that have src version set");
+    }
+  }
+
+  private void updateChildDead(final String srcId, final int srcVersion) {
+    LOG.entering("OperatorTopologyStructImpl", "updateChildDead",
+        new Object[]{getQualifiedName(), srcId, srcVersion});
+    final NodeStruct toBeRemovedchild = findChild(srcId);
+    if (!addedToDeadMsgs(toBeRemovedchild, srcId, srcVersion)) {
+      final int childVersion = toBeRemovedchild.getVersion();
+      if (srcVersion < childVersion) {
+        LOG.finest(getQualifiedName() + "Got an OOS child dead msg. " + "Has LOWER ver-" + srcVersion
+            + " than child ver-" + childVersion + ". Discarding");
+        LOG.exiting("OperatorTopologyStructImpl", "updateChildDead", Arrays.toString(new Object[]{getQualifiedName(),
+            srcId, srcVersion}));
+        return;
+      } else {
+        LOG.finest(getQualifiedName() + "Got a child dead msg. " + "Has SAME ver-" + srcVersion + " as child ver-"
+            + childVersion + "Removing child node");
+      }
+    } else {
+      LOG.fine(getQualifiedName() + "Added to dead msgs. Removing child node since ChildAdd might not turn up");
+    }
+    children.remove(toBeRemovedchild);
+    LOG.exiting("OperatorTopologyStructImpl", "updateChildDead", Arrays.toString(new Object[]{getQualifiedName(),
+        srcId, srcVersion}));
+  }
+
+  private void updateChildAdd(final String srcId, final int srcVersion) {
+    LOG.entering("OperatorTopologyStructImpl", "updateChildAdd", new Object[]{getQualifiedName(), srcId, srcVersion});
+    if (!removedDeadMsg(srcId, srcVersion)) {
+      final NodeStruct toBeAddedchild = findChild(srcId);
+      if (toBeAddedchild != null) {
+        LOG.warning(getQualifiedName() + "Child already exists");
+        final int childVersion = toBeAddedchild.getVersion();
+        if (srcVersion < childVersion) {
+          LOG.fine(getQualifiedName() + "Got an OOS child add msg. " + "Has LOWER ver-" + srcVersion
+              + " than child ver-" + childVersion + ". Discarding");
+          LOG.exiting("OperatorTopologyStructImpl", "updateChildAdd",
+              Arrays.toString(new Object[]{getQualifiedName(), srcId, srcVersion}));
+          return;
+        }
+        if (srcVersion > childVersion) {
+          LOG.fine(getQualifiedName() + "Got an OOS child add msg. " + "Has HIGHER ver-" + srcVersion
+              + " than child ver-" + childVersion + ". Bumping up version number");
+          toBeAddedchild.setVersion(srcVersion);
+          LOG.exiting("OperatorTopologyStructImpl", "updateChildAdd",
+              Arrays.toString(new Object[]{getQualifiedName(), srcId, srcVersion}));
+          return;
+        } else {
+          throw new RuntimeException(getQualifiedName() + "Got two child add msgs of same version-" + srcVersion);
+        }
+      } else {
+        LOG.finest(getQualifiedName() + "Creating new child node for " + srcId);
+        children.add(new ChildNodeStruct(srcId, srcVersion));
+      }
+    } else {
+      LOG.warning(getQualifiedName() + "Removed dead msg. Not adding child");
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "updateChildAdd", Arrays.toString(new Object[]{getQualifiedName(),
+        srcId, srcVersion}));
+  }
+
+  private void updateParentDead(final String srcId, final int srcVersion) {
+    LOG.entering("OperatorTopologyStructImpl", "updateParentDead",
+        new Object[]{getQualifiedName(), srcId, srcVersion});
+    if (!addedToDeadMsgs(parent, srcId, srcVersion)) {
+      final int parentVersion = parent.getVersion();
+      if (srcVersion < parentVersion) {
+        LOG.fine(getQualifiedName() + "Got an OOS parent dead msg. " + "Has LOWER ver-" + srcVersion
+            + " than parent ver-" + parentVersion + ". Discarding");
+        LOG.exiting("OperatorTopologyStructImpl", "updateParentDead",
+            Arrays.toString(new Object[]{getQualifiedName(), srcId, srcVersion}));
+        return;
+      } else {
+        LOG.finest(getQualifiedName() + "Got a parent dead msg. " + "Has SAME ver-" + srcVersion + " as parent ver-"
+            + parentVersion + "Setting parent node to null");
+      }
+    } else {
+      LOG.warning(getQualifiedName() + "Added to dead msgs. Setting parent to null since ParentAdd might not turn up");
+    }
+    parent = null;
+    LOG.exiting("OperatorTopologyStructImpl", "updateParentDead", Arrays.toString(new Object[]{getQualifiedName(),
+        srcId, srcVersion}));
+  }
+
+  private void updateParentAdd(final String srcId, final int srcVersion) {
+    LOG.entering("OperatorTopologyStructImpl", "updateParentAdd",
+        new Object[]{getQualifiedName(), srcId, srcVersion});
+    if (!removedDeadMsg(srcId, srcVersion)) {
+      if (parent != null) {
+        LOG.fine(getQualifiedName() + "Parent already exists");
+        final int parentVersion = parent.getVersion();
+        if (srcVersion < parentVersion) {
+          LOG.fine(getQualifiedName() + "Got an OOS parent add msg. " + "Has LOWER ver-" + srcVersion
+              + " than parent ver-" + parentVersion + ". Discarding");
+          LOG.exiting("OperatorTopologyStructImpl", "updateParentAdd",
+              Arrays.toString(new Object[]{getQualifiedName(), srcId, srcVersion}));
+          return;
+        }
+        if (srcVersion > parentVersion) {
+          LOG.fine(getQualifiedName() + "Got an OOS parent add msg. " + "Has HIGHER ver-" + srcVersion
+              + " than parent ver-" + parentVersion + ". Bumping up version number");
+          parent.setVersion(srcVersion);
+          LOG.exiting("OperatorTopologyStructImpl", "updateParentAdd",
+              Arrays.toString(new Object[]{getQualifiedName(), srcId, srcVersion}));
+          return;
+        } else {
+          throw new RuntimeException(getQualifiedName() + "Got two parent add msgs of same version-" + srcVersion);
+        }
+      } else {
+        LOG.finest(getQualifiedName() + "Creating new parent node for " + srcId);
+        parent = new ParentNodeStruct(srcId, srcVersion);
+      }
+    } else {
+      LOG.fine(getQualifiedName() + "Removed dead msg. Not adding parent");
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "updateParentAdd", Arrays.toString(new Object[]{getQualifiedName(),
+        srcId, srcVersion}));
+  }
+
+  /**
+   * @param srcId
+   * @return
+   */
+  private NodeStruct findChild(final String srcId) {
+    LOG.entering("OperatorTopologyStructImpl", "findChild", new Object[]{getQualifiedName(), srcId});
+    NodeStruct retVal = null;
+    for (final NodeStruct node : children) {
+      if (node.getId().equals(srcId)) {
+        retVal = node;
+        break;
+      }
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "findChild", Arrays.toString(new Object[]{retVal, getQualifiedName(),
+        srcId}));
+    return retVal;
+  }
+
+  @Override
+  public void update(final Set<GroupCommunicationMessage> deletionDeltas) {
+    LOG.entering("OperatorTopologyStructImpl", "update", new Object[]{"Updating topology with deleting msgs",
+        getQualifiedName(), deletionDeltas});
+    for (final GroupCommunicationMessage delDelta : deletionDeltas) {
+      update(delDelta);
+    }
+    LOG.exiting("OperatorTopologyStructImpl", "update", Arrays.toString(new Object[]{getQualifiedName(),
+        deletionDeltas}));
+  }
+
+  @Override
+  public void setChanges(final boolean changes) {
+    LOG.entering("OperatorTopologyStructImpl", "setChanges", new Object[]{getQualifiedName(), changes});
+    this.changes = changes;
+    LOG.exiting("OperatorTopologyStructImpl", "setChanges",
+        Arrays.toString(new Object[]{getQualifiedName(), changes}));
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":" + selfId + ":ver(" + version + ") - ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ParentNodeStruct.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ParentNodeStruct.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ParentNodeStruct.java
new file mode 100644
index 0000000..999f49d
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ParentNodeStruct.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.reef.io.network.group.impl.task;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+
+import java.util.logging.Logger;
+
+/**
+ *
+ */
+public class ParentNodeStruct extends NodeStructImpl {
+
+  private static final Logger LOG = Logger.getLogger(ParentNodeStruct.class.getName());
+
+  public ParentNodeStruct(final String id, final int version) {
+    super(id, version);
+  }
+
+  @Override
+  public boolean checkDead(final GroupCommunicationMessage gcm) {
+    LOG.entering("ParentNodeStruct", "checkDead", gcm);
+    final boolean retVal = gcm.getType() == ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentDead ? true : false;
+    LOG.exiting("ParentNodeStruct", "checkDead", gcm);
+    return retVal;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/BroadcastingEventHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/BroadcastingEventHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/BroadcastingEventHandler.java
new file mode 100644
index 0000000..cdc46fc
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/BroadcastingEventHandler.java
@@ -0,0 +1,44 @@
+/**
+ * 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.reef.io.network.group.impl.utils;
+
+import org.apache.reef.wake.EventHandler;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class BroadcastingEventHandler<T> implements EventHandler<T> {
+
+  List<EventHandler<T>> handlers = new ArrayList<>();
+
+  public void addHandler(final EventHandler<T> handler) {
+    handlers.add(handler);
+  }
+
+  @Override
+  public void onNext(final T msg) {
+    for (final EventHandler<T> handler : handlers) {
+      handler.onNext(msg);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ConcurrentCountingMap.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ConcurrentCountingMap.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ConcurrentCountingMap.java
new file mode 100644
index 0000000..7dc651b
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ConcurrentCountingMap.java
@@ -0,0 +1,134 @@
+/**
+ * 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.reef.io.network.group.impl.utils;
+
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Utility map class that wraps a CountingMap
+ * in a ConcurrentMap
+ * Equivalent to Map<K,Map<V,Integer>>
+ */
+public class ConcurrentCountingMap<K, V> {
+
+  private final ConcurrentMap<K, CountingMap<V>> map = new ConcurrentHashMap<>();
+
+  public boolean remove (final K key, final V value) {
+    if (!map.containsKey(key)) {
+      return false;
+    }
+    final boolean retVal = map.get(key).remove(value);
+    if (map.get(key).isEmpty()) {
+      map.remove(key);
+    }
+    return retVal;
+  }
+
+  public void add (final K key, final V value) {
+    map.putIfAbsent(key, new CountingMap<V>());
+    map.get(key).add(value);
+  }
+
+  public CountingMap<V> get (final K key) {
+    return map.get(key);
+  }
+
+  public boolean isEmpty () {
+    return map.isEmpty();
+  }
+
+  public boolean containsKey (final K key) {
+    return map.containsKey(key);
+  }
+
+  public boolean contains (final K key, final V value) {
+    if (!map.containsKey(key)) {
+      return false;
+    }
+    return map.get(key).containsKey(value);
+  }
+
+  public boolean notContains (final V value) {
+    for (final CountingMap<V> innerMap : map.values()) {
+      if (innerMap.containsKey(value)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public String toString () {
+    return map.toString();
+  }
+
+  public void clear () {
+    for (final CountingMap<V> value : map.values()) {
+      value.clear();
+    }
+    map.clear();
+  }
+
+  public static void main (final String[] args) {
+    final Logger LOG = Logger.getLogger(ConcurrentCountingMap.class.getName());
+    final ConcurrentCountingMap<ReefNetworkGroupCommProtos.GroupCommMessage.Type, String> strMap = new ConcurrentCountingMap<>();
+    LOG.log(Level.INFO, "OUT: {0}", strMap.isEmpty());
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST0");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST1");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildDead, "ST0");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildDead, "ST1");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST2");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST3");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST0");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST1");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.remove(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST2");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.remove(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST3");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.remove(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST0");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    LOG.log(Level.INFO, "OUT: {0}", strMap.get(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd));
+    LOG.log(Level.INFO, "OUT: {0}", strMap.get(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildDead));
+    strMap.remove(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST1");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.remove(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST1");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    LOG.log(Level.INFO, "OUT: {0}", strMap.containsKey(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd));
+    LOG.log(Level.INFO, "OUT: {0}", strMap.containsKey(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildDead));
+    LOG.log(Level.INFO, "OUT: {0}", strMap.contains(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST0"));
+    LOG.log(Level.INFO, "OUT: {0}", strMap.contains(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST2"));
+    strMap.remove(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "ST0");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    LOG.log(Level.INFO, "OUT: {0}", strMap.containsKey(ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd));
+    LOG.log(Level.INFO, "OUT: {0}", strMap.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingMap.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingMap.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingMap.java
new file mode 100644
index 0000000..58c3f5c
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingMap.java
@@ -0,0 +1,98 @@
+/**
+ * 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.reef.io.network.group.impl.utils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Utility class to provide a map that allows to
+ * add multiple keys and automatically
+ * incrementing the count on each add
+ * decrementing the count on each remove
+ * and removing key on count==0
+ */
+public class CountingMap<L> {
+
+  private final Map<L, Integer> map = new HashMap<>();
+
+  public boolean containsKey (final L value) {
+    return map.containsKey(value);
+  }
+
+  public int get (final L value) {
+    if (!containsKey(value)) {
+      return 0;
+    }
+    return map.get(value);
+  }
+
+  public boolean isEmpty () {
+    return map.isEmpty();
+  }
+
+  public void clear () {
+    map.clear();
+  }
+
+  public void add (final L value) {
+    int cnt = (map.containsKey(value)) ? map.get(value) : 0;
+    map.put(value, ++cnt);
+  }
+
+  public boolean remove (final L value) {
+    if (!map.containsKey(value)) {
+      return false;
+    }
+    int cnt = map.get(value);
+    --cnt;
+    if (cnt == 0) {
+      map.remove(value);
+    } else {
+      map.put(value, cnt);
+    }
+    return true;
+  }
+
+  @Override
+  public String toString () {
+    return map.toString();
+  }
+
+  public static void main (final String[] args) {
+    final Logger LOG = Logger.getLogger(CountingMap.class.getName());
+    final CountingMap<String> strMap = new CountingMap<>();
+    strMap.add("Hello");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add("World");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add("Hello");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add("Hello");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.add("World!");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.remove("Hello");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+    strMap.remove("World");
+    LOG.log(Level.INFO, "OUT: {0}", strMap);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingSemaphore.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingSemaphore.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingSemaphore.java
new file mode 100644
index 0000000..3014667
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/CountingSemaphore.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.reef.io.network.group.impl.utils;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+public class CountingSemaphore {
+
+  private static final Logger LOG = Logger.getLogger(CountingSemaphore.class.getName());
+
+  private final AtomicInteger counter;
+
+  private final String name;
+
+  private final Object lock;
+
+  private final int initCount;
+
+  public CountingSemaphore (final int initCount, final String name, final Object lock) {
+    super();
+    this.initCount = initCount;
+    this.name = name;
+    this.lock = lock;
+    this.counter = new AtomicInteger(initCount);
+    LOG.finest("Counter initialized to " + initCount);
+  }
+
+  public int getInitialCount() {
+    return initCount;
+  }
+
+  public int increment () {
+    synchronized (lock) {
+      final int retVal = counter.incrementAndGet();
+      LOG.finest(name + "Incremented counter to " + retVal);
+      logStatus();
+      return retVal;
+    }
+  }
+
+  private void logStatus () {
+    final int yetToRun = counter.get();
+    final int curRunning = initCount - yetToRun;
+    LOG.fine(name + curRunning + " workers are running & " + yetToRun + " workers are yet to run");
+  }
+
+  public int decrement () {
+    synchronized (lock) {
+      final int retVal = counter.decrementAndGet();
+      LOG.finest(name + "Decremented counter to " + retVal);
+      if (retVal < 0) {
+        LOG.warning("Counter negative. More workers exist than you expected");
+      }
+      if (retVal <= 0) {
+        LOG.finest(name + "All workers are done with their task. Notifying waiting threads");
+        lock.notifyAll();
+      } else {
+        LOG.finest(name + "Some workers are not done yet");
+      }
+      logStatus();
+      return retVal;
+    }
+  }
+
+  public int get () {
+    synchronized (lock) {
+      return counter.get();
+    }
+  }
+
+  public void await () {
+    synchronized (lock) {
+      LOG.finest(name + "Waiting for workers to be done");
+      while (counter.get() > 0) {
+        try {
+          lock.wait();
+          LOG.finest(name + "Notified with counter=" + counter.get());
+        } catch (final InterruptedException e) {
+          throw new RuntimeException("InterruptedException while waiting for counting semaphore counter", e);
+        }
+      }
+      LOG.finest(name + "Returning from wait");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ResettingCountDownLatch.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ResettingCountDownLatch.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ResettingCountDownLatch.java
new file mode 100644
index 0000000..30631c8
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/ResettingCountDownLatch.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.impl.utils;
+
+import java.util.concurrent.CountDownLatch;
+
+public class ResettingCountDownLatch {
+  private CountDownLatch latch;
+
+  public ResettingCountDownLatch (final int initialCount) {
+    latch = new CountDownLatch(initialCount);
+  }
+
+  /**
+   *
+   */
+  public void await () {
+    try {
+      latch.await();
+    } catch (final InterruptedException e) {
+      throw new RuntimeException("InterruptedException while waiting for latch", e);
+    }
+  }
+
+  public void awaitAndReset (final int resetCount) {
+    try {
+      latch.await();
+      latch = new CountDownLatch(resetCount);
+    } catch (final InterruptedException e) {
+      throw new RuntimeException("InterruptedException while waiting for latch", e);
+    }
+  }
+
+  /**
+   *
+   */
+  public void countDown () {
+    latch.countDown();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/SetMap.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/SetMap.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/SetMap.java
new file mode 100644
index 0000000..6a4b424
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/SetMap.java
@@ -0,0 +1,95 @@
+/**
+ * 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.reef.io.network.group.impl.utils;
+
+import org.apache.reef.io.network.group.impl.driver.MsgKey;
+
+import java.util.*;
+
+/**
+ * Map from K to Set<V>
+ */
+public class SetMap<K, V> {
+  private final Map<K, Set<V>> map = new HashMap<>();
+
+  public boolean containsKey(final K key) {
+    return map.containsKey(key);
+  }
+
+  public boolean contains(final K key, final V value) {
+    if (!containsKey(key)) {
+      return false;
+    }
+    return map.get(key).contains(value);
+  }
+
+  public Set<V> get(final K key) {
+    if (map.containsKey(key)) {
+      return map.get(key);
+    } else {
+      return Collections.emptySet();
+    }
+  }
+
+  public void add(final K key, final V value) {
+    final Set<V> values;
+    if (!map.containsKey(key)) {
+      values = new HashSet<>();
+      map.put(key, values);
+    } else {
+      values = map.get(key);
+    }
+    values.add(value);
+  }
+
+  public boolean remove(final K key, final V value) {
+    if (!map.containsKey(key)) {
+      return false;
+    }
+    final Set<V> set = map.get(key);
+    final boolean retVal = set.remove(value);
+    if (set.isEmpty()) {
+      map.remove(key);
+    }
+    return retVal;
+  }
+
+  /**
+   * @param key
+   * @return
+   */
+  public int count(final K key) {
+    if (!containsKey(key)) {
+      return 0;
+    } else {
+      return map.get(key).size();
+    }
+  }
+
+  /**
+   * @param key
+   */
+  public Set<V> remove(final MsgKey key) {
+    return map.remove(key);
+  }
+
+  public Set<K> keySet() {
+    return map.keySet();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/Utils.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/Utils.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/Utils.java
new file mode 100644
index 0000000..27811cb
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/utils/Utils.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.reef.io.network.group.impl.utils;
+
+import org.apache.reef.io.network.Message;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.annotations.Name;
+
+import java.util.Iterator;
+
+/**
+ *
+ */
+public class Utils {
+
+  public static final byte[] EmptyByteArr = new byte[0];
+
+  public static GroupCommunicationMessage bldVersionedGCM(final Class<? extends Name<String>> groupName,
+                                                          final Class<? extends Name<String>> operName,
+                                                          final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType, final String from, final int srcVersion,
+                                                          final String to, final int dstVersion, final byte[]... data) {
+
+    return new GroupCommunicationMessage(groupName.getName(), operName.getName(), msgType, from, srcVersion, to,
+        dstVersion, data);
+  }
+
+  public static Class<? extends Name<String>> getClass(final String className) {
+    try {
+      return (Class<? extends Name<String>>) Class.forName(className);
+    } catch (final ClassNotFoundException e) {
+      throw new RuntimeException("Unable to find class " + className, e);
+    }
+  }
+
+  public static String simpleName(final Class<?> className) {
+    if (className != null) {
+      return className.getSimpleName();
+    } else {
+      return "NULL";
+    }
+  }
+
+  public static byte[] getData(final GroupCommunicationMessage gcm) {
+    return (gcm.getMsgsCount() == 1) ? gcm.getData()[0] : null;
+  }
+
+  /**
+   * @param msg
+   * @return
+   */
+  public static GroupCommunicationMessage getGCM(final Message<GroupCommunicationMessage> msg) {
+    final Iterator<GroupCommunicationMessage> gcmIterator = msg.getData().iterator();
+    if (gcmIterator.hasNext()) {
+      final GroupCommunicationMessage gcm = gcmIterator.next();
+      if (gcmIterator.hasNext()) {
+        throw new RuntimeException("Expecting exactly one GCM object inside Message but found more");
+      }
+      return gcm;
+    } else {
+      throw new RuntimeException("Expecting exactly one GCM object inside Message but found none");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/package-info.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/package-info.java
new file mode 100644
index 0000000..4946182
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/package-info.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.
+ */
+
+/**
+ * Elastic Group Communications for REEF.
+ *
+ * Provides MPI style Group Communication operators for collective communication
+ * between tasks. These should be primarily used for any form of
+ * task to task messaging along with the point to point communication
+ * provided by {@link org.apache.reef.io.network.impl.NetworkService}
+ *
+ * The interfaces for the operators are in org.apache.reef.io.network.group.api.operators
+ * The fluent way to describe these operators is available org.apache.reef.io.network.group.config
+ * The implementation of these operators are available in org.apache.reef.io.network.group.impl
+ * Currently only a basic implementation is available
+ */
+package org.apache.reef.io.network.group;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServer.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServer.java
index 5a4c765..0737286 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServer.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServer.java
@@ -20,29 +20,11 @@
 package org.apache.reef.io.network.naming;
 
 import org.apache.reef.io.naming.NameAssignment;
-import org.apache.reef.io.network.naming.serialization.*;
-import org.apache.reef.tang.annotations.DefaultImplementation;
-import org.apache.reef.tang.annotations.Parameter;
-import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.Identifier;
-import org.apache.reef.wake.IdentifierFactory;
 import org.apache.reef.wake.Stage;
-import org.apache.reef.wake.impl.MultiEventHandler;
-import org.apache.reef.wake.impl.SyncStage;
-import org.apache.reef.wake.remote.Codec;
-import org.apache.reef.wake.remote.NetUtils;
-import org.apache.reef.wake.remote.impl.TransportEvent;
-import org.apache.reef.wake.remote.transport.Transport;
-import org.apache.reef.wake.remote.transport.netty.NettyMessagingTransport;
-import org.apache.reef.webserver.AvroReefServiceInfo;
-import org.apache.reef.webserver.ReefEventStateManager;
 
-import javax.inject.Inject;
-import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.*;
-import java.util.logging.Level;
-import java.util.logging.Logger;
+import java.util.List;
 
 /**
  * Naming server interface

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
index 4cb0cc6..ff241a0 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/naming/NameServerImpl.java
@@ -24,7 +24,6 @@ import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
-import org.apache.reef.wake.Stage;
 import org.apache.reef.wake.impl.MultiEventHandler;
 import org.apache.reef.wake.impl.SyncStage;
 import org.apache.reef.wake.remote.Codec;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/util/Utils.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/util/Utils.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/util/Utils.java
new file mode 100644
index 0000000..3e58f36
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/util/Utils.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.reef.io.network.util;
+
+import com.google.protobuf.ByteString;
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.wake.ComparableIdentifier;
+import org.apache.reef.wake.Identifier;
+import org.apache.reef.wake.IdentifierFactory;
+
+import java.net.Inet4Address;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+public class Utils {
+
+  private static final String DELIMITER = "-";
+
+  /**
+   * TODO: Merge with parseListCmp() into one generic implementation.
+   */
+  public static List<Identifier> parseList(
+      final String ids, final IdentifierFactory factory) {
+    final List<Identifier> result = new ArrayList<>();
+    for (final String token : ids.split(DELIMITER)) {
+      result.add(factory.getNewInstance(token.trim()));
+    }
+    return result;
+  }
+
+  /**
+   * TODO: Merge with parseList() into one generic implementation.
+   */
+  public static List<ComparableIdentifier> parseListCmp(
+      final String ids, final IdentifierFactory factory) {
+    final List<ComparableIdentifier> result = new ArrayList<>();
+    for (final String token : ids.split(DELIMITER)) {
+      result.add((ComparableIdentifier) factory.getNewInstance(token.trim()));
+    }
+    return result;
+  }
+
+  public static String listToString(final List<ComparableIdentifier> ids) {
+    return StringUtils.join(ids, DELIMITER);
+  }
+
+  public static List<Integer> createUniformCounts(final int elemSize, final int childSize) {
+    final int remainder = elemSize % childSize;
+    final int quotient = elemSize / childSize;
+    final ArrayList<Integer> result = new ArrayList<>(childSize);
+    result.addAll(Collections.nCopies(remainder, quotient + 1));
+    result.addAll(Collections.nCopies(childSize - remainder, quotient));
+    return Collections.unmodifiableList(result);
+  }
+
+  private static class AddressComparator implements Comparator<Inet4Address> {
+    @Override
+    public int compare(final Inet4Address aa, final Inet4Address ba) {
+      final byte[] a = aa.getAddress();
+      final byte[] b = ba.getAddress();
+      // local subnet comes after all else.
+      if (a[0] == 127 && b[0] != 127) {
+        return 1;
+      }
+      if (a[0] != 127 && b[0] == 127) {
+        return -1;
+      }
+      for (int i = 0; i < 4; i++) {
+        if (a[i] < b[i]) {
+          return -1;
+        }
+        if (a[i] > b[i]) {
+          return 1;
+        }
+      }
+      return 0;
+    }
+  }
+
+  public static ReefNetworkGroupCommProtos.GroupCommMessage bldGCM(
+      final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType,
+      final Identifier from, final Identifier to, final byte[]... elements) {
+
+    final ReefNetworkGroupCommProtos.GroupCommMessage.Builder GCMBuilder =
+        ReefNetworkGroupCommProtos.GroupCommMessage.newBuilder()
+            .setType(msgType)
+            .setSrcid(from.toString())
+            .setDestid(to.toString());
+
+    final ReefNetworkGroupCommProtos.GroupMessageBody.Builder bodyBuilder =
+        ReefNetworkGroupCommProtos.GroupMessageBody.newBuilder();
+
+    for (final byte[] element : elements) {
+      bodyBuilder.setData(ByteString.copyFrom(element));
+      GCMBuilder.addMsgs(bodyBuilder.build());
+    }
+
+    return GCMBuilder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/storage/ram/RamMap.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/storage/ram/RamMap.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/storage/ram/RamMap.java
index 35a20e1..ffceb5d 100644
--- a/lang/java/reef-io/src/main/java/org/apache/reef/io/storage/ram/RamMap.java
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/storage/ram/RamMap.java
@@ -33,12 +33,11 @@ import java.util.concurrent.ConcurrentSkipListMap;
  * default ExternalMap provided by StorageManagerRam.
  */
 public class RamMap<T> implements ExternalMap<T> {
-  private final ConcurrentSkipListMap<CharSequence, T> map
-      = new ConcurrentSkipListMap<CharSequence, T>();
+
+  private final ConcurrentSkipListMap<CharSequence, T> map = new ConcurrentSkipListMap<CharSequence, T>();
 
   @Inject
   public RamMap(RamStorageService ramStore) {
-    //this.localStore = localStore;
   }
 
   @Override
@@ -70,5 +69,4 @@ public class RamMap<T> implements ExternalMap<T> {
   public Iterable<Entry<CharSequence, T>> getAll(Set<? extends CharSequence> keys) {
     return new GetAllIterable<>(keys, this);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/proto/group_comm_protocol.proto
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/proto/group_comm_protocol.proto b/lang/java/reef-io/src/main/proto/group_comm_protocol.proto
new file mode 100644
index 0000000..7e2f60f
--- /dev/null
+++ b/lang/java/reef-io/src/main/proto/group_comm_protocol.proto
@@ -0,0 +1,64 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.reef.io.network.proto";
+option java_outer_classname = "ReefNetworkGroupCommProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+message GroupCommMessage {
+  enum Type { 
+  	Scatter=1; 
+  	Gather=2; 
+  	Broadcast=3; 
+  	Reduce=4; 
+  	AllGather=5; 
+  	AllReduce=6; 
+  	ReduceScatter=7; 
+  	SourceDead=8; 
+  	SourceAdd=9; 
+  	ParentAdd=10; 
+  	ChildAdd=11;
+  	ParentDead=12; 
+  	ChildDead=13;
+  	ParentAdded=14; 
+  	ChildAdded=15;
+  	ParentRemoved=16; 
+  	ChildRemoved=17;
+  	TopologySetup=18;
+  	UpdateTopology=19;
+  	TopologyUpdated=20;
+  	TopologyChanges=21;
+  }
+
+  // identifies which field is filled in
+  required Type type = 1;
+  
+  required string srcid = 2;
+  required string destid = 3;
+  optional string groupname = 4;
+  optional string operatorname = 5;
+  optional int32 version = 6;
+  optional int32 srcVersion = 7;
+  repeated GroupMessageBody msgs = 8;
+}
+
+message GroupMessageBody {
+  required bytes data = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/test/java/org/apache/reef/io/network/group/GroupCommunicationMessageCodecTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/test/java/org/apache/reef/io/network/group/GroupCommunicationMessageCodecTest.java b/lang/java/reef-io/src/test/java/org/apache/reef/io/network/group/GroupCommunicationMessageCodecTest.java
new file mode 100644
index 0000000..a83daa6
--- /dev/null
+++ b/lang/java/reef-io/src/test/java/org/apache/reef/io/network/group/GroupCommunicationMessageCodecTest.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.reef.io.network.group;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessageCodec;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.Random;
+
+/**
+ *
+ */
+public class GroupCommunicationMessageCodecTest {
+
+  @NamedParameter
+  class GroupName implements Name<String> {
+  }
+
+  @NamedParameter
+  class OperName implements Name<String> {
+  }
+
+  @Test(timeout = 100)
+  public final void testInstantiation() throws InjectionException {
+    final GroupCommunicationMessageCodec codec = Tang.Factory.getTang().newInjector().getInstance(GroupCommunicationMessageCodec.class);
+    Assert.assertNotNull("tang.getInstance(GroupCommunicationMessageCodec.class): ", codec);
+  }
+
+  @Test(timeout = 100)
+  public final void testEncodeDecode() {
+    final Random r = new Random();
+    final byte[] data = new byte[100];
+    r.nextBytes(data);
+    final GroupCommunicationMessage expMsg = Utils.bldVersionedGCM(GroupName.class, OperName.class, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, "From", 0, "To", 1, data);
+    final GroupCommunicationMessageCodec codec = new GroupCommunicationMessageCodec();
+    final GroupCommunicationMessage actMsg1 = codec.decode(codec.encode(expMsg));
+    Assert.assertEquals("decode(encode(msg)): ", expMsg, actMsg1);
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    final DataOutputStream daos = new DataOutputStream(baos);
+    codec.encodeToStream(expMsg, daos);
+    final GroupCommunicationMessage actMsg2 = codec.decodeFromStream(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
+    Assert.assertEquals("decodeFromStream(encodeToStream(msg)): ", expMsg, actMsg2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/test/java/org/apache/reef/io/network/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/test/java/org/apache/reef/io/network/util/TestUtils.java b/lang/java/reef-io/src/test/java/org/apache/reef/io/network/util/TestUtils.java
new file mode 100644
index 0000000..a41cf04
--- /dev/null
+++ b/lang/java/reef-io/src/test/java/org/apache/reef/io/network/util/TestUtils.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.reef.io.network.util;
+
+import com.google.protobuf.ByteString;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.wake.Identifier;
+
+public class TestUtils {
+  public static ReefNetworkGroupCommProtos.GroupCommMessage bldGCM(final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType, final Identifier from, final Identifier to, final byte[]... elements) {
+    final ReefNetworkGroupCommProtos.GroupCommMessage.Builder GCMBuilder = ReefNetworkGroupCommProtos.GroupCommMessage.newBuilder();
+    GCMBuilder.setType(msgType);
+    GCMBuilder.setSrcid(from.toString());
+    GCMBuilder.setDestid(to.toString());
+    final ReefNetworkGroupCommProtos.GroupMessageBody.Builder bodyBuilder = ReefNetworkGroupCommProtos.GroupMessageBody.newBuilder();
+    for (final byte[] element : elements) {
+      bodyBuilder.setData(ByteString.copyFrom(element));
+      GCMBuilder.addMsgs(bodyBuilder.build());
+    }
+    final ReefNetworkGroupCommProtos.GroupCommMessage msg = GCMBuilder.build();
+    return msg;
+  }
+
+  /**
+   * @param type
+   * @return
+   */
+  public static boolean controlMessage(final ReefNetworkGroupCommProtos.GroupCommMessage.Type type) {
+
+    switch (type) {
+      case AllGather:
+      case AllReduce:
+      case Broadcast:
+      case Gather:
+      case Reduce:
+      case ReduceScatter:
+      case Scatter:
+        return false;
+
+      default:
+        return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
index 96cf7d4..17765f1 100644
--- a/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
+++ b/lang/java/reef-io/src/test/java/org/apache/reef/services/network/NetworkServiceTest.java
@@ -28,8 +28,6 @@ import org.apache.reef.io.network.naming.NameServerImpl;
 import org.apache.reef.io.network.util.StringIdentifierFactory;
 import org.apache.reef.services.network.util.Monitor;
 import org.apache.reef.services.network.util.StringCodec;
-import org.apache.reef.tang.Injector;
-import org.apache.reef.tang.Tang;
 import org.apache.reef.wake.EventHandler;
 import org.apache.reef.wake.Identifier;
 import org.apache.reef.wake.IdentifierFactory;
@@ -37,7 +35,7 @@ import org.apache.reef.wake.remote.NetUtils;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-import org.junit.Assert;
+
 import java.net.InetSocketAddress;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -55,8 +53,6 @@ public class NetworkServiceTest {
 
   /**
    * NetworkService messaging test
-   *
-   * @throws Exception
    */
   @Test
   public void testMessagingNetworkService() throws Exception {
@@ -113,8 +109,6 @@ public class NetworkServiceTest {
 
   /**
    * NetworkService messaging rate benchmark
-   *
-   * @throws Exception
    */
   @Test
   public void testMessagingNetworkServiceRate() throws Exception {
@@ -187,8 +181,6 @@ public class NetworkServiceTest {
 
   /**
    * NetworkService messaging rate benchmark
-   *
-   * @throws Exception
    */
   @Test
   public void testMessagingNetworkServiceRateDisjoint() throws Exception {
@@ -372,8 +364,6 @@ public class NetworkServiceTest {
 
   /**
    * NetworkService messaging rate benchmark
-   *
-   * @throws Exception
    */
   @Test
   public void testMessagingNetworkServiceBatchingRate() throws Exception {
@@ -452,8 +442,6 @@ public class NetworkServiceTest {
 
   /**
    * Test message handler
-   *
-   * @param <T> type
    */
   class MessageHandler<T> implements EventHandler<Message<T>> {
 
@@ -470,13 +458,17 @@ public class NetworkServiceTest {
 
     @Override
     public void onNext(Message<T> value) {
+
       count.incrementAndGet();
 
-      //System.out.print(name + " received " + value.getData() + " from " + value.getSrcId() + " to " + value.getDestId());
-      for (T obj : value.getData()) {
-        // System.out.print(" data: " + obj);
+      LOG.log(Level.FINEST,
+          "OUT: {0} received {1} from {2} to {3}",
+          new Object[] { name, value.getData(), value.getSrcId(), value.getDestId() });
+
+      for (final T obj : value.getData()) {
+        LOG.log(Level.FINEST, "OUT: data: {0}", obj);
       }
-      //LOG.log(Level.FINEST, );
+
       if (count.get() == expected) {
         monitor.mnotify();
       }


[2/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
new file mode 100644
index 0000000..3a6791c
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceReceiver.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.impl.operators;
+
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.api.task.CommGroupNetworkHandler;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupServiceClient;
+import org.apache.reef.io.network.group.api.task.OperatorTopology;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.parameters.*;
+import org.apache.reef.io.network.group.impl.task.OperatorTopologyImpl;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.Identifier;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+public class ReduceReceiver<T> implements Reduce.Receiver<T>, EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(ReduceReceiver.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final CommGroupNetworkHandler commGroupNetworkHandler;
+  private final Codec<T> dataCodec;
+  private final NetworkService<GroupCommunicationMessage> netService;
+  private final Sender sender;
+  private final ReduceFunction<T> reduceFunction;
+
+  private final OperatorTopology topology;
+
+  private final CommunicationGroupServiceClient commGroupClient;
+
+  private final AtomicBoolean init = new AtomicBoolean(false);
+
+  private final int version;
+
+  @Inject
+  public ReduceReceiver(@Parameter(CommunicationGroupName.class) final String groupName,
+                        @Parameter(OperatorName.class) final String operName,
+                        @Parameter(TaskConfigurationOptions.Identifier.class) final String selfId,
+                        @Parameter(DataCodec.class) final Codec<T> dataCodec,
+                        @Parameter(ReduceFunctionParam.class) final ReduceFunction<T> reduceFunction,
+                        @Parameter(DriverIdentifier.class) final String driverId,
+                        @Parameter(TaskVersion.class) final int version,
+                        final CommGroupNetworkHandler commGroupNetworkHandler,
+                        final NetworkService<GroupCommunicationMessage> netService,
+                        final CommunicationGroupServiceClient commGroupClient) {
+    super();
+    this.version = version;
+    LOG.finest(operName + " has CommGroupHandler-" + commGroupNetworkHandler.toString());
+    this.groupName = Utils.getClass(groupName);
+    this.operName = Utils.getClass(operName);
+    this.dataCodec = dataCodec;
+    this.reduceFunction = reduceFunction;
+    this.commGroupNetworkHandler = commGroupNetworkHandler;
+    this.netService = netService;
+    this.sender = new Sender(this.netService);
+    this.topology = new OperatorTopologyImpl(this.groupName, this.operName, selfId, driverId, sender, version);
+    this.commGroupNetworkHandler.register(this.operName, this);
+    this.commGroupClient = commGroupClient;
+  }
+
+  @Override
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void initialize() throws ParentDeadException {
+    topology.initialize();
+  }
+
+  @Override
+  public Class<? extends Name<String>> getOperName() {
+    return operName;
+  }
+
+  @Override
+  public Class<? extends Name<String>> getGroupName() {
+    return groupName;
+  }
+
+  @Override
+  public String toString() {
+    return "ReduceReceiver:" + Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":" + version;
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    topology.handle(msg);
+  }
+
+  @Override
+  public T reduce() throws InterruptedException, NetworkException {
+    LOG.entering("ReduceReceiver", "reduce", this);
+    LOG.fine("I am " + this);
+
+    if (init.compareAndSet(false, true)) {
+      commGroupClient.initialize();
+    }
+    // I am root
+    LOG.fine(this + " Waiting to receive reduced value");
+    // Wait for children to send
+    final T redVal;
+    try {
+      redVal = topology.recvFromChildren(reduceFunction, dataCodec);
+    } catch (final ParentDeadException e) {
+      throw new RuntimeException("ParentDeadException", e);
+    }
+    LOG.fine(this + " Received Reduced value: " + (redVal != null ? redVal : "NULL"));
+    LOG.exiting("ReduceReceiver", "reduce", Arrays.toString(new Object[]{redVal}));
+    return redVal;
+  }
+
+  @Override
+  public T reduce(final List<? extends Identifier> order) throws InterruptedException, NetworkException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ReduceFunction<T> getReduceFunction() {
+    return reduceFunction;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
new file mode 100644
index 0000000..505f072
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/ReduceSender.java
@@ -0,0 +1,161 @@
+/**
+ * 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.reef.io.network.group.impl.operators;
+
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.api.task.CommGroupNetworkHandler;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupServiceClient;
+import org.apache.reef.io.network.group.api.task.OperatorTopology;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.parameters.*;
+import org.apache.reef.io.network.group.impl.task.OperatorTopologyImpl;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class ReduceSender<T> implements Reduce.Sender<T>, EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(ReduceSender.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final CommGroupNetworkHandler commGroupNetworkHandler;
+  private final Codec<T> dataCodec;
+  private final NetworkService<GroupCommunicationMessage> netService;
+  private final Sender sender;
+  private final ReduceFunction<T> reduceFunction;
+
+  private final OperatorTopology topology;
+
+  private final CommunicationGroupServiceClient commGroupClient;
+
+  private final AtomicBoolean init = new AtomicBoolean(false);
+
+  private final int version;
+
+  @Inject
+  public ReduceSender(
+      final @Parameter(CommunicationGroupName.class) String groupName,
+      final @Parameter(OperatorName.class) String operName,
+      final @Parameter(TaskConfigurationOptions.Identifier.class) String selfId,
+      final @Parameter(DataCodec.class) Codec<T> dataCodec,
+      final @Parameter(ReduceFunctionParam.class) ReduceFunction<T> reduceFunction,
+      final @Parameter(DriverIdentifier.class) String driverId,
+      final @Parameter(TaskVersion.class) int version,
+      final CommGroupNetworkHandler commGroupNetworkHandler,
+      final NetworkService<GroupCommunicationMessage> netService,
+      final CommunicationGroupServiceClient commGroupClient) {
+
+    super();
+
+    LOG.log(Level.FINEST, "{0} has CommGroupHandler-{1}",
+        new Object[]{operName, commGroupNetworkHandler});
+
+    this.version = version;
+    this.groupName = Utils.getClass(groupName);
+    this.operName = Utils.getClass(operName);
+    this.dataCodec = dataCodec;
+    this.reduceFunction = reduceFunction;
+    this.commGroupNetworkHandler = commGroupNetworkHandler;
+    this.netService = netService;
+    this.sender = new Sender(this.netService);
+    this.topology = new OperatorTopologyImpl(this.groupName, this.operName, selfId, driverId, sender, version);
+    this.commGroupNetworkHandler.register(this.operName, this);
+    this.commGroupClient = commGroupClient;
+  }
+
+  @Override
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void initialize() throws ParentDeadException {
+    topology.initialize();
+  }
+
+  @Override
+  public Class<? extends Name<String>> getOperName() {
+    return operName;
+  }
+
+  @Override
+  public Class<? extends Name<String>> getGroupName() {
+    return groupName;
+  }
+
+  @Override
+  public String toString() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":" + version;
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    topology.handle(msg);
+  }
+
+  @Override
+  public void send(final T myData) throws NetworkException, InterruptedException {
+    LOG.entering("ReduceSender", "send", new Object[]{this, myData});
+    LOG.fine("I am " + this);
+
+    if (init.compareAndSet(false, true)) {
+      commGroupClient.initialize();
+    }
+    // I am an intermediate node or leaf.
+    LOG.finest("Waiting for children");
+    // Wait for children to send
+    try {
+      final T reducedValueOfChildren = topology.recvFromChildren(reduceFunction, dataCodec);
+      final List<T> vals = new ArrayList<>(2);
+      vals.add(myData);
+      if (reducedValueOfChildren != null) {
+        vals.add(reducedValueOfChildren);
+      }
+      final T reducedValue = reduceFunction.apply(vals);
+      LOG.fine(this + " Sending local " + reducedValue + " to parent");
+      topology.sendToParent(dataCodec.encode(reducedValue), ReefNetworkGroupCommProtos.GroupCommMessage.Type.Reduce);
+    } catch (final ParentDeadException e) {
+      throw new RuntimeException("ParentDeadException", e);
+    }
+    LOG.exiting("ReduceSender", "send", Arrays.toString(new Object[]{this, myData}));
+  }
+
+  @Override
+  public ReduceFunction<T> getReduceFunction() {
+    return reduceFunction;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.java
new file mode 100644
index 0000000..b124906
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/Sender.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.reef.io.network.group.impl.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.Connection;
+import org.apache.reef.io.network.group.api.operators.AbstractGroupCommOperator;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.util.StringIdentifierFactory;
+import org.apache.reef.wake.Identifier;
+import org.apache.reef.wake.IdentifierFactory;
+
+import java.util.Arrays;
+import java.util.logging.Logger;
+
+public class Sender extends AbstractGroupCommOperator {
+
+  private static final Logger LOG = Logger.getLogger(Sender.class.getName());
+
+  private final NetworkService<GroupCommunicationMessage> netService;
+  private final IdentifierFactory idFac = new StringIdentifierFactory();
+
+  public Sender(final NetworkService<GroupCommunicationMessage> netService) {
+    this.netService = netService;
+  }
+
+  public void send(final GroupCommunicationMessage msg) throws NetworkException {
+    LOG.entering("Sender", "send", msg);
+    final String dest = msg.getDestid();
+    send(msg, dest);
+    LOG.exiting("Sender", "send", msg);
+  }
+
+  public void send(final GroupCommunicationMessage msg, final String dest) throws NetworkException {
+    LOG.entering("Sender", "send", new Object[]{msg, dest});
+    final Identifier destId = idFac.getNewInstance(dest);
+    final Connection<GroupCommunicationMessage> link = netService.newConnection(destId);
+    link.open();
+    link.write(msg);
+    LOG.exiting("Sender", "send", Arrays.toString(new Object[]{msg, dest}));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ChildNodeStruct.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ChildNodeStruct.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ChildNodeStruct.java
new file mode 100644
index 0000000..b69ec5d
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/ChildNodeStruct.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.impl.task;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+
+import java.util.logging.Logger;
+
+public class ChildNodeStruct extends NodeStructImpl {
+
+  private static final Logger LOG = Logger.getLogger(ChildNodeStruct.class.getName());
+
+  public ChildNodeStruct(final String id, final int version) {
+    super(id, version);
+  }
+
+  @Override
+  public boolean checkDead(final GroupCommunicationMessage gcm) {
+    LOG.entering("ChildNodeStruct", "checkDead", gcm);
+    final boolean retVal = gcm.getType() == ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildDead ? true : false;
+    LOG.exiting("ChildNodeStruct", "checkDead", gcm);
+    return retVal;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommGroupNetworkHandlerImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommGroupNetworkHandlerImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommGroupNetworkHandlerImpl.java
new file mode 100644
index 0000000..72af1ec
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommGroupNetworkHandlerImpl.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.reef.io.network.group.impl.task;
+
+import org.apache.reef.io.network.group.api.task.CommGroupNetworkHandler;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Logger;
+
+public class CommGroupNetworkHandlerImpl implements
+    CommGroupNetworkHandler {
+
+  private static final Logger LOG = Logger.getLogger(CommGroupNetworkHandlerImpl.class.getName());
+
+  private final Map<Class<? extends Name<String>>, EventHandler<GroupCommunicationMessage>> operHandlers = new ConcurrentHashMap<>();
+  private final Map<Class<? extends Name<String>>, BlockingQueue<GroupCommunicationMessage>> topologyNotifications = new ConcurrentHashMap<>();
+
+  @Inject
+  public CommGroupNetworkHandlerImpl() {
+  }
+
+  @Override
+  public void register(final Class<? extends Name<String>> operName,
+                       final EventHandler<GroupCommunicationMessage> operHandler) {
+    LOG.entering("CommGroupNetworkHandlerImpl", "register", new Object[]{Utils.simpleName(operName), operHandler});
+    operHandlers.put(operName, operHandler);
+    LOG.exiting("CommGroupNetworkHandlerImpl", "register", Arrays.toString(new Object[]{Utils.simpleName(operName), operHandler}));
+  }
+
+  @Override
+  public void addTopologyElement(final Class<? extends Name<String>> operName) {
+    LOG.entering("CommGroupNetworkHandlerImpl", "addTopologyElement", Utils.simpleName(operName));
+    LOG.finest("Creating LBQ for " + operName);
+    topologyNotifications.put(operName, new LinkedBlockingQueue<GroupCommunicationMessage>());
+    LOG.exiting("CommGroupNetworkHandlerImpl", "addTopologyElement", Utils.simpleName(operName));
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    LOG.entering("CommGroupNetworkHandlerImpl", "onNext", msg);
+    final Class<? extends Name<String>> operName = Utils.getClass(msg.getOperatorname());
+    if (msg.getType() == ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyUpdated || msg.getType() == ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyChanges) {
+      topologyNotifications.get(operName).add(msg);
+    } else {
+      operHandlers.get(operName).onNext(msg);
+    }
+    LOG.exiting("CommGroupNetworkHandlerImpl", "onNext", msg);
+  }
+
+  @Override
+  public byte[] waitForTopologyChanges(final Class<? extends Name<String>> operName) {
+    LOG.entering("CommGroupNetworkHandlerImpl", "waitForTopologyChanges", Utils.simpleName(operName));
+    try {
+      final byte[] retVal = Utils.getData(topologyNotifications.get(operName).take());
+      LOG.exiting("CommGroupNetworkHandlerImpl", "waitForTopologyChanges", retVal);
+      return retVal;
+    } catch (final InterruptedException e) {
+      throw new RuntimeException("InterruptedException while waiting for topology update of "
+          + operName.getSimpleName(), e);
+    }
+  }
+
+  @Override
+  public GroupCommunicationMessage waitForTopologyUpdate(final Class<? extends Name<String>> operName) {
+    LOG.entering("CommGroupNetworkHandlerImpl", "waitForTopologyUpdate", Utils.simpleName(operName));
+    try {
+      final GroupCommunicationMessage retVal = topologyNotifications.get(operName).take();
+      LOG.exiting("CommGroupNetworkHandlerImpl", "waitForTopologyUpdate", retVal);
+      return retVal;
+    } catch (final InterruptedException e) {
+      throw new RuntimeException("InterruptedException while waiting for topology update of "
+          + operName.getSimpleName(), e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommunicationGroupClientImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommunicationGroupClientImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommunicationGroupClientImpl.java
new file mode 100644
index 0000000..d0e6e9e
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/CommunicationGroupClientImpl.java
@@ -0,0 +1,296 @@
+/**
+ * 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.reef.io.network.group.impl.task;
+
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.group.api.operators.GroupCommOperator;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.network.group.api.task.CommGroupNetworkHandler;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupServiceClient;
+import org.apache.reef.io.network.group.api.task.GroupCommNetworkHandler;
+import org.apache.reef.io.network.group.impl.GroupChangesCodec;
+import org.apache.reef.io.network.group.impl.GroupChangesImpl;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.parameters.CommunicationGroupName;
+import org.apache.reef.io.network.group.impl.config.parameters.OperatorName;
+import org.apache.reef.io.network.group.impl.config.parameters.SerializedOperConfigs;
+import org.apache.reef.io.network.group.impl.operators.Sender;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.wake.EStage;
+import org.apache.reef.wake.impl.ThreadPoolStage;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+public class CommunicationGroupClientImpl implements CommunicationGroupServiceClient {
+  private static final Logger LOG = Logger.getLogger(CommunicationGroupClientImpl.class.getName());
+
+  private final GroupCommNetworkHandler groupCommNetworkHandler;
+  private final Class<? extends Name<String>> groupName;
+  private final Map<Class<? extends Name<String>>, GroupCommOperator> operators;
+  private final Sender sender;
+
+  private final String taskId;
+
+  private final String driverId;
+
+  private final CommGroupNetworkHandler commGroupNetworkHandler;
+
+  private final AtomicBoolean init = new AtomicBoolean(false);
+
+  @Inject
+  public CommunicationGroupClientImpl(@Parameter(CommunicationGroupName.class) final String groupName,
+                                      @Parameter(TaskConfigurationOptions.Identifier.class) final String taskId,
+                                      @Parameter(DriverIdentifier.class) final String driverId,
+                                      final GroupCommNetworkHandler groupCommNetworkHandler,
+                                      @Parameter(SerializedOperConfigs.class) final Set<String> operatorConfigs,
+                                      final ConfigurationSerializer configSerializer,
+                                      final NetworkService<GroupCommunicationMessage> netService) {
+    this.taskId = taskId;
+    this.driverId = driverId;
+    LOG.finest(groupName + " has GroupCommHandler-" + groupCommNetworkHandler.toString());
+    this.groupName = Utils.getClass(groupName);
+    this.groupCommNetworkHandler = groupCommNetworkHandler;
+    this.sender = new Sender(netService);
+    this.operators = new TreeMap<>(new Comparator<Class<? extends Name<String>>>() {
+
+      @Override
+      public int compare(final Class<? extends Name<String>> o1, final Class<? extends Name<String>> o2) {
+        final String s1 = o1.getSimpleName();
+        final String s2 = o2.getSimpleName();
+        return s1.compareTo(s2);
+      }
+    });
+    try {
+      this.commGroupNetworkHandler = Tang.Factory.getTang().newInjector().getInstance(CommGroupNetworkHandler.class);
+      this.groupCommNetworkHandler.register(this.groupName, commGroupNetworkHandler);
+
+      for (final String operatorConfigStr : operatorConfigs) {
+
+        final Configuration operatorConfig = configSerializer.fromString(operatorConfigStr);
+        final Injector injector = Tang.Factory.getTang().newInjector(operatorConfig);
+
+        injector.bindVolatileParameter(TaskConfigurationOptions.Identifier.class, taskId);
+        injector.bindVolatileParameter(CommunicationGroupName.class, groupName);
+        injector.bindVolatileInstance(CommGroupNetworkHandler.class, commGroupNetworkHandler);
+        injector.bindVolatileInstance(NetworkService.class, netService);
+        injector.bindVolatileInstance(CommunicationGroupServiceClient.class, this);
+
+        final GroupCommOperator operator = injector.getInstance(GroupCommOperator.class);
+        final String operName = injector.getNamedInstance(OperatorName.class);
+        this.operators.put(Utils.getClass(operName), operator);
+        LOG.finest(operName + " has CommGroupHandler-" + commGroupNetworkHandler.toString());
+      }
+    } catch (final InjectionException | IOException e) {
+      throw new RuntimeException("Unable to deserialize operator config", e);
+    }
+  }
+
+  @Override
+  public Broadcast.Sender getBroadcastSender(final Class<? extends Name<String>> operatorName) {
+    LOG.entering("CommunicationGroupClientImpl", "getBroadcastSender", new Object[]{getQualifiedName(),
+        Utils.simpleName(operatorName)});
+    final GroupCommOperator op = operators.get(operatorName);
+    if (!(op instanceof Broadcast.Sender)) {
+      throw new RuntimeException("Configured operator is not a broadcast sender");
+    }
+    commGroupNetworkHandler.addTopologyElement(operatorName);
+    LOG.exiting("CommunicationGroupClientImpl", "getBroadcastSender", getQualifiedName() + op);
+    return (Broadcast.Sender) op;
+  }
+
+  @Override
+  public Reduce.Receiver getReduceReceiver(final Class<? extends Name<String>> operatorName) {
+    LOG.entering("CommunicationGroupClientImpl", "getReduceReceiver", new Object[]{getQualifiedName(),
+        Utils.simpleName(operatorName)});
+    final GroupCommOperator op = operators.get(operatorName);
+    if (!(op instanceof Reduce.Receiver)) {
+      throw new RuntimeException("Configured operator is not a reduce receiver");
+    }
+    commGroupNetworkHandler.addTopologyElement(operatorName);
+    LOG.exiting("CommunicationGroupClientImpl", "getReduceReceiver", getQualifiedName() + op);
+    return (Reduce.Receiver) op;
+  }
+
+  @Override
+  public Broadcast.Receiver getBroadcastReceiver(final Class<? extends Name<String>> operatorName) {
+    LOG.entering("CommunicationGroupClientImpl", "getBroadcastReceiver", new Object[]{getQualifiedName(),
+        Utils.simpleName(operatorName)});
+    final GroupCommOperator op = operators.get(operatorName);
+    if (!(op instanceof Broadcast.Receiver)) {
+      throw new RuntimeException("Configured operator is not a broadcast receiver");
+    }
+    commGroupNetworkHandler.addTopologyElement(operatorName);
+    LOG.exiting("CommunicationGroupClientImpl", "getBroadcastReceiver", getQualifiedName() + op);
+    return (Broadcast.Receiver) op;
+  }
+
+  @Override
+  public Reduce.Sender getReduceSender(final Class<? extends Name<String>> operatorName) {
+    LOG.entering("CommunicationGroupClientImpl", "getReduceSender", new Object[]{getQualifiedName(),
+        Utils.simpleName(operatorName)});
+    final GroupCommOperator op = operators.get(operatorName);
+    if (!(op instanceof Reduce.Sender)) {
+      throw new RuntimeException("Configured operator is not a reduce sender");
+    }
+    commGroupNetworkHandler.addTopologyElement(operatorName);
+    LOG.exiting("CommunicationGroupClientImpl", "getReduceSender", getQualifiedName() + op);
+    return (Reduce.Sender) op;
+  }
+
+  @Override
+  public void initialize() {
+    LOG.entering("CommunicationGroupClientImpl", "initialize", getQualifiedName());
+    if (init.compareAndSet(false, true)) {
+      LOG.finest("CommGroup-" + groupName + " is initializing");
+      final CountDownLatch initLatch = new CountDownLatch(operators.size());
+
+      final InitHandler initHandler = new InitHandler(initLatch);
+      final EStage<GroupCommOperator> initStage = new ThreadPoolStage<>(initHandler, operators.size());
+      for (final GroupCommOperator op : operators.values()) {
+        initStage.onNext(op);
+      }
+
+      try {
+        initLatch.await();
+      } catch (final InterruptedException e) {
+        throw new RuntimeException("InterruptedException while waiting for initialization", e);
+      }
+
+      if (initHandler.getException() != null) {
+        throw new RuntimeException(getQualifiedName() + "Parent dead. Current behavior is for the child to die too.");
+      }
+    }
+    LOG.exiting("CommunicationGroupClientImpl", "initialize", getQualifiedName());
+  }
+
+  @Override
+  public GroupChanges getTopologyChanges() {
+    LOG.entering("CommunicationGroupClientImpl", "getTopologyChanges", getQualifiedName());
+    for (final GroupCommOperator op : operators.values()) {
+      final Class<? extends Name<String>> operName = op.getOperName();
+      LOG.finest("Sending TopologyChanges msg to driver");
+      try {
+        sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyChanges, taskId, op.getVersion(), driverId,
+            0, Utils.EmptyByteArr));
+      } catch (final NetworkException e) {
+        throw new RuntimeException("NetworkException while sending GetTopologyChanges", e);
+      }
+    }
+    final Codec<GroupChanges> changesCodec = new GroupChangesCodec();
+    final Map<Class<? extends Name<String>>, GroupChanges> perOpChanges = new HashMap<>();
+    for (final GroupCommOperator op : operators.values()) {
+      final Class<? extends Name<String>> operName = op.getOperName();
+      final byte[] changes = commGroupNetworkHandler.waitForTopologyChanges(operName);
+      perOpChanges.put(operName, changesCodec.decode(changes));
+    }
+    final GroupChanges retVal = mergeGroupChanges(perOpChanges);
+    LOG.exiting("CommunicationGroupClientImpl", "getTopologyChanges", getQualifiedName() + retVal);
+    return retVal;
+  }
+
+  /**
+   * @param perOpChanges
+   * @return
+   */
+  private GroupChanges mergeGroupChanges(final Map<Class<? extends Name<String>>, GroupChanges> perOpChanges) {
+    LOG.entering("CommunicationGroupClientImpl", "mergeGroupChanges", new Object[]{getQualifiedName(), perOpChanges});
+    boolean doChangesExist = false;
+    for (final GroupChanges change : perOpChanges.values()) {
+      if (change.exist()) {
+        doChangesExist = true;
+        break;
+      }
+    }
+    final GroupChanges changes = new GroupChangesImpl(doChangesExist);
+    LOG.exiting("CommunicationGroupClientImpl", "mergeGroupChanges", getQualifiedName() + changes);
+    return changes;
+  }
+
+  @Override
+  public void updateTopology() {
+    LOG.entering("CommunicationGroupClientImpl", "updateTopology", getQualifiedName());
+    for (final GroupCommOperator op : operators.values()) {
+      final Class<? extends Name<String>> operName = op.getOperName();
+      try {
+        sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.UpdateTopology, taskId, op.getVersion(), driverId,
+            0, Utils.EmptyByteArr));
+      } catch (final NetworkException e) {
+        throw new RuntimeException("NetworkException while sending UpdateTopology", e);
+      }
+    }
+    for (final GroupCommOperator op : operators.values()) {
+      final Class<? extends Name<String>> operName = op.getOperName();
+      GroupCommunicationMessage msg;
+      do {
+        msg = commGroupNetworkHandler.waitForTopologyUpdate(operName);
+      } while (!isMsgVersionOk(msg));
+    }
+    LOG.exiting("CommunicationGroupClientImpl", "updateTopology", getQualifiedName());
+  }
+
+  private boolean isMsgVersionOk(final GroupCommunicationMessage msg) {
+    LOG.entering("CommunicationGroupClientImpl", "isMsgVersionOk", new Object[]{getQualifiedName(), msg});
+    if (msg.hasVersion()) {
+      final int msgVersion = msg.getVersion();
+      final GroupCommOperator operator = operators.get(Utils.getClass(msg.getOperatorname()));
+      final int nodeVersion = operator.getVersion();
+      final boolean retVal;
+      if (msgVersion < nodeVersion) {
+        LOG.warning(getQualifiedName() + "Received a ver-" + msgVersion + " msg while expecting ver-" + nodeVersion
+            + ". Discarding msg");
+        retVal = false;
+      } else {
+        retVal = true;
+      }
+      LOG.exiting("CommunicationGroupClientImpl", "isMsgVersionOk", Arrays.toString(new Object[]{retVal, getQualifiedName(), msg}));
+      return retVal;
+    } else {
+      throw new RuntimeException(getQualifiedName() + "can only deal with versioned msgs");
+    }
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + " ";
+  }
+
+  @Override
+  public Class<? extends Name<String>> getName() {
+    return groupName;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommClientImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommClientImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommClientImpl.java
new file mode 100644
index 0000000..f8f6db4
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommClientImpl.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.impl.task;
+
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupClient;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupServiceClient;
+import org.apache.reef.io.network.group.api.task.GroupCommClient;
+import org.apache.reef.io.network.group.api.task.GroupCommNetworkHandler;
+import org.apache.reef.io.network.group.impl.config.parameters.SerializedGroupConfigs;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class GroupCommClientImpl implements GroupCommClient {
+  private static final Logger LOG = Logger.getLogger(GroupCommClientImpl.class.getName());
+
+  private final Map<Class<? extends Name<String>>, CommunicationGroupServiceClient> communicationGroups = new HashMap<>();
+
+  @Inject
+  public GroupCommClientImpl(
+      final @Parameter(SerializedGroupConfigs.class) Set<String> groupConfigs,
+      final @Parameter(TaskConfigurationOptions.Identifier.class) String taskId,
+      final GroupCommNetworkHandler groupCommNetworkHandler,
+      final NetworkService<ReefNetworkGroupCommProtos.GroupCommMessage> netService,
+      final ConfigurationSerializer configSerializer) {
+
+    LOG.log(Level.FINEST, "GroupCommHandler-{0}", groupCommNetworkHandler);
+
+    for (final String groupConfigStr : groupConfigs) {
+      try {
+        final Configuration groupConfig = configSerializer.fromString(groupConfigStr);
+
+        final Injector injector = Tang.Factory.getTang().newInjector(groupConfig);
+        injector.bindVolatileParameter(TaskConfigurationOptions.Identifier.class, taskId);
+        injector.bindVolatileInstance(GroupCommNetworkHandler.class, groupCommNetworkHandler);
+        injector.bindVolatileInstance(NetworkService.class, netService);
+
+        final CommunicationGroupServiceClient commGroupClient =
+            injector.getInstance(CommunicationGroupServiceClient.class);
+
+        this.communicationGroups.put(commGroupClient.getName(), commGroupClient);
+
+      } catch (final InjectionException | IOException e) {
+        throw new RuntimeException("Unable to deserialize operator config", e);
+      }
+    }
+  }
+
+  @Override
+  public CommunicationGroupClient getCommunicationGroup(
+      final Class<? extends Name<String>> groupName) {
+    return communicationGroups.get(groupName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommNetworkHandlerImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommNetworkHandlerImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommNetworkHandlerImpl.java
new file mode 100644
index 0000000..a1e9277
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/GroupCommNetworkHandlerImpl.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.reef.io.network.group.impl.task;
+
+import org.apache.reef.io.network.Message;
+import org.apache.reef.io.network.group.api.task.GroupCommNetworkHandler;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.logging.Logger;
+
+public class GroupCommNetworkHandlerImpl implements GroupCommNetworkHandler {
+
+  private static final Logger LOG = Logger.getLogger(GroupCommNetworkHandlerImpl.class.getName());
+
+  private final Map<Class<? extends Name<String>>, EventHandler<GroupCommunicationMessage>> commGroupHandlers = new ConcurrentHashMap<>();
+
+  @Inject
+  public GroupCommNetworkHandlerImpl() {
+  }
+
+  @Override
+  public void onNext(final Message<GroupCommunicationMessage> mesg) {
+    LOG.entering("GroupCommNetworkHandlerImpl", "onNext", mesg);
+    final Iterator<GroupCommunicationMessage> iter = mesg.getData().iterator();
+    final GroupCommunicationMessage msg = iter.hasNext() ? iter.next() : null;
+    try {
+      final Class<? extends Name<String>> groupName = (Class<? extends Name<String>>) Class.forName(msg.getGroupname());
+      commGroupHandlers.get(groupName).onNext(msg);
+    } catch (final ClassNotFoundException e) {
+      throw new RuntimeException("GroupName not found", e);
+    }
+    LOG.exiting("GroupCommNetworkHandlerImpl", "onNext", mesg);
+  }
+
+  @Override
+  public void register(final Class<? extends Name<String>> groupName,
+                       final EventHandler<GroupCommunicationMessage> commGroupNetworkHandler) {
+    LOG.entering("GroupCommNetworkHandlerImpl", "register", new Object[]{groupName,
+        commGroupNetworkHandler});
+    commGroupHandlers.put(groupName, commGroupNetworkHandler);
+    LOG.exiting("GroupCommNetworkHandlerImpl", "register", Arrays.toString(new Object[]{groupName,
+        commGroupNetworkHandler}));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/InitHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/InitHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/InitHandler.java
new file mode 100644
index 0000000..13bd644
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/InitHandler.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.reef.io.network.group.impl.task;
+
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.GroupCommOperator;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.logging.Logger;
+
+class InitHandler implements EventHandler<GroupCommOperator> {
+
+  private static final Logger LOG = Logger.getLogger(InitHandler.class.getName());
+
+  private ParentDeadException exception = null;
+  private final CountDownLatch initLatch;
+
+  public InitHandler(final CountDownLatch initLatch) {
+    this.initLatch = initLatch;
+  }
+
+  @Override
+  public void onNext(final GroupCommOperator op) {
+    LOG.entering("InitHandler", "onNext", op);
+    try {
+      op.initialize();
+    } catch (final ParentDeadException e) {
+      this.exception = e;
+    }
+    initLatch.countDown();
+    LOG.exiting("InitHandler", "onNext", op);
+  }
+
+  public ParentDeadException getException() {
+    return exception;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/NodeStructImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/NodeStructImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/NodeStructImpl.java
new file mode 100644
index 0000000..5976352
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/NodeStructImpl.java
@@ -0,0 +1,98 @@
+/**
+ * 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.reef.io.network.group.impl.task;
+
+import org.apache.reef.io.network.group.api.task.NodeStruct;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Logger;
+
+public abstract class NodeStructImpl implements NodeStruct {
+
+  private static final Logger LOG = Logger.getLogger(NodeStructImpl.class.getName());
+
+  private final String id;
+  private final BlockingQueue<GroupCommunicationMessage> dataQue = new LinkedBlockingQueue<>();
+
+  private int version;
+
+  public NodeStructImpl(final String id, final int version) {
+    super();
+    this.id = id;
+    this.version = version;
+  }
+
+  @Override
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void setVersion(final int version) {
+    this.version = version;
+  }
+
+  @Override
+  public String getId() {
+    return id;
+  }
+
+  @Override
+  public void addData(final GroupCommunicationMessage msg) {
+    LOG.entering("NodeStructImpl", "addData", msg);
+    dataQue.add(msg);
+    LOG.exiting("NodeStructImpl", "addData", msg);
+  }
+
+  @Override
+  public byte[] getData() {
+    LOG.entering("NodeStructImpl", "getData");
+    GroupCommunicationMessage gcm;
+    try {
+      gcm = dataQue.take();
+    } catch (final InterruptedException e) {
+      throw new RuntimeException("InterruptedException while waiting for data from " + id, e);
+    }
+
+    final byte[] retVal = checkDead(gcm) ? null : Utils.getData(gcm);
+    LOG.exiting("NodeStructImpl", "getData", retVal);
+    return retVal;
+  }
+
+  @Override
+  public String toString() {
+    return "(" + id + "," + version + ")";
+  }
+
+  @Override
+  public boolean equals(final Object obj) {
+    if (obj instanceof NodeStructImpl) {
+      final NodeStructImpl that = (NodeStructImpl) obj;
+      return this.id.equals(that.id) && this.version == that.version;
+    } else {
+      return false;
+    }
+  }
+
+  public abstract boolean checkDead(final GroupCommunicationMessage gcm);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
new file mode 100644
index 0000000..f46031b
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/task/OperatorTopologyImpl.java
@@ -0,0 +1,466 @@
+/**
+ * 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.reef.io.network.group.impl.task;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.task.OperatorTopology;
+import org.apache.reef.io.network.group.api.task.OperatorTopologyStruct;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.operators.Sender;
+import org.apache.reef.io.network.group.impl.utils.ResettingCountDownLatch;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EStage;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.impl.SingleThreadStage;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+public class OperatorTopologyImpl implements OperatorTopology {
+
+  private static final Logger LOG = Logger.getLogger(OperatorTopologyImpl.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String selfId;
+  private final String driverId;
+  private final Sender sender;
+  private final Object topologyLock = new Object();
+
+  private final int version;
+
+  private final BlockingQueue<GroupCommunicationMessage> deltas = new LinkedBlockingQueue<>();
+  private final BlockingQueue<GroupCommunicationMessage> deletionDeltas = new LinkedBlockingQueue<>();
+
+  private OperatorTopologyStruct baseTopology;
+  private OperatorTopologyStruct effectiveTopology;
+  private final ResettingCountDownLatch topologyLockAquired = new ResettingCountDownLatch(1);
+  private final AtomicBoolean updatingTopo = new AtomicBoolean(false);
+
+  private final EventHandler<GroupCommunicationMessage> baseTopologyUpdateHandler = new BaseTopologyUpdateHandler();
+
+  private final EStage<GroupCommunicationMessage> baseTopologyUpdateStage = new SingleThreadStage<>(
+      "BaseTopologyUpdateStage",
+      baseTopologyUpdateHandler,
+      5);
+
+  private final EventHandler<GroupCommunicationMessage> dataHandlingStageHandler = new DataHandlingStageHandler();
+
+  // The queue capacity might determine how many tasks can be handled
+  private final EStage<GroupCommunicationMessage> dataHandlingStage = new SingleThreadStage<>("DataHandlingStage",
+      dataHandlingStageHandler,
+      10000);
+
+  @Inject
+  public OperatorTopologyImpl(final Class<? extends Name<String>> groupName,
+                              final Class<? extends Name<String>> operName, final String selfId,
+                              final String driverId, final Sender sender, final int version) {
+    super();
+    this.groupName = groupName;
+    this.operName = operName;
+    this.selfId = selfId;
+    this.driverId = driverId;
+    this.sender = sender;
+    this.version = version;
+  }
+
+  /**
+   * Handle messages meant for this operator. Data msgs are passed on
+   * to the DataHandlingStage while Ctrl msgs are queued up for the
+   * base topology to update later. Ctrl msgs signalling death of a
+   * task are also routed to the effectiveTopology in order to notify
+   * a waiting operation. During initialization when effective topology
+   * is not yet set-up, these *Dead msgs are queued in deletionDeltas
+   * for the small time window when these arrive after baseTopology has
+   * received TopologySetup but not yet created the effectiveTopology.
+   * Most times the msgs in the deletionDeltas will be discarded as stale
+   * msgs
+   * <p/>
+   * No synchronization is needed while handling *Dead messages.
+   * There 2 states: UpdatingTopo & NotUpdatingTopo
+   * If UpdatingTopo, deltas.put still takes care of adding this msg to effTop through baseTopo changes.
+   * If not, we add to effTopo. So we are good.
+   * <p/>
+   * However, for data msgs synchronization is needed. Look at doc of
+   * DataHandlingStage
+   * <p/>
+   * Adding to deletionDeltas should be outside
+   * effTopo!=null block. There is a rare possibility that during initialization
+   * just after baseTopo is created(so deltas will be ignored) & just before
+   * effTopo is created(so effTopo will be null) where we can miss a deletion
+   * msg if not added to deletionDelta because this method is synchronized
+   */
+  @Override
+  public void handle(final GroupCommunicationMessage msg) {
+    LOG.entering("OperatorTopologyImpl", "handle", new Object[]{getQualifiedName(), msg});
+    if (isMsgVersionOk(msg)) {
+      try {
+        switch (msg.getType()) {
+          case UpdateTopology:
+            updatingTopo.set(true);
+            baseTopologyUpdateStage.onNext(msg);
+            topologyLockAquired.awaitAndReset(1);
+            LOG.finest(getQualifiedName() + "topoLockAquired CDL released. Resetting it to new CDL");
+            sendAckToDriver(msg);
+            break;
+
+          case TopologySetup:
+            LOG.finest(getQualifiedName() + "Adding to deltas queue");
+            deltas.put(msg);
+            break;
+
+          case ParentAdd:
+          case ChildAdd:
+            LOG.finest(getQualifiedName() + "Adding to deltas queue");
+            deltas.put(msg);
+            break;
+
+          case ParentDead:
+          case ChildDead:
+            LOG.finest(getQualifiedName() + "Adding to deltas queue");
+            deltas.put(msg);
+
+            LOG.finest(getQualifiedName() + "Adding to deletionDeltas queue");
+            deletionDeltas.put(msg);
+
+            if (effectiveTopology != null) {
+              LOG.finest(getQualifiedName() + "Adding as data msg to non-null effective topology struct");
+              effectiveTopology.addAsData(msg);
+            } else {
+              LOG.fine(getQualifiedName() + "Received a death message before effective topology was setup. CAUTION");
+            }
+            break;
+
+          default:
+            dataHandlingStage.onNext(msg);
+        }
+      } catch (final InterruptedException e) {
+        throw new RuntimeException("InterruptedException while trying to put ctrl msg into delta queue", e);
+      }
+    }
+    LOG.exiting("OperatorTopologyImpl", "handle", Arrays.toString(new Object[]{getQualifiedName(), msg}));
+  }
+
+  private boolean isMsgVersionOk(final GroupCommunicationMessage msg) {
+    LOG.entering("OperatorTopologyImpl", "isMsgVersionOk", new Object[]{getQualifiedName(), msg});
+    if (msg.hasVersion()) {
+      final int msgVersion = msg.getVersion();
+      final boolean retVal;
+      if (msgVersion < version) {
+        LOG.warning(getQualifiedName() + "Received a ver-" + msgVersion + " msg while expecting ver-" + version
+            + ". Discarding msg");
+        retVal = false;
+      } else {
+        retVal = true;
+      }
+      LOG.exiting("OperatorTopologyImpl", "isMsgVersionOk", Arrays.toString(new Object[]{retVal, getQualifiedName(), msg}));
+      return retVal;
+    } else {
+      throw new RuntimeException(getQualifiedName() + "can only deal with versioned msgs");
+    }
+  }
+
+  @Override
+  public void initialize() throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "initialize", getQualifiedName());
+    createBaseTopology();
+    LOG.exiting("OperatorTopologyImpl", "initialize", getQualifiedName());
+  }
+
+  @Override
+  public void sendToParent(final byte[] data, final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "sendToParent", new Object[]{getQualifiedName(), data, msgType});
+    refreshEffectiveTopology();
+    assert (effectiveTopology != null);
+    effectiveTopology.sendToParent(data, msgType);
+    LOG.exiting("OperatorTopologyImpl", "sendToParent", Arrays.toString(new Object[]{getQualifiedName(), data, msgType}));
+  }
+
+  @Override
+  public void sendToChildren(final byte[] data, final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "sendToChildren", new Object[]{getQualifiedName(), data, msgType});
+    refreshEffectiveTopology();
+    assert (effectiveTopology != null);
+    effectiveTopology.sendToChildren(data, msgType);
+    LOG.exiting("OperatorTopologyImpl", "sendToChildren", Arrays.toString(new Object[]{getQualifiedName(), data, msgType}));
+  }
+
+  @Override
+  public byte[] recvFromParent() throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "recvFromParent", getQualifiedName());
+    refreshEffectiveTopology();
+    assert (effectiveTopology != null);
+    final byte[] retVal = effectiveTopology.recvFromParent();
+    LOG.exiting("OperatorTopologyImpl", "recvFromParent", Arrays.toString(new Object[]{getQualifiedName(), retVal}));
+    return retVal;
+  }
+
+  @Override
+  public <T> T recvFromChildren(final Reduce.ReduceFunction<T> redFunc, final Codec<T> dataCodec) throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "recvFromChildren", getQualifiedName());
+    refreshEffectiveTopology();
+    assert (effectiveTopology != null);
+    final T retVal = effectiveTopology.recvFromChildren(redFunc, dataCodec);
+    LOG.exiting("OperatorTopologyImpl", "recvFromChildren", Arrays.toString(new Object[]{getQualifiedName(), retVal}));
+    return retVal;
+  }
+
+  /**
+   * Only refreshes the effective topology with deletion msgs from
+   * deletionDeltas queue
+   *
+   * @throws ParentDeadException
+   */
+  private void refreshEffectiveTopology() throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "refreshEffectiveTopology", getQualifiedName());
+    LOG.finest(getQualifiedName() + "Waiting to acquire topoLock");
+    synchronized (topologyLock) {
+      LOG.finest(getQualifiedName() + "Acquired topoLock");
+
+      assert (effectiveTopology != null);
+
+      final Set<GroupCommunicationMessage> deletionDeltas = new HashSet<>();
+      copyDeletionDeltas(deletionDeltas);
+
+      LOG.finest(getQualifiedName() + "Updating effective topology struct with deletion msgs");
+      effectiveTopology.update(deletionDeltas);
+      LOG.finest(getQualifiedName() + "Released topoLock");
+    }
+    LOG.exiting("OperatorTopologyImpl", "refreshEffectiveTopology", getQualifiedName());
+  }
+
+  /**
+   * @throws ParentDeadException
+   */
+  private void createBaseTopology() throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "createBaseTopology", getQualifiedName());
+    baseTopology = new OperatorTopologyStructImpl(groupName, operName, selfId, driverId, sender, version);
+    updateBaseTopology();
+    LOG.exiting("OperatorTopologyImpl", "createBaseTopology", getQualifiedName());
+  }
+
+  /**
+   * Blocking method that waits till the base topology is updated Unblocks when
+   * we receive a TopologySetup msg from driver
+   * <p/>
+   * Will also update the effective topology when the base topology is updated
+   * so that creation of effective topology is limited to just this method and
+   * refresh will only refresh the effective topology with deletion msgs from
+   * deletionDeltas queue
+   *
+   * @throws ParentDeadException
+   */
+  private void updateBaseTopology() throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "updateBaseTopology", getQualifiedName());
+    LOG.finest(getQualifiedName() + "Waiting to acquire topoLock");
+    synchronized (topologyLock) {
+      LOG.finest(getQualifiedName() + "Acquired topoLock");
+      try {
+        assert (baseTopology != null);
+        LOG.finest(getQualifiedName() + "Updating base topology. So setting dirty bit");
+        baseTopology.setChanges(true);
+
+        LOG.finest(getQualifiedName() + "Waiting for ctrl msgs");
+        for (GroupCommunicationMessage msg = deltas.take(); msg.getType() != ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologySetup; msg = deltas.take()) {
+          LOG.finest(getQualifiedName() + "Got " + msg.getType() + " msg from " + msg.getSrcid());
+          if (effectiveTopology == null && msg.getType() == ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentDead) {
+            /**
+             * If effectiveTopology!=null, this method is being called from the BaseTopologyUpdateStage
+             * And exception thrown will be caught by uncaughtExceptionHandler leading to System.exit
+             */
+            LOG.finer(getQualifiedName() + "Throwing ParentDeadException");
+            throw new ParentDeadException(getQualifiedName()
+                + "Parent dead. Current behavior is for the child to die too.");
+          } else {
+            LOG.finest(getQualifiedName() + "Updating basetopology struct");
+            baseTopology.update(msg);
+            sendAckToDriver(msg);
+          }
+          LOG.finest(getQualifiedName() + "Waiting for ctrl msgs");
+        }
+
+        updateEffTopologyFromBaseTopology();
+
+      } catch (final InterruptedException e) {
+        throw new RuntimeException("InterruptedException while waiting for delta msg from driver", e);
+      }
+      LOG.finest(getQualifiedName() + "Released topoLock");
+    }
+    LOG.exiting("OperatorTopologyImpl", "updateBaseTopology", getQualifiedName());
+  }
+
+  private void sendAckToDriver(final GroupCommunicationMessage msg) {
+    LOG.entering("OperatorTopologyImpl", "sendAckToDriver", new Object[]{getQualifiedName(), msg});
+    try {
+      final String srcId = msg.getSrcid();
+      if (msg.hasVersion()) {
+        final int srcVersion = msg.getSrcVersion();
+        switch (msg.getType()) {
+          case UpdateTopology:
+            sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologySetup, selfId, this.version, driverId,
+                srcVersion, Utils.EmptyByteArr));
+            break;
+          case ParentAdd:
+            sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentAdded, selfId, this.version, srcId,
+                srcVersion, Utils.EmptyByteArr), driverId);
+            break;
+          case ParentDead:
+            sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentRemoved, selfId, this.version, srcId,
+                srcVersion, Utils.EmptyByteArr), driverId);
+            break;
+          case ChildAdd:
+            sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdded, selfId, this.version, srcId,
+                srcVersion, Utils.EmptyByteArr), driverId);
+            break;
+          case ChildDead:
+            sender.send(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildRemoved, selfId, this.version, srcId,
+                srcVersion, Utils.EmptyByteArr), driverId);
+            break;
+          default:
+            throw new RuntimeException("Received a non control message for acknowledgement");
+        }
+      } else {
+        throw new RuntimeException(getQualifiedName() + "Ack Sender can only deal with versioned msgs");
+      }
+    } catch (final NetworkException e) {
+      throw new RuntimeException("NetworkException while sending ack to driver for delta msg " + msg.getType(), e);
+    }
+    LOG.exiting("OperatorTopologyImpl", "sendAckToDriver", Arrays.toString(new Object[]{getQualifiedName(), msg}));
+  }
+
+  private void updateEffTopologyFromBaseTopology() {
+    LOG.entering("OperatorTopologyImpl", "updateEffTopologyFromBaseTopology", getQualifiedName());
+    assert (baseTopology != null);
+    LOG.finest(getQualifiedName() + "Updaing effective topology");
+    if (baseTopology.hasChanges()) {
+      //Create effectiveTopology from baseTopology
+      effectiveTopology = new OperatorTopologyStructImpl(baseTopology);
+      baseTopology.setChanges(false);
+    }
+    LOG.exiting("OperatorTopologyImpl", "updateEffTopologyFromBaseTopology", getQualifiedName());
+  }
+
+  /**
+   * @param deletionDeltasForUpdate
+   * @throws ParentDeadException
+   */
+  private void copyDeletionDeltas(final Set<GroupCommunicationMessage> deletionDeltasForUpdate)
+      throws ParentDeadException {
+    LOG.entering("OperatorTopologyImpl", "copyDeletionDeltas", new Object[]{getQualifiedName(),
+        deletionDeltasForUpdate});
+    this.deletionDeltas.drainTo(deletionDeltasForUpdate);
+    for (final GroupCommunicationMessage msg : deletionDeltasForUpdate) {
+      final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType = msg.getType();
+      if (msgType == ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentDead) {
+        throw new ParentDeadException(getQualifiedName() + "Parent dead. Current behavior is for the child to die too.");
+      }
+    }
+    LOG.exiting("OperatorTopologyImpl", "copyDeletionDeltas", Arrays.toString(new Object[]{getQualifiedName(),
+        deletionDeltasForUpdate}));
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":" + selfId + ":ver(" + version + ") - ";
+  }
+
+  /**
+   * Unlike Dead msgs this needs to be synchronized because data msgs are not
+   * routed through the base topo changes So we need to make sure to wait for
+   * updateTopo to complete and for the new effective topo to take effect. Hence
+   * updatinTopo is set to false in refreshEffTopo. Also, since this is called
+   * from a netty IO thread, we need to create a stage to move the msgs from
+   * netty space to application space and release the netty threads. Otherwise
+   * weird deadlocks can happen Ex: Sent model to k nodes using broadcast. Send
+   * to K+1 th is waiting for ACK. The K nodes already compute their states and
+   * reduce send their results. If we haven't finished refreshEffTopo because of
+   * which updatingTopo is true, we can't add the new msgs if the #netty threads
+   * is k All k threads are waiting to add data. Single user thread that is
+   * waiting for ACK does not come around to refreshEffTopo and we are
+   * deadlocked because there aren't enough netty threads to dispatch msgs to
+   * the application. Hence the stage
+   */
+  private final class DataHandlingStageHandler implements EventHandler<GroupCommunicationMessage> {
+    @Override
+    public void onNext(final GroupCommunicationMessage dataMsg) {
+      LOG.entering("OperatorTopologyImpl.DataHandlingStageHandler", "onNext", new Object[]{getQualifiedName(),
+          dataMsg});
+      LOG.finest(getQualifiedName() + "Waiting to acquire topoLock");
+      synchronized (topologyLock) {
+        LOG.finest(getQualifiedName() + "Aqcuired topoLock");
+        while (updatingTopo.get()) {
+          try {
+            LOG.finest(getQualifiedName() + "Topology is being updated. Released topoLock, Waiting on topoLock");
+            topologyLock.wait();
+            LOG.finest(getQualifiedName() + "Aqcuired topoLock");
+          } catch (final InterruptedException e) {
+            throw new RuntimeException("InterruptedException while data handling"
+                + "stage was waiting for updatingTopo to become false", e);
+          }
+        }
+        if (effectiveTopology != null) {
+          LOG.finest(getQualifiedName() + "Non-null effectiveTopo.addAsData(msg)");
+          effectiveTopology.addAsData(dataMsg);
+        } else {
+          LOG.fine("Received a data message before effective topology was setup");
+        }
+        LOG.finest(getQualifiedName() + "Released topoLock");
+      }
+      LOG.exiting("OperatorTopologyImpl.DataHandlingStageHandler", "onNext",
+          Arrays.toString(new Object[]{getQualifiedName(), dataMsg}));
+    }
+  }
+
+  private final class BaseTopologyUpdateHandler implements EventHandler<GroupCommunicationMessage> {
+    @Override
+    public void onNext(final GroupCommunicationMessage msg) {
+      assert (msg.getType() == ReefNetworkGroupCommProtos.GroupCommMessage.Type.UpdateTopology);
+      assert (effectiveTopology != null);
+      LOG.entering("OperatorTopologyImpl.BaseTopologyUpdateHandler", "onNext", new Object[]{getQualifiedName(), msg});
+      LOG.finest(getQualifiedName() + "Waiting to acquire topoLock");
+      synchronized (topologyLock) {
+        LOG.finest(getQualifiedName() + "Acquired topoLock");
+        LOG.finest(getQualifiedName() + "Releasing topoLoackAcquired CDL");
+        topologyLockAquired.countDown();
+        try {
+          updateBaseTopology();
+          LOG.finest(getQualifiedName() + "Completed updating base & effective topologies");
+        } catch (final ParentDeadException e) {
+          throw new RuntimeException(getQualifiedName() + "BaseTopologyUpdateStage: Unexpected ParentDeadException", e);
+        }
+        updatingTopo.set(false);
+        LOG.finest(getQualifiedName() + "Topology update complete. Notifying waiting threads");
+        topologyLock.notifyAll();
+        LOG.finest(getQualifiedName() + "Released topoLock");
+      }
+      LOG.exiting("OperatorTopologyImpl.BaseTopologyUpdateHandler", "onNext",
+          Arrays.toString(new Object[]{getQualifiedName(), msg}));
+    }
+  }
+}


[7/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/WeightedLogisticLossFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/WeightedLogisticLossFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/WeightedLogisticLossFunction.java
new file mode 100644
index 0000000..2ad11fc
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/WeightedLogisticLossFunction.java
@@ -0,0 +1,74 @@
+/**
+ * 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.reef.examples.group.bgd.loss;
+
+import javax.inject.Inject;
+
+public final class WeightedLogisticLossFunction implements LossFunction {
+
+  private static final double POS = 0.0025;
+  private static final double NEG = 0.9975;
+
+  private final double posWeight;
+  private final double negWeight;
+
+  /**
+   * Trivial constructor.
+   */
+  @Inject
+  public WeightedLogisticLossFunction() {
+    this.posWeight = (this.POS + this.NEG) / (2 * this.POS);
+    this.negWeight = (this.POS + this.NEG) / (2 * this.NEG);
+  }
+
+  @Override
+  public double computeLoss(double y, double f) {
+
+    final double predictedTimesLabel = y * f;
+    final double weight = y == -1 ? this.negWeight : this.posWeight;
+
+    if (predictedTimesLabel >= 0) {
+      return weight * Math.log(1 + Math.exp(-predictedTimesLabel));
+    } else {
+      return weight * (-predictedTimesLabel + Math.log(1 + Math.exp(predictedTimesLabel)));
+    }
+  }
+
+  @Override
+  public double computeGradient(double y, double f) {
+
+    final double predictedTimesLabel = y * f;
+    final double weight = y == -1 ? this.negWeight : this.posWeight;
+
+    final double probability;
+    if (predictedTimesLabel >= 0) {
+      probability = 1 / (1 + Math.exp(-predictedTimesLabel));
+    } else {
+      final double ExpVal = Math.exp(predictedTimesLabel);
+      probability = ExpVal / (1 + ExpVal);
+    }
+
+    return (probability - 1) * y * weight;
+  }
+
+  @Override
+  public String toString() {
+    return "WeightedLogisticLossFunction{}";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ControlMessageBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ControlMessageBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ControlMessageBroadcaster.java
new file mode 100644
index 0000000..d7013fb
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ControlMessageBroadcaster.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Used to identify the broadcast operator for control flow messages.
+ */
+@NamedParameter()
+public final class ControlMessageBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/DescentDirectionBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/DescentDirectionBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/DescentDirectionBroadcaster.java
new file mode 100644
index 0000000..1d8a148
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/DescentDirectionBroadcaster.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Name of the broadcast operator used to send descent directions during linesearch.
+ */
+@NamedParameter()
+public final class DescentDirectionBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LineSearchEvaluationsReducer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LineSearchEvaluationsReducer.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LineSearchEvaluationsReducer.java
new file mode 100644
index 0000000..411fd17
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LineSearchEvaluationsReducer.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Name of the reducer used to aggregate line search results.
+ */
+@NamedParameter()
+public final class LineSearchEvaluationsReducer implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LossAndGradientReducer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LossAndGradientReducer.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LossAndGradientReducer.java
new file mode 100644
index 0000000..91261ce
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/LossAndGradientReducer.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Name used for the Reduce operator for loss and gradient aggregation.
+ */
+@NamedParameter()
+public final class LossAndGradientReducer implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/MinEtaBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/MinEtaBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/MinEtaBroadcaster.java
new file mode 100644
index 0000000..eb230d2
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/MinEtaBroadcaster.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter()
+public final class MinEtaBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelAndDescentDirectionBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelAndDescentDirectionBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelAndDescentDirectionBroadcaster.java
new file mode 100644
index 0000000..755cc00
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelAndDescentDirectionBroadcaster.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Name of the broadcast operator used to send a model and descent direction during line search.
+ */
+@NamedParameter()
+public final class ModelAndDescentDirectionBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelBroadcaster.java
new file mode 100644
index 0000000..8a1aae0
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/ModelBroadcaster.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.reef.examples.group.bgd.operatornames;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The name of the broadcast operator used for model broadcasts.
+ */
+@NamedParameter()
+public final class ModelBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/package-info.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/package-info.java
new file mode 100644
index 0000000..5c364cc
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/operatornames/package-info.java
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+/**
+ * Parameter names used to identify the various operators used in BGD.
+ */
+package org.apache.reef.examples.group.bgd.operatornames;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/AllCommunicationGroup.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/AllCommunicationGroup.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/AllCommunicationGroup.java
new file mode 100644
index 0000000..dd173a3
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/AllCommunicationGroup.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter()
+public final class AllCommunicationGroup implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDControlParameters.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDControlParameters.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDControlParameters.java
new file mode 100644
index 0000000..ab60d16
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDControlParameters.java
@@ -0,0 +1,126 @@
+/**
+ * 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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.examples.group.bgd.loss.LossFunction;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.formats.CommandLine;
+
+import javax.inject.Inject;
+
+public final class BGDControlParameters {
+
+  private final int dimensions;
+  private final double lambda;
+  private final double eps;
+  private final int iters;
+  private final int minParts;
+  private final boolean rampup;
+
+  private final double eta;
+  private final double probOfSuccessfulIteration;
+  private final BGDLossType lossType;
+
+  @Inject
+  public BGDControlParameters(
+      final @Parameter(ModelDimensions.class) int dimensions,
+      final @Parameter(Lambda.class) double lambda,
+      final @Parameter(Eps.class) double eps,
+      final @Parameter(Eta.class) double eta,
+      final @Parameter(ProbabilityOfSuccesfulIteration.class) double probOfSuccessfulIteration,
+      final @Parameter(Iterations.class) int iters,
+      final @Parameter(EnableRampup.class) boolean rampup,
+      final @Parameter(MinParts.class) int minParts,
+      final BGDLossType lossType) {
+    this.dimensions = dimensions;
+    this.lambda = lambda;
+    this.eps = eps;
+    this.eta = eta;
+    this.probOfSuccessfulIteration = probOfSuccessfulIteration;
+    this.iters = iters;
+    this.rampup = rampup;
+    this.minParts = minParts;
+    this.lossType = lossType;
+  }
+
+  public Configuration getConfiguration() {
+    return Tang.Factory.getTang().newConfigurationBuilder()
+        .bindNamedParameter(ModelDimensions.class, Integer.toString(this.dimensions))
+        .bindNamedParameter(Lambda.class, Double.toString(this.lambda))
+        .bindNamedParameter(Eps.class, Double.toString(this.eps))
+        .bindNamedParameter(Eta.class, Double.toString(this.eta))
+        .bindNamedParameter(ProbabilityOfSuccesfulIteration.class, Double.toString(probOfSuccessfulIteration))
+        .bindNamedParameter(Iterations.class, Integer.toString(this.iters))
+        .bindNamedParameter(EnableRampup.class, Boolean.toString(this.rampup))
+        .bindNamedParameter(MinParts.class, Integer.toString(this.minParts))
+        .bindNamedParameter(LossFunctionType.class, lossType.lossFunctionString())
+        .build();
+  }
+
+  public static CommandLine registerShortNames(final CommandLine commandLine) {
+    return commandLine
+        .registerShortNameOfClass(ModelDimensions.class)
+        .registerShortNameOfClass(Lambda.class)
+        .registerShortNameOfClass(Eps.class)
+        .registerShortNameOfClass(Eta.class)
+        .registerShortNameOfClass(ProbabilityOfSuccesfulIteration.class)
+        .registerShortNameOfClass(Iterations.class)
+        .registerShortNameOfClass(EnableRampup.class)
+        .registerShortNameOfClass(MinParts.class)
+        .registerShortNameOfClass(LossFunctionType.class);
+  }
+
+  public int getDimensions() {
+    return this.dimensions;
+  }
+
+  public double getLambda() {
+    return this.lambda;
+  }
+
+  public double getEps() {
+    return this.eps;
+  }
+
+  public double getEta() {
+    return this.eta;
+  }
+
+  public double getProbOfSuccessfulIteration() {
+    return probOfSuccessfulIteration;
+  }
+
+  public int getIters() {
+    return this.iters;
+  }
+
+  public int getMinParts() {
+    return this.minParts;
+  }
+
+  public boolean isRampup() {
+    return this.rampup;
+  }
+
+  public Class<? extends LossFunction> getLossFunction() {
+    return this.lossType.getLossFunction();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDLossType.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDLossType.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDLossType.java
new file mode 100644
index 0000000..e20dcaf
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/BGDLossType.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.examples.group.bgd.loss.LogisticLossFunction;
+import org.apache.reef.examples.group.bgd.loss.LossFunction;
+import org.apache.reef.examples.group.bgd.loss.SquaredErrorLossFunction;
+import org.apache.reef.examples.group.bgd.loss.WeightedLogisticLossFunction;
+import org.apache.reef.tang.annotations.Parameter;
+
+import javax.inject.Inject;
+import java.util.HashMap;
+import java.util.Map;
+
+public class BGDLossType {
+
+  private static final Map<String, Class<? extends LossFunction>> LOSS_FUNCTIONS =
+      new HashMap<String, Class<? extends LossFunction>>() {{
+        put("logLoss", LogisticLossFunction.class);
+        put("weightedLogLoss", WeightedLogisticLossFunction.class);
+        put("squaredError", SquaredErrorLossFunction.class);
+      }};
+
+  private final Class<? extends LossFunction> lossFunction;
+
+  private final String lossFunctionStr;
+
+  @Inject
+  public BGDLossType(@Parameter(LossFunctionType.class) final String lossFunctionStr) {
+    this.lossFunctionStr = lossFunctionStr;
+    this.lossFunction = LOSS_FUNCTIONS.get(lossFunctionStr);
+    if (this.lossFunction == null) {
+      throw new RuntimeException("Specified loss function type: " + lossFunctionStr +
+          " is not implemented. Supported types are logLoss|weightedLogLoss|squaredError");
+    }
+  }
+
+  public Class<? extends LossFunction> getLossFunction() {
+    return this.lossFunction;
+  }
+
+  public String lossFunctionString() {
+    return lossFunctionStr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EnableRampup.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EnableRampup.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EnableRampup.java
new file mode 100644
index 0000000..cc388a9
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EnableRampup.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Maximum Number of Iterations.
+ */
+@NamedParameter(doc = "Should we ram-up?", short_name = "rampup", default_value = "false")
+public final class EnableRampup implements Name<Boolean> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eps.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eps.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eps.java
new file mode 100644
index 0000000..05d7b84
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eps.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Break criterion for the optimizer. If the progress in mean loss between
+ * two iterations is less than this, the optimization stops.
+ */
+@NamedParameter(short_name = "eps", default_value = "1e-6")
+public final class Eps implements Name<Double> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eta.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eta.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eta.java
new file mode 100644
index 0000000..59ef312
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Eta.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Break criterion for the optimizer. If the progress in mean loss between
+ * two iterations is less than this, the optimization stops.
+ */
+@NamedParameter(short_name = "eta", default_value = "0.01")
+public final class Eta implements Name<Double> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EvaluatorMemory.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EvaluatorMemory.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EvaluatorMemory.java
new file mode 100644
index 0000000..7b1015e
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/EvaluatorMemory.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The memory used for each Evaluator. In MB.
+ */
+@NamedParameter(short_name = "memory", default_value = "1024", doc = "The memory used for each Evaluator. In MB.")
+public final class EvaluatorMemory implements Name<Integer> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/InputDir.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/InputDir.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/InputDir.java
new file mode 100644
index 0000000..9ad5656
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/InputDir.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The input folder of the learner.
+ */
+@NamedParameter(short_name = "input")
+public final class InputDir implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Iterations.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Iterations.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Iterations.java
new file mode 100644
index 0000000..79530be
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Iterations.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Maximum Number of Iterations.
+ */
+@NamedParameter(doc = "Number of iterations", short_name = "iterations", default_value = "100")
+public final class Iterations implements Name<Integer> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Lambda.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Lambda.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Lambda.java
new file mode 100644
index 0000000..c278ca9
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Lambda.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The regularization constant
+ */
+@NamedParameter(doc = "The regularization constant", short_name = "lambda", default_value = "1e-4")
+public final class Lambda implements Name<Double> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/LossFunctionType.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/LossFunctionType.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/LossFunctionType.java
new file mode 100644
index 0000000..e740c95
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/LossFunctionType.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+@NamedParameter(doc = "Loss Function to be used: logLoss|weightedLogLoss|squaredError", short_name = "loss", default_value = "logLoss")
+public class LossFunctionType implements Name<String> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/MinParts.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/MinParts.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/MinParts.java
new file mode 100644
index 0000000..6488c56
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/MinParts.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Maximum Number of Iterations.
+ */
+@NamedParameter(doc = "Min Number of partitions", short_name = "minparts", default_value = "2")
+public final class MinParts implements Name<Integer> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ModelDimensions.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ModelDimensions.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ModelDimensions.java
new file mode 100644
index 0000000..cd19085
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ModelDimensions.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The dimensionality of the model learned.
+ */
+@NamedParameter(doc = "Model dimensions", short_name = "dim")
+public class ModelDimensions implements Name<Integer> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumSplits.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumSplits.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumSplits.java
new file mode 100644
index 0000000..dbbbead
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumSplits.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+// TODO: Document
+@NamedParameter(short_name = "splits", default_value = "5")
+public final class NumSplits implements Name<Integer> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumberOfReceivers.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumberOfReceivers.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumberOfReceivers.java
new file mode 100644
index 0000000..bedfb5a
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/NumberOfReceivers.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+@NamedParameter(doc = "The number of receivers for the operators")
+public class NumberOfReceivers implements Name<Integer> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfFailure.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfFailure.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfFailure.java
new file mode 100644
index 0000000..53fc5db
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfFailure.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Break criterion for the optimizer. If the progress in mean loss between
+ * two iterations is less than this, the optimization stops.
+ */
+@NamedParameter(default_value = "0.01")
+public final class ProbabilityOfFailure implements Name<Double> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfSuccesfulIteration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfSuccesfulIteration.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfSuccesfulIteration.java
new file mode 100644
index 0000000..243b82c
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/ProbabilityOfSuccesfulIteration.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Break criterion for the optimizer. If the progress in mean loss between
+ * two iterations is less than this, the optimization stops.
+ */
+@NamedParameter(short_name = "psuccess", default_value = "0.5")
+public final class ProbabilityOfSuccesfulIteration implements Name<Double> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Timeout.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Timeout.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Timeout.java
new file mode 100644
index 0000000..8332514
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/parameters/Timeout.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.reef.examples.group.bgd.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+
+// TODO: Document
+@NamedParameter(short_name = "timeout", default_value = "2")
+public final class Timeout implements Name<Integer> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/StepSizes.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/StepSizes.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/StepSizes.java
new file mode 100644
index 0000000..adcb2ce
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/StepSizes.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.reef.examples.group.bgd.utils;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class StepSizes {
+
+  private static final Logger LOG = Logger.getLogger(StepSizes.class.getName());
+
+  private final double[] t;
+  private final int gridSize = 21;
+
+  @Inject
+  public StepSizes() {
+    this.t = new double[gridSize];
+    final int mid = (gridSize / 2);
+    t[mid] = 1;
+    for (int i = mid - 1; i >= 0; i--) {
+      t[i] = t[i + 1] / 2.0;
+    }
+    for (int i = mid + 1; i < gridSize; i++) {
+      t[i] = t[i - 1] * 2.0;
+    }
+  }
+
+  public double[] getT() {
+    return t;
+  }
+
+  public int getGridSize() {
+    return gridSize;
+  }
+
+  public static void main(final String[] args) {
+    // TODO Auto-generated method stub
+    final StepSizes t = new StepSizes();
+    LOG.log(Level.INFO, "OUT: {0}", Arrays.toString(t.getT()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/SubConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/SubConfiguration.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/SubConfiguration.java
new file mode 100644
index 0000000..4d74356
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/utils/SubConfiguration.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.reef.examples.group.bgd.utils;
+
+import org.apache.reef.driver.task.TaskConfiguration;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.examples.group.bgd.MasterTask;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.AvroConfigurationSerializer;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class SubConfiguration {
+
+  private static final Logger LOG = Logger.getLogger(SubConfiguration.class.getName());
+
+  @SafeVarargs
+  public static Configuration from(
+      final Configuration baseConf, final Class<? extends Name<?>>... classes) {
+
+    final Injector injector = Tang.Factory.getTang().newInjector(baseConf);
+    final JavaConfigurationBuilder confBuilder = Tang.Factory.getTang().newConfigurationBuilder();
+
+    for (final Class<? extends Name<?>> clazz : classes) {
+      try {
+        confBuilder.bindNamedParameter(clazz,
+            injector.getNamedInstance((Class<? extends Name<Object>>) clazz).toString());
+      } catch (final InjectionException ex) {
+        final String msg = "Exception while creating subconfiguration";
+        LOG.log(Level.WARNING, msg, ex);
+        throw new RuntimeException(msg, ex);
+      }
+    }
+
+    return confBuilder.build();
+  }
+
+  public static void main(final String[] args) throws InjectionException {
+
+    final Configuration conf = TaskConfiguration.CONF
+        .set(TaskConfiguration.IDENTIFIER, "TASK")
+        .set(TaskConfiguration.TASK, MasterTask.class)
+        .build();
+
+    final ConfigurationSerializer confSerizalizer = new AvroConfigurationSerializer();
+    final Configuration subConf = SubConfiguration.from(conf, TaskConfigurationOptions.Identifier.class);
+    LOG.log(Level.INFO, "OUT: Base conf:\n{0}", confSerizalizer.toString(conf));
+    LOG.log(Level.INFO, "OUT: Sub conf:\n{0}", confSerizalizer.toString(subConf));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastDriver.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastDriver.java
new file mode 100644
index 0000000..5505859
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastDriver.java
@@ -0,0 +1,285 @@
+/**
+ * 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.reef.examples.group.broadcast;
+
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.ClosedContext;
+import org.apache.reef.driver.context.ContextConfiguration;
+import org.apache.reef.driver.evaluator.AllocatedEvaluator;
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.apache.reef.driver.evaluator.EvaluatorRequestor;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.TaskConfiguration;
+import org.apache.reef.evaluator.context.parameters.ContextIdentifier;
+import org.apache.reef.examples.group.bgd.operatornames.ControlMessageBroadcaster;
+import org.apache.reef.examples.group.bgd.parameters.AllCommunicationGroup;
+import org.apache.reef.examples.group.bgd.parameters.ModelDimensions;
+import org.apache.reef.examples.group.broadcast.parameters.ModelBroadcaster;
+import org.apache.reef.examples.group.broadcast.parameters.ModelReceiveAckReducer;
+import org.apache.reef.examples.group.broadcast.parameters.NumberOfReceivers;
+import org.apache.reef.io.network.group.api.driver.CommunicationGroupDriver;
+import org.apache.reef.io.network.group.api.driver.GroupCommDriver;
+import org.apache.reef.io.network.group.impl.config.BroadcastOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.ReduceOperatorSpec;
+import org.apache.reef.io.serialization.SerializableCodec;
+import org.apache.reef.poison.PoisonedConfiguration;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.annotations.Unit;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.event.StartTime;
+
+import javax.inject.Inject;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+@DriverSide
+@Unit
+public class BroadcastDriver {
+
+  private static final Logger LOG = Logger.getLogger(BroadcastDriver.class.getName());
+
+  private final AtomicBoolean masterSubmitted = new AtomicBoolean(false);
+  private final AtomicInteger slaveIds = new AtomicInteger(0);
+  private final AtomicInteger failureSet = new AtomicInteger(0);
+
+  private final GroupCommDriver groupCommDriver;
+  private final CommunicationGroupDriver allCommGroup;
+  private final ConfigurationSerializer confSerializer;
+  private final int dimensions;
+  private final EvaluatorRequestor requestor;
+  private final int numberOfReceivers;
+  private final AtomicInteger numberOfAllocatedEvaluators;
+
+  private String groupCommConfiguredMasterId;
+
+  @Inject
+  public BroadcastDriver(
+      final EvaluatorRequestor requestor,
+      final GroupCommDriver groupCommDriver,
+      final ConfigurationSerializer confSerializer,
+      final @Parameter(ModelDimensions.class) int dimensions,
+      final @Parameter(NumberOfReceivers.class) int numberOfReceivers) {
+
+    this.requestor = requestor;
+    this.groupCommDriver = groupCommDriver;
+    this.confSerializer = confSerializer;
+    this.dimensions = dimensions;
+    this.numberOfReceivers = numberOfReceivers;
+    this.numberOfAllocatedEvaluators = new AtomicInteger(numberOfReceivers + 1);
+
+    this.allCommGroup = this.groupCommDriver.newCommunicationGroup(
+        AllCommunicationGroup.class, numberOfReceivers + 1);
+
+    LOG.info("Obtained all communication group");
+
+    this.allCommGroup
+        .addBroadcast(ControlMessageBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .addBroadcast(ModelBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .addReduce(ModelReceiveAckReducer.class,
+            ReduceOperatorSpec.newBuilder()
+                .setReceiverId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .setReduceFunctionClass(ModelReceiveAckReduceFunction.class)
+                .build())
+        .finalise();
+
+    LOG.info("Added operators to allCommGroup");
+  }
+
+  /**
+   * Handles the StartTime event: Request numOfReceivers Evaluators.
+   */
+  final class StartHandler implements EventHandler<StartTime> {
+    @Override
+    public void onNext(final StartTime startTime) {
+      final int numEvals = BroadcastDriver.this.numberOfReceivers + 1;
+      LOG.log(Level.FINE, "Requesting {0} evaluators", numEvals);
+      BroadcastDriver.this.requestor.submit(EvaluatorRequest.newBuilder()
+          .setNumber(numEvals)
+          .setMemory(2048)
+          .build());
+    }
+  }
+
+  /**
+   * Handles AllocatedEvaluator: Submits a context with an id
+   */
+  final class EvaluatorAllocatedHandler implements EventHandler<AllocatedEvaluator> {
+    @Override
+    public void onNext(final AllocatedEvaluator allocatedEvaluator) {
+      LOG.log(Level.INFO, "Submitting an id context to AllocatedEvaluator: {0}", allocatedEvaluator);
+      final Configuration contextConfiguration = ContextConfiguration.CONF
+          .set(ContextConfiguration.IDENTIFIER, "BroadcastContext-" +
+              BroadcastDriver.this.numberOfAllocatedEvaluators.getAndDecrement())
+          .build();
+      allocatedEvaluator.submitContext(contextConfiguration);
+    }
+  }
+
+  public class FailedTaskHandler implements EventHandler<FailedTask> {
+
+    @Override
+    public void onNext(final FailedTask failedTask) {
+
+      LOG.log(Level.FINE, "Got failed Task: {0}", failedTask.getId());
+
+      final ActiveContext activeContext = failedTask.getActiveContext().get();
+      final Configuration partialTaskConf = Tang.Factory.getTang()
+          .newConfigurationBuilder(
+              TaskConfiguration.CONF
+                  .set(TaskConfiguration.IDENTIFIER, failedTask.getId())
+                  .set(TaskConfiguration.TASK, SlaveTask.class)
+                  .build(),
+              PoisonedConfiguration.TASK_CONF
+                  .set(PoisonedConfiguration.CRASH_PROBABILITY, "0")
+                  .set(PoisonedConfiguration.CRASH_TIMEOUT, "1")
+                  .build())
+          .bindNamedParameter(ModelDimensions.class, "" + dimensions)
+          .build();
+
+      // Do not add the task back:
+      // allCommGroup.addTask(partialTaskConf);
+
+      final Configuration taskConf = groupCommDriver.getTaskConfiguration(partialTaskConf);
+      LOG.log(Level.FINER, "Submit SlaveTask conf: {0}", confSerializer.toString(taskConf));
+
+      activeContext.submitTask(taskConf);
+    }
+  }
+
+  public class ContextActiveHandler implements EventHandler<ActiveContext> {
+
+    private final AtomicBoolean storeMasterId = new AtomicBoolean(false);
+
+    @Override
+    public void onNext(final ActiveContext activeContext) {
+
+      LOG.log(Level.FINE, "Got active context: {0}", activeContext.getId());
+
+      /**
+       * The active context can be either from data loading service or after network
+       * service has loaded contexts. So check if the GroupCommDriver knows if it was
+       * configured by one of the communication groups.
+       */
+      if (groupCommDriver.isConfigured(activeContext)) {
+
+        if (activeContext.getId().equals(groupCommConfiguredMasterId) && !masterTaskSubmitted()) {
+
+          final Configuration partialTaskConf = Tang.Factory.getTang()
+              .newConfigurationBuilder(
+                  TaskConfiguration.CONF
+                      .set(TaskConfiguration.IDENTIFIER, MasterTask.TASK_ID)
+                      .set(TaskConfiguration.TASK, MasterTask.class)
+                      .build())
+              .bindNamedParameter(ModelDimensions.class, Integer.toString(dimensions))
+              .build();
+
+          allCommGroup.addTask(partialTaskConf);
+
+          final Configuration taskConf = groupCommDriver.getTaskConfiguration(partialTaskConf);
+          LOG.log(Level.FINER, "Submit MasterTask conf: {0}", confSerializer.toString(taskConf));
+
+          activeContext.submitTask(taskConf);
+
+        } else {
+
+          final Configuration partialTaskConf = Tang.Factory.getTang()
+              .newConfigurationBuilder(
+                  TaskConfiguration.CONF
+                      .set(TaskConfiguration.IDENTIFIER, getSlaveId(activeContext))
+                      .set(TaskConfiguration.TASK, SlaveTask.class)
+                      .build(),
+                  PoisonedConfiguration.TASK_CONF
+                      .set(PoisonedConfiguration.CRASH_PROBABILITY, "0.4")
+                      .set(PoisonedConfiguration.CRASH_TIMEOUT, "1")
+                      .build())
+              .bindNamedParameter(ModelDimensions.class, Integer.toString(dimensions))
+              .build();
+
+          allCommGroup.addTask(partialTaskConf);
+
+          final Configuration taskConf = groupCommDriver.getTaskConfiguration(partialTaskConf);
+          LOG.log(Level.FINER, "Submit SlaveTask conf: {0}", confSerializer.toString(taskConf));
+
+          activeContext.submitTask(taskConf);
+        }
+      } else {
+
+        final Configuration contextConf = groupCommDriver.getContextConfiguration();
+        final String contextId = contextId(contextConf);
+
+        if (storeMasterId.compareAndSet(false, true)) {
+          groupCommConfiguredMasterId = contextId;
+        }
+
+        final Configuration serviceConf = groupCommDriver.getServiceConfiguration();
+        LOG.log(Level.FINER, "Submit GCContext conf: {0}", confSerializer.toString(contextConf));
+        LOG.log(Level.FINER, "Submit Service conf: {0}", confSerializer.toString(serviceConf));
+
+        activeContext.submitContextAndService(contextConf, serviceConf);
+      }
+    }
+
+    private String contextId(final Configuration contextConf) {
+      try {
+        final Injector injector = Tang.Factory.getTang().newInjector(contextConf);
+        return injector.getNamedInstance(ContextIdentifier.class);
+      } catch (final InjectionException e) {
+        throw new RuntimeException("Unable to inject context identifier from context conf", e);
+      }
+    }
+
+    private String getSlaveId(final ActiveContext activeContext) {
+      return "SlaveTask-" + slaveIds.getAndIncrement();
+    }
+
+    private boolean masterTaskSubmitted() {
+      return !masterSubmitted.compareAndSet(false, true);
+    }
+  }
+
+  public class ContextCloseHandler implements EventHandler<ClosedContext> {
+
+    @Override
+    public void onNext(final ClosedContext closedContext) {
+      LOG.log(Level.FINE, "Got closed context: {0}", closedContext.getId());
+      final ActiveContext parentContext = closedContext.getParentContext();
+      if (parentContext != null) {
+        LOG.log(Level.FINE, "Closing parent context: {0}", parentContext.getId());
+        parentContext.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastREEF.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastREEF.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastREEF.java
new file mode 100644
index 0000000..cac6ccd
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/BroadcastREEF.java
@@ -0,0 +1,148 @@
+/**
+ * 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.reef.examples.group.broadcast;
+
+import org.apache.reef.annotations.audience.ClientSide;
+import org.apache.reef.client.DriverConfiguration;
+import org.apache.reef.client.DriverLauncher;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.examples.group.bgd.parameters.ModelDimensions;
+import org.apache.reef.examples.group.broadcast.parameters.NumberOfReceivers;
+import org.apache.reef.io.network.group.impl.driver.GroupCommService;
+import org.apache.reef.runtime.local.client.LocalRuntimeConfiguration;
+import org.apache.reef.runtime.yarn.client.YarnClientConfiguration;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.AvroConfigurationSerializer;
+import org.apache.reef.tang.formats.CommandLine;
+import org.apache.reef.util.EnvironmentUtils;
+
+import java.io.IOException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+@ClientSide
+public class BroadcastREEF {
+  private static final Logger LOG = Logger.getLogger(BroadcastREEF.class.getName());
+
+  private static final String NUM_LOCAL_THREADS = "20";
+
+  /**
+   * Number of milliseconds to wait for the job to complete.
+   */
+  private static final int JOB_TIMEOUT = 2 * 60 * 1000;
+
+  /**
+   * Command line parameter = true to run locally, or false to run on YARN.
+   */
+  @NamedParameter(doc = "Whether or not to run on the local runtime", short_name = "local", default_value = "true")
+  public static final class Local implements Name<Boolean> {
+  }
+
+  @NamedParameter(short_name = "input")
+  public static final class InputDir implements Name<String> {
+  }
+
+  private static boolean local;
+  private static int dimensions;
+  private static int numberOfReceivers;
+
+  private static Configuration parseCommandLine(final String[] aArgs) {
+    final JavaConfigurationBuilder cb = Tang.Factory.getTang().newConfigurationBuilder();
+    try {
+      final CommandLine cl = new CommandLine(cb);
+      cl.registerShortNameOfClass(Local.class);
+      cl.registerShortNameOfClass(ModelDimensions.class);
+      cl.registerShortNameOfClass(NumberOfReceivers.class);
+      cl.processCommandLine(aArgs);
+    } catch (final IOException ex) {
+      final String msg = "Unable to parse command line";
+      LOG.log(Level.SEVERE, msg, ex);
+      throw new RuntimeException(msg, ex);
+    }
+    return cb.build();
+  }
+
+  /**
+   * copy the parameters from the command line required for the Client configuration
+   */
+  private static void storeCommandLineArgs(
+      final Configuration commandLineConf) throws InjectionException {
+    final Injector injector = Tang.Factory.getTang().newInjector(commandLineConf);
+    local = injector.getNamedInstance(Local.class);
+    dimensions = injector.getNamedInstance(ModelDimensions.class);
+    numberOfReceivers = injector.getNamedInstance(NumberOfReceivers.class);
+  }
+
+  /**
+   * @return (immutable) TANG Configuration object.
+   */
+  private static Configuration getRunTimeConfiguration() {
+    final Configuration runtimeConfiguration;
+    if (local) {
+      LOG.log(Level.INFO, "Running Broadcast example using group API on the local runtime");
+      runtimeConfiguration = LocalRuntimeConfiguration.CONF
+          .set(LocalRuntimeConfiguration.NUMBER_OF_THREADS, NUM_LOCAL_THREADS)
+          .build();
+    } else {
+      LOG.log(Level.INFO, "Running Broadcast example using group API on YARN");
+      runtimeConfiguration = YarnClientConfiguration.CONF.build();
+    }
+    return runtimeConfiguration;
+  }
+
+  public static LauncherStatus runBGDReef(
+      final Configuration runtimeConfiguration) throws InjectionException {
+
+    final Configuration driverConfiguration = EnvironmentUtils
+        .addClasspath(DriverConfiguration.CONF, DriverConfiguration.GLOBAL_LIBRARIES)
+        .set(DriverConfiguration.ON_DRIVER_STARTED, BroadcastDriver.StartHandler.class)
+        .set(DriverConfiguration.ON_EVALUATOR_ALLOCATED, BroadcastDriver.EvaluatorAllocatedHandler.class)
+        .set(DriverConfiguration.ON_CONTEXT_ACTIVE, BroadcastDriver.ContextActiveHandler.class)
+        .set(DriverConfiguration.ON_CONTEXT_CLOSED, BroadcastDriver.ContextCloseHandler.class)
+        .set(DriverConfiguration.ON_TASK_FAILED, BroadcastDriver.FailedTaskHandler.class)
+        .set(DriverConfiguration.DRIVER_IDENTIFIER, "BroadcastDriver")
+        .build();
+
+    final Configuration groupCommServConfiguration = GroupCommService.getConfiguration();
+
+    final Configuration mergedDriverConfiguration = Tang.Factory.getTang()
+        .newConfigurationBuilder(groupCommServConfiguration, driverConfiguration)
+        .bindNamedParameter(ModelDimensions.class, Integer.toString(dimensions))
+        .bindNamedParameter(NumberOfReceivers.class, Integer.toString(numberOfReceivers))
+        .build();
+
+    LOG.info(new AvroConfigurationSerializer().toString(mergedDriverConfiguration));
+
+    return DriverLauncher.getLauncher(runtimeConfiguration).run(mergedDriverConfiguration, JOB_TIMEOUT);
+  }
+
+  public static void main(final String[] args) throws InjectionException {
+    final Configuration commandLineConf = parseCommandLine(args);
+    storeCommandLineArgs(commandLineConf);
+    final Configuration runtimeConfiguration = getRunTimeConfiguration();
+    final LauncherStatus state = runBGDReef(runtimeConfiguration);
+    LOG.log(Level.INFO, "REEF job completed: {0}", state);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ControlMessages.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ControlMessages.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ControlMessages.java
new file mode 100644
index 0000000..dc51076
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ControlMessages.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.reef.examples.group.broadcast;
+
+import java.io.Serializable;
+
+public enum ControlMessages implements Serializable {
+  ReceiveModel,
+  Stop
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/MasterTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/MasterTask.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/MasterTask.java
new file mode 100644
index 0000000..b5627d2
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/MasterTask.java
@@ -0,0 +1,97 @@
+/**
+ * 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.reef.examples.group.broadcast;
+
+import org.apache.reef.examples.group.bgd.operatornames.ControlMessageBroadcaster;
+import org.apache.reef.examples.group.bgd.parameters.AllCommunicationGroup;
+import org.apache.reef.examples.group.bgd.parameters.ModelDimensions;
+import org.apache.reef.examples.group.broadcast.parameters.ModelBroadcaster;
+import org.apache.reef.examples.group.broadcast.parameters.ModelReceiveAckReducer;
+import org.apache.reef.examples.group.utils.math.DenseVector;
+import org.apache.reef.examples.group.utils.math.Vector;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupClient;
+import org.apache.reef.io.network.group.api.task.GroupCommClient;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.task.Task;
+import org.mortbay.log.Log;
+
+import javax.inject.Inject;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class MasterTask implements Task {
+
+  public static final String TASK_ID = "MasterTask";
+
+  private static final Logger LOG = Logger.getLogger(MasterTask.class.getName());
+
+  private final CommunicationGroupClient communicationGroupClient;
+  private final Broadcast.Sender<ControlMessages> controlMessageBroadcaster;
+  private final Broadcast.Sender<Vector> modelBroadcaster;
+  private final Reduce.Receiver<Boolean> modelReceiveAckReducer;
+
+  private final int dimensions;
+
+  @Inject
+  public MasterTask(
+      final GroupCommClient groupCommClient,
+      final @Parameter(ModelDimensions.class) int dimensions) {
+
+    this.dimensions = dimensions;
+
+    this.communicationGroupClient = groupCommClient.getCommunicationGroup(AllCommunicationGroup.class);
+    this.controlMessageBroadcaster = communicationGroupClient.getBroadcastSender(ControlMessageBroadcaster.class);
+    this.modelBroadcaster = communicationGroupClient.getBroadcastSender(ModelBroadcaster.class);
+    this.modelReceiveAckReducer = communicationGroupClient.getReduceReceiver(ModelReceiveAckReducer.class);
+  }
+
+  @Override
+  public byte[] call(final byte[] memento) throws Exception {
+
+    final Vector model = new DenseVector(dimensions);
+    final long time1 = System.currentTimeMillis();
+    final int numIters = 10;
+
+    for (int i = 0; i < numIters; i++) {
+
+      controlMessageBroadcaster.send(ControlMessages.ReceiveModel);
+      modelBroadcaster.send(model);
+      modelReceiveAckReducer.reduce();
+
+      final GroupChanges changes = communicationGroupClient.getTopologyChanges();
+      if (changes.exist()) {
+        Log.info("There exist topology changes. Asking to update Topology");
+        communicationGroupClient.updateTopology();
+      } else {
+        Log.info("No changes in topology exist. So not updating topology");
+      }
+    }
+
+    final long time2 = System.currentTimeMillis();
+    LOG.log(Level.FINE, "Broadcasting vector of dimensions {0} took {1} secs",
+        new Object[]{dimensions, (time2 - time1) / (numIters * 1000.0)});
+
+    controlMessageBroadcaster.send(ControlMessages.Stop);
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ModelReceiveAckReduceFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ModelReceiveAckReduceFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ModelReceiveAckReduceFunction.java
new file mode 100644
index 0000000..e549cbc
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/ModelReceiveAckReduceFunction.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.reef.examples.group.broadcast;
+
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+
+import javax.inject.Inject;
+
+/**
+ *
+ */
+public class ModelReceiveAckReduceFunction implements ReduceFunction<Boolean> {
+
+  @Inject
+  public ModelReceiveAckReduceFunction() {
+  }
+
+  @Override
+  public Boolean apply(final Iterable<Boolean> elements) {
+    return true;
+  }
+
+}


[3/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeImpl.java
new file mode 100644
index 0000000..7483483
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeImpl.java
@@ -0,0 +1,476 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.api.driver.TaskNode;
+import org.apache.reef.io.network.group.api.driver.TaskNodeStatus;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EStage;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+public class TaskNodeImpl implements TaskNode {
+
+  private static final Logger LOG = Logger.getLogger(TaskNodeImpl.class.getName());
+
+  private final EStage<GroupCommunicationMessage> senderStage;
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String taskId;
+  private final String driverId;
+
+  private final boolean isRoot;
+  private TaskNode parent;
+  private TaskNode sibling;
+  private final List<TaskNode> children = new ArrayList<>();
+
+  private final AtomicBoolean running = new AtomicBoolean(false);
+  private final AtomicBoolean topoSetupSent = new AtomicBoolean(false);
+
+  private final TaskNodeStatus taskNodeStatus;
+
+  private final AtomicInteger version = new AtomicInteger(0);
+
+  public TaskNodeImpl(final EStage<GroupCommunicationMessage> senderStage,
+                      final Class<? extends Name<String>> groupName, final Class<? extends Name<String>> operatorName,
+                      final String taskId, final String driverId, final boolean isRoot) {
+    this.senderStage = senderStage;
+    this.groupName = groupName;
+    this.operName = operatorName;
+    this.taskId = taskId;
+    this.driverId = driverId;
+    this.isRoot = isRoot;
+    taskNodeStatus = new TaskNodeStatusImpl(groupName, operatorName, taskId, this);
+  }
+
+  @Override
+  public void setSibling(final TaskNode leaf) {
+    LOG.entering("TaskNodeImpl", "setSibling", new Object[]{getQualifiedName(), leaf});
+    sibling = leaf;
+    LOG.exiting("TaskNodeImpl", "setSibling", getQualifiedName());
+  }
+
+  @Override
+  public int getNumberOfChildren() {
+    LOG.entering("TaskNodeImpl", "getNumberOfChildren", getQualifiedName());
+    final int size = children.size();
+    LOG.exiting("TaskNodeImpl", "getNumberOfChildren", getQualifiedName() + size);
+    return size;
+  }
+
+  @Override
+  public TaskNode successor() {
+    LOG.entering("TaskNodeImpl", "successor", getQualifiedName());
+    LOG.exiting("TaskNodeImpl", "successor", getQualifiedName() + sibling);
+    return sibling;
+  }
+
+  @Override
+  public String toString() {
+    return "(" + taskId + "," + version.get() + ")";
+  }
+
+  /**
+   * * Methods pertaining to my status change ***
+   */
+  @Override
+  public void onFailedTask() {
+    LOG.entering("TaskNodeImpl", "onFailedTask", getQualifiedName());
+    if (!running.compareAndSet(true, false)) {
+      LOG.fine(getQualifiedName() + "Trying to set failed on an already failed task. Something fishy!!!");
+      LOG.exiting("TaskNodeImpl", "onFailedTask", getQualifiedName() + "Trying to set failed on an already failed task. Something fishy!!!");
+      return;
+    }
+    taskNodeStatus.clearStateAndReleaseLocks();
+    LOG.finest(getQualifiedName() + "Changed status to failed.");
+    LOG.finest(getQualifiedName() + "Resetting topoSetupSent to false");
+    topoSetupSent.set(false);
+    if (parent != null && parent.isRunning()) {
+      parent.onChildDead(taskId);
+    } else {
+      LOG.finest(getQualifiedName() + "Skipping asking parent to process child death");
+    }
+    for (final TaskNode child : children) {
+      if (child.isRunning()) {
+        child.onParentDead();
+      }
+    }
+    final int version = this.version.incrementAndGet();
+    LOG.finest(getQualifiedName() + "Bumping up to version-" + version);
+    LOG.exiting("TaskNodeImpl", "onFailedTask", getQualifiedName());
+  }
+
+  @Override
+  public void onRunningTask() {
+    LOG.entering("TaskNodeImpl", "onRunningTask", getQualifiedName());
+    if (!running.compareAndSet(false, true)) {
+      LOG.fine(getQualifiedName() + "Trying to set running on an already running task. Something fishy!!!");
+      LOG.exiting("TaskNodeImpl", "onRunningTask", getQualifiedName() + "Trying to set running on an already running task. Something fishy!!!");
+      return;
+    }
+    final int version = this.version.get();
+    LOG.finest(getQualifiedName() + "Changed status to running version-" + version);
+    if (parent != null && parent.isRunning()) {
+      final GroupCommunicationMessage gcm = Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentAdd, parent.getTaskId(),
+          parent.getVersion(), taskId,
+          version, Utils.EmptyByteArr);
+      taskNodeStatus.expectAckFor(gcm.getType(), gcm.getSrcid());
+      senderStage.onNext(gcm);
+      parent.onChildRunning(taskId);
+    } else {
+      LOG.finest(getQualifiedName() + "Skipping src add to & for parent");
+    }
+    for (final TaskNode child : children) {
+      if (child.isRunning()) {
+        final GroupCommunicationMessage gcm = Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, child.getTaskId(),
+            child.getVersion(), taskId, version,
+            Utils.EmptyByteArr);
+        taskNodeStatus.expectAckFor(gcm.getType(), gcm.getSrcid());
+        senderStage.onNext(gcm);
+        child.onParentRunning();
+      }
+    }
+    LOG.exiting("TaskNodeImpl", "onRunningTask", getQualifiedName());
+  }
+
+  /**
+   * * Methods pertaining to my status change ends ***
+   */
+
+  @Override
+  public void onParentRunning() {
+    LOG.entering("TaskNodeImpl", "onParentRunning", getQualifiedName());
+    if (parent != null && parent.isRunning()) {
+      final int parentVersion = parent.getVersion();
+      final String parentTaskId = parent.getTaskId();
+      final GroupCommunicationMessage gcm = Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentAdd, parentTaskId,
+          parentVersion, taskId, version.get(),
+          Utils.EmptyByteArr);
+      taskNodeStatus.expectAckFor(gcm.getType(), gcm.getSrcid());
+      senderStage.onNext(gcm);
+    } else {
+      LOG.finer(getQualifiedName() + "Parent was running when I was asked to add him."
+          + " However, he is not active anymore. Returning without sending ParentAdd" + " msg. ***CHECK***");
+    }
+    LOG.exiting("TaskNodeImpl", "onParentRunning", getQualifiedName());
+  }
+
+  @Override
+  public void onParentDead() {
+    LOG.entering("TaskNodeImpl", "onParentDead", getQualifiedName());
+    if (parent != null) {
+      final int parentVersion = parent.getVersion();
+      final String parentTaskId = parent.getTaskId();
+      taskNodeStatus.updateFailureOf(parent.getTaskId());
+      final GroupCommunicationMessage gcm = Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ParentDead, parentTaskId,
+          parentVersion, taskId, version.get(),
+          Utils.EmptyByteArr);
+      taskNodeStatus.expectAckFor(gcm.getType(), gcm.getSrcid());
+      senderStage.onNext(gcm);
+    } else {
+      throw new RuntimeException(getQualifiedName() + "Don't expect parent to be null. Something wrong");
+    }
+    LOG.exiting("TaskNodeImpl", "onParentDead", getQualifiedName());
+  }
+
+  @Override
+  public void onChildRunning(final String childId) {
+    LOG.entering("TaskNodeImpl", "onChildRunning", new Object[]{getQualifiedName(), childId});
+    final TaskNode childTask = findTask(childId);
+    if (childTask != null && childTask.isRunning()) {
+      final int childVersion = childTask.getVersion();
+      final GroupCommunicationMessage gcm = Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildAdd, childId,
+          childVersion, taskId, version.get(),
+          Utils.EmptyByteArr);
+      taskNodeStatus.expectAckFor(gcm.getType(), gcm.getSrcid());
+      senderStage.onNext(gcm);
+    } else {
+      LOG.fine(getQualifiedName() + childId + " was running when I was asked to add him."
+          + " However, I can't find a task corresponding to him now."
+          + " Returning without sending ChildAdd msg. ***CHECK***");
+    }
+    LOG.exiting("TaskNodeImpl", "onChildRunning", getQualifiedName() + childId);
+  }
+
+  @Override
+  public void onChildDead(final String childId) {
+    LOG.entering("TaskNodeImpl", "onChildDead", new Object[]{getQualifiedName(), childId});
+    final TaskNode childTask = findChildTask(childId);
+    if (childTask != null) {
+      final int childVersion = childTask.getVersion();
+      taskNodeStatus.updateFailureOf(childId);
+      final GroupCommunicationMessage gcm = Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.ChildDead, childId,
+          childVersion, taskId, version.get(),
+          Utils.EmptyByteArr);
+      taskNodeStatus.expectAckFor(gcm.getType(), gcm.getSrcid());
+      senderStage.onNext(gcm);
+    } else {
+      throw new RuntimeException(getQualifiedName() + "Don't expect task for " + childId + " to be null. Something wrong");
+    }
+    LOG.exiting("TaskNodeImpl", "onChildDead", getQualifiedName() + childId);
+  }
+
+  /**
+   * * Methods pertaining to my neighbors status change ends ***
+   */
+
+  @Override
+  public void onReceiptOfAcknowledgement(final GroupCommunicationMessage msg) {
+    LOG.entering("TaskNodeImpl", "onReceiptOfAcknowledgement", new Object[]{getQualifiedName(), msg});
+    taskNodeStatus.processAcknowledgement(msg);
+    LOG.exiting("TaskNodeImpl", "onReceiptOfAcknowledgement", getQualifiedName() + msg);
+  }
+
+  @Override
+  public void updatingTopology() {
+    LOG.entering("TaskNodeImpl", "updatingTopology", getQualifiedName());
+    taskNodeStatus.updatingTopology();
+    LOG.exiting("TaskNodeImpl", "updatingTopology", getQualifiedName());
+  }
+
+  @Override
+  public String getTaskId() {
+    return taskId;
+  }
+
+  @Override
+  public void addChild(final TaskNode child) {
+    LOG.entering("TaskNodeImpl", "addChild", new Object[]{getQualifiedName(), child.getTaskId()});
+    children.add(child);
+    LOG.exiting("TaskNodeImpl", "addChild", getQualifiedName() + child);
+  }
+
+  @Override
+  public void removeChild(final TaskNode child) {
+    LOG.entering("TaskNodeImpl", "removeChild", new Object[]{getQualifiedName(), child.getTaskId()});
+    children.remove(child);
+    LOG.exiting("TaskNodeImpl", "removeChild", getQualifiedName() + child);
+  }
+
+  @Override
+  public void setParent(final TaskNode parent) {
+    LOG.entering("TaskNodeImpl", "setParent", new Object[]{getQualifiedName(), parent});
+    this.parent = parent;
+    LOG.exiting("TaskNodeImpl", "setParent", getQualifiedName() + parent);
+  }
+
+  @Override
+  public boolean isRunning() {
+    LOG.entering("TaskNodeImpl", "isRunning", getQualifiedName());
+    final boolean b = running.get();
+    LOG.exiting("TaskNodeImpl", "isRunning", getQualifiedName() + b);
+    return b;
+  }
+
+  @Override
+  public TaskNode getParent() {
+    LOG.entering("TaskNodeImpl", "getParent", getQualifiedName());
+    LOG.exiting("TaskNodeImpl", "getParent", getQualifiedName() + parent);
+    return parent;
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":(" + taskId + "," + getVersion() + ") - ";
+  }
+
+  @Override
+  public boolean isNeighborActive(final String neighborId) {
+    LOG.entering("TaskNodeImpl", "isNeighborActive", new Object[]{getQualifiedName(), neighborId});
+    final boolean active = taskNodeStatus.isActive(neighborId);
+    LOG.exiting("TaskNodeImpl", "isNeighborActive", getQualifiedName() + active);
+    return active;
+  }
+
+  @Override
+  public boolean resetTopologySetupSent() {
+    LOG.entering("TaskNodeImpl", "resetTopologySetupSent", new Object[]{getQualifiedName(),});
+    final boolean retVal = topoSetupSent.compareAndSet(true, false);
+    LOG.exiting("TaskNodeImpl", "resetTopologySetupSent", getQualifiedName() + retVal);
+    return retVal;
+  }
+
+  @Override
+  public void checkAndSendTopologySetupMessage() {
+    LOG.entering("TaskNodeImpl", "checkAndSendTopologySetupMessage", getQualifiedName());
+    if (!topoSetupSent.get()
+        && (parentActive() && activeNeighborOfParent())
+        && (allChildrenActive() && activeNeighborOfAllChildren())) {
+      sendTopoSetupMsg();
+    }
+    LOG.exiting("TaskNodeImpl", "checkAndSendTopologySetupMessage", getQualifiedName());
+  }
+
+  private void sendTopoSetupMsg() {
+    LOG.entering("TaskNodeImpl", "sendTopoSetupMsg", getQualifiedName() + taskId);
+    LOG.fine(getQualifiedName() + "is an active participant in the topology");
+    senderStage.onNext(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologySetup, driverId, 0, taskId,
+        version.get(), Utils.EmptyByteArr));
+    taskNodeStatus.onTopologySetupMessageSent();
+    final boolean sentAlready = !topoSetupSent.compareAndSet(false, true);
+    if (sentAlready) {
+      LOG.fine(getQualifiedName() + "TopologySetup msg was sent more than once. Something fishy!!!");
+    }
+    LOG.exiting("TaskNodeImpl", "sendTopoSetupMsg", getQualifiedName());
+  }
+
+  @Override
+  public void checkAndSendTopologySetupMessageFor(final String source) {
+    LOG.entering("TaskNodeImpl", "checkAndSendTopologySetupMessageFor", new Object[]{getQualifiedName(), source});
+    final TaskNode srcNode = findTask(source);
+    if (srcNode != null) {
+      srcNode.checkAndSendTopologySetupMessage();
+    }
+    LOG.exiting("TaskNodeImpl", "checkAndSendTopologySetupMessageFor", getQualifiedName() + source);
+  }
+
+  /**
+   * @param sourceId
+   * @return
+   */
+  private TaskNode findTask(final String sourceId) {
+    LOG.entering("TaskNodeImpl", "findTask", new Object[]{getQualifiedName(), sourceId});
+    final TaskNode retNode;
+    if (parent != null && parent.getTaskId().equals(sourceId)) {
+      retNode = parent;
+    } else {
+      retNode = findChildTask(sourceId);
+    }
+    LOG.exiting("TaskNodeImpl", "findTask", getQualifiedName() + retNode);
+    return retNode;
+  }
+
+  private TaskNode findChildTask(final String sourceId) {
+    LOG.entering("TaskNodeImpl", "findChildTask", new Object[]{getQualifiedName(), sourceId});
+    TaskNode retNode = null;
+    for (final TaskNode child : children) {
+      if (child.getTaskId().equals(sourceId)) {
+        retNode = child;
+        break;
+      }
+    }
+    LOG.exiting("TaskNodeImpl", "findChildTask", getQualifiedName() + retNode);
+    return retNode;
+  }
+
+  private boolean parentActive() {
+    LOG.entering("TaskNodeImpl", "parentActive", getQualifiedName());
+    if (isRoot) {
+      LOG.exiting("TaskNodeImpl", "parentActive", Arrays.toString(new Object[]{true, getQualifiedName(), "I am root. Will never have parent. So signalling active"}));
+      return true;
+    }
+    if (isNeighborActive(parent.getTaskId())) {
+      LOG.exiting("TaskNodeImpl", "parentActive", Arrays.toString(new Object[]{true, getQualifiedName(), parent, " is an active neghbor"}));
+      return true;
+    }
+    LOG.exiting("TaskNodeImpl", "parentActive", getQualifiedName() + "Neither root Nor is " + parent + " an active neghbor");
+    return false;
+  }
+
+  private boolean activeNeighborOfParent() {
+    LOG.entering("TaskNodeImpl", "activeNeighborOfParent", getQualifiedName());
+    if (isRoot) {
+      LOG.exiting("TaskNodeImpl", "activeNeighborOfParent", Arrays.toString(new Object[]{true, getQualifiedName(), "I am root. Will never have parent. So signalling active"}));
+      return true;
+    }
+    if (parent.isNeighborActive(taskId)) {
+      LOG.exiting("TaskNodeImpl", "activeNeighborOfParent", Arrays.toString(new Object[]{true, getQualifiedName(), "I am an active neighbor of parent ", parent}));
+      return true;
+    }
+    LOG.exiting("TaskNodeImpl", "activeNeighborOfParent", Arrays.toString(new Object[]{false, getQualifiedName(), "Neither is parent null Nor am I an active neighbor of parent ", parent}));
+    return false;
+  }
+
+  private boolean allChildrenActive() {
+    LOG.entering("TaskNodeImpl", "allChildrenActive", getQualifiedName());
+    for (final TaskNode child : children) {
+      final String childId = child.getTaskId();
+      if (child.isRunning() && !isNeighborActive(childId)) {
+        LOG.exiting("TaskNodeImpl", "allChildrenActive", Arrays.toString(new Object[]{false, getQualifiedName(), childId, " not active yet"}));
+        return false;
+      }
+    }
+    LOG.exiting("TaskNodeImpl", "allChildrenActive", Arrays.toString(new Object[]{true, getQualifiedName(), "All children active"}));
+    return true;
+  }
+
+  private boolean activeNeighborOfAllChildren() {
+    LOG.entering("TaskNodeImpl", "activeNeighborOfAllChildren", getQualifiedName());
+    for (final TaskNode child : children) {
+      if (child.isRunning() && !child.isNeighborActive(taskId)) {
+        LOG.exiting("TaskNodeImpl", "activeNeighborOfAllChildren", Arrays.toString(new Object[]{false, getQualifiedName(), "Not an active neighbor of child ", child}));
+        return false;
+      }
+    }
+    LOG.exiting("TaskNodeImpl", "activeNeighborOfAllChildren", Arrays.toString(new Object[]{true, getQualifiedName(), "Active neighbor of all children"}));
+    return true;
+  }
+
+  @Override
+  public void waitForTopologySetupOrFailure() {
+    LOG.entering("TaskNodeImpl", "waitForTopologySetupOrFailure", getQualifiedName());
+    taskNodeStatus.waitForTopologySetup();
+    LOG.exiting("TaskNodeImpl", "waitForTopologySetupOrFailure", getQualifiedName());
+  }
+
+  @Override
+  public boolean hasChanges() {
+    LOG.entering("TaskNodeImpl", "hasChanges", getQualifiedName());
+    final boolean changes = taskNodeStatus.hasChanges();
+    LOG.exiting("TaskNodeImpl", "hasChanges", getQualifiedName() + changes);
+    return changes;
+  }
+
+  @Override
+  public int getVersion() {
+    return version.get();
+  }
+
+  @Override
+  public int hashCode() {
+    int r = taskId.hashCode();
+    r = 31 * r + version.get();
+    return r;
+  }
+
+  @Override
+  public boolean equals(final Object obj) {
+    if (obj != this) {
+      if (obj instanceof TaskNodeImpl) {
+        final TaskNodeImpl that = (TaskNodeImpl) obj;
+        if (this.taskId.equals(that.taskId) && this.version.get() == that.version.get()) {
+          return true;
+        } else {
+          return false;
+        }
+      } else {
+        return false;
+      }
+    } else {
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeStatusImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeStatusImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeStatusImpl.java
new file mode 100644
index 0000000..3f904e2
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskNodeStatusImpl.java
@@ -0,0 +1,267 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.api.driver.TaskNode;
+import org.apache.reef.io.network.group.api.driver.TaskNodeStatus;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.ConcurrentCountingMap;
+import org.apache.reef.io.network.group.impl.utils.CountingMap;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos.GroupCommMessage.Type;
+import org.apache.reef.tang.annotations.Name;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+public class TaskNodeStatusImpl implements TaskNodeStatus {
+
+  private static final Logger LOG = Logger.getLogger(TaskNodeStatusImpl.class.getName());
+
+  private final ConcurrentCountingMap<Type, String> statusMap = new ConcurrentCountingMap<>();
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String taskId;
+  private final Set<String> activeNeighbors = new HashSet<>();
+  private final CountingMap<String> neighborStatus = new CountingMap<>();
+  private final AtomicBoolean updatingTopo = new AtomicBoolean(false);
+  private final Object topoUpdateStageLock = new Object();
+  private final Object topoSetupSentLock = new Object();
+  private final TaskNode node;
+
+  public TaskNodeStatusImpl(final Class<? extends Name<String>> groupName,
+                            final Class<? extends Name<String>> operName, final String taskId, final TaskNode node) {
+    this.groupName = groupName;
+    this.operName = operName;
+    this.taskId = taskId;
+    this.node = node;
+  }
+
+  private boolean isDeadMsg(final Type msgAcked) {
+    return msgAcked == Type.ParentDead || msgAcked == Type.ChildDead;
+  }
+
+  private boolean isAddMsg(final Type msgAcked) {
+    return msgAcked == Type.ParentAdd || msgAcked == Type.ChildAdd;
+  }
+
+  private Type getAckedMsg(final Type msgType) {
+    switch (msgType) {
+      case ParentAdded:
+        return Type.ParentAdd;
+      case ChildAdded:
+        return Type.ChildAdd;
+      case ParentRemoved:
+        return Type.ParentDead;
+      case ChildRemoved:
+        return Type.ChildDead;
+      default:
+        return msgType;
+    }
+  }
+
+  private void chkIamActiveToSendTopoSetup(final Type msgDealt) {
+    LOG.entering("TaskNodeStatusImpl", "chkAndSendTopoSetup", new Object[]{getQualifiedName(), msgDealt});
+    if (statusMap.isEmpty()) {
+      LOG.finest(getQualifiedName() + "Empty status map.");
+      node.checkAndSendTopologySetupMessage();
+    } else {
+      LOG.finest(getQualifiedName() + "Status map non-empty" + statusMap);
+    }
+    LOG.exiting("TaskNodeStatusImpl", "chkAndSendTopoSetup", getQualifiedName() + msgDealt);
+  }
+
+  @Override
+  public void onTopologySetupMessageSent() {
+    LOG.entering("TaskNodeStatusImpl", "onTopologySetupMessageSent", getQualifiedName());
+    neighborStatus.clear();
+    synchronized (topoSetupSentLock) {
+      topoSetupSentLock.notifyAll();
+    }
+    LOG.exiting("TaskNodeStatusImpl", "onTopologySetupMessageSent", getQualifiedName());
+  }
+
+  @Override
+  public boolean isActive(final String neighborId) {
+    LOG.entering("TaskNodeStatusImpl", "isActive", new Object[]{getQualifiedName(), neighborId});
+    final boolean contains = activeNeighbors.contains(neighborId);
+    LOG.exiting("TaskNodeStatusImpl", "isActive", getQualifiedName() + contains);
+    return contains;
+  }
+
+  /**
+   * This needs to happen in line rather than in a stage because we need to note
+   * the messages we send to the tasks before we start processing msgs from the
+   * nodes.(Acks & Topology msgs)
+   */
+  @Override
+  public void expectAckFor(final Type msgType, final String srcId) {
+    LOG.entering("TaskNodeStatusImpl", "expectAckFor", new Object[]{getQualifiedName(), msgType, srcId});
+    LOG.finest(getQualifiedName() + "Adding " + srcId + " to sources");
+    statusMap.add(msgType, srcId);
+    LOG.exiting("TaskNodeStatusImpl", "expectAckFor", getQualifiedName() + "Sources from which ACKs for " + msgType + " are expected: " + statusMap.get(msgType));
+  }
+
+  @Override
+  public void clearStateAndReleaseLocks() {
+    LOG.entering("TaskNodeStatusImpl", "clearStateAndReleaseLocks", getQualifiedName());
+    statusMap.clear();
+    activeNeighbors.clear();
+    neighborStatus.clear();
+    updatingTopo.compareAndSet(true, false);
+    synchronized (topoSetupSentLock) {
+      topoSetupSentLock.notifyAll();
+    }
+    synchronized (topoUpdateStageLock) {
+      topoUpdateStageLock.notifyAll();
+    }
+    LOG.exiting("TaskNodeStatusImpl", "clearStateAndReleaseLocks", getQualifiedName());
+  }
+
+  @Override
+  public void updateFailureOf(final String taskId) {
+    LOG.entering("TaskNodeStatusImpl", "updateFailureOf", new Object[]{getQualifiedName(), taskId});
+    activeNeighbors.remove(taskId);
+    neighborStatus.remove(taskId);
+    LOG.exiting("TaskNodeStatusImpl", "updateFailureOf", getQualifiedName());
+  }
+
+  @Override
+  public void processAcknowledgement(final GroupCommunicationMessage gcm) {
+    LOG.entering("TaskNodeStatusImpl", "processMsg", new Object[]{getQualifiedName(), gcm});
+    final String self = gcm.getSrcid();
+    final Type msgType = gcm.getType();
+    final Type msgAcked = getAckedMsg(msgType);
+    final String sourceId = gcm.getDestid();
+    switch (msgType) {
+      case TopologySetup:
+        synchronized (topoUpdateStageLock) {
+          if (!updatingTopo.compareAndSet(true, false)) {
+            LOG.fine(getQualifiedName() + "Was expecting updateTopo to be true but it was false");
+          }
+          topoUpdateStageLock.notifyAll();
+        }
+        break;
+      case ParentAdded:
+      case ChildAdded:
+      case ParentRemoved:
+      case ChildRemoved:
+        processNeighborAcks(gcm, msgType, msgAcked, sourceId);
+        break;
+
+      default:
+        LOG.fine(getQualifiedName() + "Non ACK msg " + gcm.getType() + " for " + gcm.getDestid() + " unexpected");
+        break;
+    }
+    LOG.exiting("TaskNodeStatusImpl", "processMsg", getQualifiedName());
+  }
+
+  private void processNeighborAcks(final GroupCommunicationMessage gcm, final Type msgType, final Type msgAcked,
+                                   final String sourceId) {
+    LOG.entering("TaskNodeStatusImpl", "processNeighborAcks", getQualifiedName() + gcm);
+    if (statusMap.containsKey(msgAcked)) {
+      if (statusMap.contains(msgAcked, sourceId)) {
+        statusMap.remove(msgAcked, sourceId);
+        updateNeighborStatus(msgAcked, sourceId);
+        checkNeighborActiveToSendTopoSetup(sourceId);
+        chkIamActiveToSendTopoSetup(msgAcked);
+      } else {
+        LOG.fine(getQualifiedName() + "NodeStatusMsgProcessorStage Got " + msgType + " from a source(" + sourceId
+            + ") to whom ChildAdd was not sent. "
+            + "Perhaps reset during failure. If context not indicative use ***CAUTION***");
+      }
+    } else {
+      LOG.fine(getQualifiedName() + "NodeStatusMsgProcessorStage There were no " + msgAcked
+          + " msgs sent in the previous update cycle. "
+          + "Perhaps reset during failure. If context not indicative use ***CAUTION***");
+    }
+    LOG.exiting("TaskNodeStatusImpl", "processNeighborAcks", getQualifiedName() + gcm);
+  }
+
+  private void checkNeighborActiveToSendTopoSetup(final String sourceId) {
+    LOG.entering("TaskNodeStatusImpl", "checkNeighborActiveToSendTopoSetup", new Object[]{getQualifiedName(),
+        sourceId});
+    if (statusMap.notContains(sourceId)) {
+      //All msgs corresponding to sourceId have been ACKed
+      if (neighborStatus.get(sourceId) > 0) {
+        activeNeighbors.add(sourceId);
+        node.checkAndSendTopologySetupMessageFor(sourceId);
+      } else {
+        LOG.finest(getQualifiedName() + sourceId + " is not a neighbor anymore");
+      }
+    } else {
+      LOG.finest(getQualifiedName() + "Not done processing " + sourceId + " acks yet. So it is still inactive");
+    }
+    LOG.exiting("TaskNodeStatusImpl", "checkNeighborActiveToSendTopoSetup", getQualifiedName() + sourceId);
+  }
+
+  private void updateNeighborStatus(final Type msgAcked, final String sourceId) {
+    LOG.entering("TaskNodeStatusImpl", "updateNeighborStatus", new Object[]{getQualifiedName(), msgAcked, sourceId});
+    if (isAddMsg(msgAcked)) {
+      neighborStatus.add(sourceId);
+    } else if (isDeadMsg(msgAcked)) {
+      neighborStatus.remove(sourceId);
+    } else {
+      throw new RuntimeException("Can only deal with Neigbor ACKs while I received " + msgAcked + " from " + sourceId);
+    }
+    LOG.exiting("TaskNodeStatusImpl", "updateNeighborStatus", new Object[]{getQualifiedName(), msgAcked, sourceId});
+  }
+
+  @Override
+  public void updatingTopology() {
+    LOG.entering("TaskNodeStatusImpl", "updatingTopology", getQualifiedName());
+    final boolean topoBeingUpdated = !updatingTopo.compareAndSet(false, true);
+    if (topoBeingUpdated) {
+      throw new RuntimeException(getQualifiedName() + "Was expecting updateTopo to be false but it was true");
+    }
+    LOG.exiting("TaskNodeStatusImpl", "updatingTopology", getQualifiedName());
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":(" + taskId + "," + node.getVersion() + ") - ";
+  }
+
+  @Override
+  public boolean hasChanges() {
+    LOG.entering("TaskNodeStatusImpl", "hasChanges", getQualifiedName());
+    final boolean notEmpty = !statusMap.isEmpty();
+    LOG.exiting("TaskNodeStatusImpl", "hasChanges", getQualifiedName() + notEmpty);
+    return notEmpty;
+  }
+
+  @Override
+  public void waitForTopologySetup() {
+    LOG.entering("TaskNodeStatusImpl", "waitForTopologySetup", getQualifiedName());
+    LOG.finest("Waiting to acquire topoUpdateStageLock");
+    synchronized (topoUpdateStageLock) {
+      LOG.finest(getQualifiedName() + "Acquired topoUpdateStageLock. updatingTopo: " + updatingTopo.get());
+      while (updatingTopo.get() && node.isRunning()) {
+        try {
+          LOG.finest(getQualifiedName() + "Waiting on topoUpdateStageLock");
+          topoUpdateStageLock.wait();
+        } catch (final InterruptedException e) {
+          throw new RuntimeException("InterruptedException in NodeTopologyUpdateWaitStage "
+              + "while waiting for receiving TopologySetup", e);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskState.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskState.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskState.java
new file mode 100644
index 0000000..8404d89
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TaskState.java
@@ -0,0 +1,23 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+public enum TaskState {
+  NOT_STARTED, RUNNING, FAILED;
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedEvaluatorHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedEvaluatorHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedEvaluatorHandler.java
new file mode 100644
index 0000000..228cf4b
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedEvaluatorHandler.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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.logging.Logger;
+
+public class TopologyFailedEvaluatorHandler implements EventHandler<FailedEvaluator> {
+
+  private static final Logger LOG = Logger.getLogger(TopologyFailedEvaluatorHandler.class.getName());
+
+
+  private final CommunicationGroupDriverImpl communicationGroupDriverImpl;
+
+  public TopologyFailedEvaluatorHandler(final CommunicationGroupDriverImpl communicationGroupDriverImpl) {
+    this.communicationGroupDriverImpl = communicationGroupDriverImpl;
+  }
+
+  @Override
+  public void onNext(final FailedEvaluator failedEvaluator) {
+    final String failedEvaluatorId = failedEvaluator.getId();
+    LOG.entering("TopologyFailedEvaluatorHandler", "onNext", failedEvaluatorId);
+    if (failedEvaluator.getFailedTask().isPresent()) {
+      final String failedTaskId = failedEvaluator.getFailedTask().get().getId();
+      LOG.finest("Failed Evaluator contains a failed task: " + failedTaskId);
+      communicationGroupDriverImpl.failTask(failedTaskId);
+      communicationGroupDriverImpl.removeTask(failedTaskId);
+    }
+    LOG.exiting("TopologyFailedEvaluatorHandler", "onNext", failedEvaluatorId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedTaskHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedTaskHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedTaskHandler.java
new file mode 100644
index 0000000..a7ce7f7
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyFailedTaskHandler.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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.logging.Logger;
+
+public class TopologyFailedTaskHandler implements EventHandler<FailedTask> {
+
+  private static final Logger LOG = Logger.getLogger(TopologyFailedTaskHandler.class.getName());
+
+
+  private final CommunicationGroupDriverImpl communicationGroupDriverImpl;
+
+  public TopologyFailedTaskHandler(final CommunicationGroupDriverImpl communicationGroupDriverImpl) {
+    this.communicationGroupDriverImpl = communicationGroupDriverImpl;
+  }
+
+  @Override
+  public void onNext(final FailedTask failedTask) {
+    final String failedTaskId = failedTask.getId();
+    LOG.entering("TopologyFailedTaskHandler", "onNext", failedTaskId);
+    communicationGroupDriverImpl.failTask(failedTaskId);
+    LOG.exiting("TopologyFailedTaskHandler", "onNext", failedTaskId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyMessageHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyMessageHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyMessageHandler.java
new file mode 100644
index 0000000..e651aa7
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyMessageHandler.java
@@ -0,0 +1,44 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.logging.Logger;
+
+public class TopologyMessageHandler implements EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(TopologyMessageHandler.class.getName());
+
+
+  private final CommunicationGroupDriverImpl communicationGroupDriverImpl;
+
+  public TopologyMessageHandler(final CommunicationGroupDriverImpl communicationGroupDriverImpl) {
+    this.communicationGroupDriverImpl = communicationGroupDriverImpl;
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    LOG.entering("TopologyMessageHandler", "onNext", msg);
+    communicationGroupDriverImpl.processMsg(msg);
+    LOG.exiting("TopologyMessageHandler", "onNext", msg);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyRunningTaskHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyRunningTaskHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyRunningTaskHandler.java
new file mode 100644
index 0000000..d6e0fae
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyRunningTaskHandler.java
@@ -0,0 +1,44 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.logging.Logger;
+
+public class TopologyRunningTaskHandler implements EventHandler<RunningTask> {
+
+  private static final Logger LOG = Logger.getLogger(TopologyRunningTaskHandler.class.getName());
+
+  private final CommunicationGroupDriverImpl communicationGroupDriverImpl;
+
+  public TopologyRunningTaskHandler(final CommunicationGroupDriverImpl communicationGroupDriverImpl) {
+    this.communicationGroupDriverImpl = communicationGroupDriverImpl;
+  }
+
+  @Override
+  public void onNext(final RunningTask runningTask) {
+    final String runningTaskId = runningTask.getId();
+    LOG.entering("TopologyRunningTaskHandler", "onNext", runningTaskId);
+    communicationGroupDriverImpl.runTask(runningTaskId);
+    LOG.exiting("TopologyRunningTaskHandler", "onNext", runningTaskId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyUpdateWaitHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyUpdateWaitHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyUpdateWaitHandler.java
new file mode 100644
index 0000000..3905d47
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TopologyUpdateWaitHandler.java
@@ -0,0 +1,94 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.api.driver.TaskNode;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EStage;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.List;
+import java.util.logging.Logger;
+
+/**
+ *
+ */
+public class TopologyUpdateWaitHandler implements EventHandler<List<TaskNode>> {
+
+  private static final Logger LOG = Logger.getLogger(TopologyUpdateWaitHandler.class.getName());
+  private final EStage<GroupCommunicationMessage> senderStage;
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String driverId;
+  private final int driverVersion;
+  private final String dstId;
+  private final int dstVersion;
+  private final String qualifiedName;
+
+
+  /**
+   * The handler will wait for all nodes to acquire topoLock
+   * and send TopologySetup msg. Then it will send TopologyUpdated
+   * msg. However, any local topology changes are not in effect
+   * till driver sends TopologySetup once statusMap is emptied
+   * The operations in the tasks that have topology changes will
+   * wait for this. However other tasks that do not have any changes
+   * will continue their regular operation
+   */
+  public TopologyUpdateWaitHandler(final EStage<GroupCommunicationMessage> senderStage,
+                                   final Class<? extends Name<String>> groupName, final Class<? extends Name<String>> operName,
+                                   final String driverId, final int driverVersion, final String dstId, final int dstVersion,
+                                   final String qualifiedName) {
+    super();
+    this.senderStage = senderStage;
+    this.groupName = groupName;
+    this.operName = operName;
+    this.driverId = driverId;
+    this.driverVersion = driverVersion;
+    this.dstId = dstId;
+    this.dstVersion = dstVersion;
+    this.qualifiedName = qualifiedName;
+  }
+
+
+  @Override
+  public void onNext(final List<TaskNode> nodes) {
+    LOG.entering("TopologyUpdateWaitHandler", "onNext", new Object[]{qualifiedName, nodes});
+
+    for (final TaskNode node : nodes) {
+      LOG.fine(qualifiedName + "Waiting for " + node + " to enter TopologyUdate phase");
+      node.waitForTopologySetupOrFailure();
+      if (node.isRunning()) {
+        LOG.fine(qualifiedName + node + " is in TopologyUpdate phase");
+      } else {
+        LOG.fine(qualifiedName + node + " has failed");
+      }
+    }
+    LOG.finest(qualifiedName + "NodeTopologyUpdateWaitStage All to be updated nodes " + "have received TopologySetup");
+    LOG.fine(qualifiedName + "All affected parts of the topology are in TopologyUpdate phase. Will send a note to ("
+        + dstId + "," + dstVersion + ")");
+    senderStage.onNext(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyUpdated, driverId, driverVersion, dstId,
+        dstVersion, Utils.EmptyByteArr));
+    LOG.exiting("TopologyUpdateWaitHandler", "onNext", qualifiedName);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TreeTopology.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TreeTopology.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TreeTopology.java
new file mode 100644
index 0000000..b1695a9
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/TreeTopology.java
@@ -0,0 +1,345 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.api.operators.GroupCommOperator;
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.network.group.api.config.OperatorSpec;
+import org.apache.reef.io.network.group.api.driver.TaskNode;
+import org.apache.reef.io.network.group.api.driver.Topology;
+import org.apache.reef.io.network.group.impl.GroupChangesCodec;
+import org.apache.reef.io.network.group.impl.GroupChangesImpl;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.BroadcastOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.ReduceOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.parameters.DataCodec;
+import org.apache.reef.io.network.group.impl.config.parameters.ReduceFunctionParam;
+import org.apache.reef.io.network.group.impl.config.parameters.TaskVersion;
+import org.apache.reef.io.network.group.impl.operators.BroadcastReceiver;
+import org.apache.reef.io.network.group.impl.operators.BroadcastSender;
+import org.apache.reef.io.network.group.impl.operators.ReduceReceiver;
+import org.apache.reef.io.network.group.impl.operators.ReduceSender;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.formats.AvroConfigurationSerializer;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.wake.EStage;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.impl.SingleThreadStage;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.logging.Logger;
+
+/**
+ * Implements a tree topology with the specified Fan Out
+ */
+public class TreeTopology implements Topology {
+
+  private static final Logger LOG = Logger.getLogger(TreeTopology.class.getName());
+
+  private final EStage<GroupCommunicationMessage> senderStage;
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String driverId;
+  private String rootId;
+  private OperatorSpec operatorSpec;
+
+  private TaskNode root;
+  private TaskNode logicalRoot;
+  private TaskNode prev;
+  private final int fanOut;
+
+  private final ConcurrentMap<String, TaskNode> nodes = new ConcurrentSkipListMap<>();
+  private final ConfigurationSerializer confSer = new AvroConfigurationSerializer();
+
+
+  public TreeTopology(final EStage<GroupCommunicationMessage> senderStage,
+                      final Class<? extends Name<String>> groupName, final Class<? extends Name<String>> operatorName,
+                      final String driverId, final int numberOfTasks, final int fanOut) {
+    this.senderStage = senderStage;
+    this.groupName = groupName;
+    this.operName = operatorName;
+    this.driverId = driverId;
+    this.fanOut = fanOut;
+    LOG.config(getQualifiedName() + "Tree Topology running with a fan-out of " + fanOut);
+  }
+
+  @Override
+  public void setRootTask(final String rootId) {
+    LOG.entering("TreeTopology", "setRootTask", new Object[]{getQualifiedName(), rootId});
+    this.rootId = rootId;
+    LOG.exiting("TreeTopology", "setRootTask", getQualifiedName() + rootId);
+  }
+
+  @Override
+  public String getRootId() {
+    LOG.entering("TreeTopology", "getRootId", getQualifiedName());
+    LOG.exiting("TreeTopology", "getRootId", getQualifiedName() + rootId);
+    return rootId;
+  }
+
+  @Override
+  public void setOperatorSpecification(final OperatorSpec spec) {
+    LOG.entering("TreeTopology", "setOperSpec", new Object[]{getQualifiedName(), spec});
+    this.operatorSpec = spec;
+    LOG.exiting("TreeTopology", "setOperSpec", getQualifiedName() + spec);
+  }
+
+  @Override
+  public Configuration getTaskConfiguration(final String taskId) {
+    LOG.entering("TreeTopology", "getTaskConfig", new Object[]{getQualifiedName(), taskId});
+    final TaskNode taskNode = nodes.get(taskId);
+    if (taskNode == null) {
+      throw new RuntimeException(getQualifiedName() + taskId + " does not exist");
+    }
+
+    final int version = getNodeVersion(taskId);
+    final JavaConfigurationBuilder jcb = Tang.Factory.getTang().newConfigurationBuilder();
+    jcb.bindNamedParameter(DataCodec.class, operatorSpec.getDataCodecClass());
+    jcb.bindNamedParameter(TaskVersion.class, Integer.toString(version));
+    if (operatorSpec instanceof BroadcastOperatorSpec) {
+      final BroadcastOperatorSpec broadcastOperatorSpec = (BroadcastOperatorSpec) operatorSpec;
+      if (taskId.equals(broadcastOperatorSpec.getSenderId())) {
+        jcb.bindImplementation(GroupCommOperator.class, BroadcastSender.class);
+      } else {
+        jcb.bindImplementation(GroupCommOperator.class, BroadcastReceiver.class);
+      }
+    } else if (operatorSpec instanceof ReduceOperatorSpec) {
+      final ReduceOperatorSpec reduceOperatorSpec = (ReduceOperatorSpec) operatorSpec;
+      jcb.bindNamedParameter(ReduceFunctionParam.class, reduceOperatorSpec.getRedFuncClass());
+      if (taskId.equals(reduceOperatorSpec.getReceiverId())) {
+        jcb.bindImplementation(GroupCommOperator.class, ReduceReceiver.class);
+      } else {
+        jcb.bindImplementation(GroupCommOperator.class, ReduceSender.class);
+      }
+    }
+    final Configuration retConf = jcb.build();
+    LOG.exiting("TreeTopology", "getTaskConfig", getQualifiedName() + confSer.toString(retConf));
+    return retConf;
+  }
+
+  @Override
+  public int getNodeVersion(final String taskId) {
+    LOG.entering("TreeTopology", "getNodeVersion", new Object[]{getQualifiedName(), taskId});
+    final TaskNode node = nodes.get(taskId);
+    if (node == null) {
+      throw new RuntimeException(getQualifiedName() + taskId + " is not available on the nodes map");
+    }
+    final int version = node.getVersion();
+    LOG.exiting("TreeTopology", "getNodeVersion", getQualifiedName() + " " + taskId + " " + version);
+    return version;
+  }
+
+  @Override
+  public void removeTask(final String taskId) {
+    LOG.entering("TreeTopology", "removeTask", new Object[]{getQualifiedName(), taskId});
+    if (!nodes.containsKey(taskId)) {
+      LOG.fine("Trying to remove a non-existent node in the task graph");
+      LOG.exiting("TreeTopology", "removeTask", getQualifiedName());
+      return;
+    }
+    if (taskId.equals(rootId)) {
+      unsetRootNode(taskId);
+    } else {
+      removeChild(taskId);
+    }
+    LOG.exiting("TreeTopology", "removeTask", getQualifiedName() + taskId);
+  }
+
+  @Override
+  public void addTask(final String taskId) {
+    LOG.entering("TreeTopology", "addTask", new Object[]{getQualifiedName(), taskId});
+    if (nodes.containsKey(taskId)) {
+      LOG.fine("Got a request to add a task that is already in the graph. " +
+          "We need to block this request till the delete finishes. ***CAUTION***");
+    }
+
+    if (taskId.equals(rootId)) {
+      setRootNode(taskId);
+    } else {
+      addChild(taskId);
+    }
+    prev = nodes.get(taskId);
+    LOG.exiting("TreeTopology", "addTask", getQualifiedName() + taskId);
+  }
+
+  private void addChild(final String taskId) {
+    LOG.entering("TreeTopology", "addChild", new Object[]{getQualifiedName(), taskId});
+    LOG.finest(getQualifiedName() + "Adding leaf " + taskId);
+    final TaskNode node = new TaskNodeImpl(senderStage, groupName, operName, taskId, driverId, false);
+    if (logicalRoot != null) {
+      addTaskNode(node);
+    }
+    nodes.put(taskId, node);
+    LOG.exiting("TreeTopology", "addChild", getQualifiedName() + taskId);
+  }
+
+  private void addTaskNode(final TaskNode node) {
+    LOG.entering("TreeTopology", "addTaskNode", new Object[]{getQualifiedName(), node});
+    if (logicalRoot.getNumberOfChildren() >= this.fanOut) {
+      logicalRoot = logicalRoot.successor();
+    }
+    node.setParent(logicalRoot);
+    logicalRoot.addChild(node);
+    prev.setSibling(node);
+    LOG.exiting("TreeTopology", "addTaskNode", getQualifiedName() + node);
+  }
+
+  private void removeChild(final String taskId) {
+    LOG.entering("TreeTopology", "removeChild", new Object[]{getQualifiedName(), taskId});
+    if (root != null) {
+      root.removeChild(nodes.get(taskId));
+    }
+    nodes.remove(taskId);
+    LOG.exiting("TreeTopology", "removeChild", getQualifiedName() + taskId);
+  }
+
+  private void setRootNode(final String rootId) {
+    LOG.entering("TreeTopology", "setRootNode", new Object[]{getQualifiedName(), rootId});
+    final TaskNode node = new TaskNodeImpl(senderStage, groupName, operName, rootId, driverId, true);
+    this.root = node;
+    this.logicalRoot = this.root;
+    this.prev = this.root;
+
+    for (final Map.Entry<String, TaskNode> nodeEntry : nodes.entrySet()) {
+      final TaskNode leaf = nodeEntry.getValue();
+      addTaskNode(leaf);
+      this.prev = leaf;
+    }
+    nodes.put(rootId, root);
+    LOG.exiting("TreeTopology", "setRootNode", getQualifiedName() + rootId);
+  }
+
+  private void unsetRootNode(final String taskId) {
+    LOG.entering("TreeTopology", "unsetRootNode", new Object[]{getQualifiedName(), taskId});
+    nodes.remove(rootId);
+
+    for (final Map.Entry<String, TaskNode> nodeEntry : nodes.entrySet()) {
+      final String id = nodeEntry.getKey();
+      final TaskNode leaf = nodeEntry.getValue();
+      leaf.setParent(null);
+    }
+    LOG.exiting("TreeTopology", "unsetRootNode", getQualifiedName() + taskId);
+  }
+
+  @Override
+  public void onFailedTask(final String taskId) {
+    LOG.entering("TreeTopology", "onFailedTask", new Object[]{getQualifiedName(), taskId});
+    final TaskNode taskNode = nodes.get(taskId);
+    if (taskNode == null) {
+      throw new RuntimeException(getQualifiedName() + taskId + " does not exist");
+    }
+    taskNode.onFailedTask();
+    LOG.exiting("TreeTopology", "onFailedTask", getQualifiedName() + taskId);
+  }
+
+  @Override
+  public void onRunningTask(final String taskId) {
+    LOG.entering("TreeTopology", "onRunningTask", new Object[]{getQualifiedName(), taskId});
+    final TaskNode taskNode = nodes.get(taskId);
+    if (taskNode == null) {
+      throw new RuntimeException(getQualifiedName() + taskId + " does not exist");
+    }
+    taskNode.onRunningTask();
+    LOG.exiting("TreeTopology", "onRunningTask", getQualifiedName() + taskId);
+  }
+
+  @Override
+  public void onReceiptOfMessage(final GroupCommunicationMessage msg) {
+    LOG.entering("TreeTopology", "onReceiptOfMessage", new Object[]{getQualifiedName(), msg});
+    switch (msg.getType()) {
+      case TopologyChanges:
+        onTopologyChanges(msg);
+        break;
+      case UpdateTopology:
+        onUpdateTopology(msg);
+        break;
+
+      default:
+        nodes.get(msg.getSrcid()).onReceiptOfAcknowledgement(msg);
+        break;
+    }
+    LOG.exiting("TreeTopology", "onReceiptOfMessage", getQualifiedName() + msg);
+  }
+
+  private void onUpdateTopology(final GroupCommunicationMessage msg) {
+    LOG.entering("TreeTopology", "onUpdateTopology", new Object[]{getQualifiedName(), msg});
+    LOG.fine(getQualifiedName() + "Update affected parts of Topology");
+    final String dstId = msg.getSrcid();
+    final int version = getNodeVersion(dstId);
+
+    LOG.finest(getQualifiedName() + "Creating NodeTopologyUpdateWaitStage to wait on nodes to be updated");
+    final EventHandler<List<TaskNode>> topoUpdateWaitHandler = new TopologyUpdateWaitHandler(senderStage, groupName,
+        operName, driverId, 0,
+        dstId, version,
+        getQualifiedName());
+    final EStage<List<TaskNode>> nodeTopologyUpdateWaitStage = new SingleThreadStage<>("NodeTopologyUpdateWaitStage",
+        topoUpdateWaitHandler,
+        nodes.size());
+
+    final List<TaskNode> toBeUpdatedNodes = new ArrayList<>(nodes.size());
+    LOG.finest(getQualifiedName() + "Checking which nodes need to be updated");
+    for (final TaskNode node : nodes.values()) {
+      if (node.isRunning() && node.hasChanges() && node.resetTopologySetupSent()) {
+        toBeUpdatedNodes.add(node);
+      }
+    }
+    for (final TaskNode node : toBeUpdatedNodes) {
+      node.updatingTopology();
+      LOG.fine(getQualifiedName() + "Asking " + node + " to UpdateTopology");
+      senderStage.onNext(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.UpdateTopology, driverId, 0, node.getTaskId(),
+          node.getVersion(), Utils.EmptyByteArr));
+    }
+    nodeTopologyUpdateWaitStage.onNext(toBeUpdatedNodes);
+    LOG.exiting("TreeTopology", "onUpdateTopology", getQualifiedName() + msg);
+  }
+
+  private void onTopologyChanges(final GroupCommunicationMessage msg) {
+    LOG.entering("TreeTopology", "onTopologyChanges", new Object[]{getQualifiedName(), msg});
+    LOG.fine(getQualifiedName() + "Check TopologyChanges");
+    final String dstId = msg.getSrcid();
+    boolean hasTopologyChanged = false;
+    LOG.finest(getQualifiedName() + "Checking which nodes need to be updated");
+    for (final TaskNode node : nodes.values()) {
+      if (!node.isRunning() || node.hasChanges()) {
+        hasTopologyChanged = true;
+        break;
+      }
+    }
+    final GroupChanges changes = new GroupChangesImpl(hasTopologyChanged);
+    final Codec<GroupChanges> changesCodec = new GroupChangesCodec();
+    LOG.fine(getQualifiedName() + "TopologyChanges: " + changes);
+    senderStage.onNext(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyChanges, driverId, 0, dstId, getNodeVersion(dstId),
+        changesCodec.encode(changes)));
+    LOG.exiting("TreeTopology", "onTopologyChanges", getQualifiedName() + msg);
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + " - ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/package-info.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/package-info.java
new file mode 100644
index 0000000..82424ce
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/package-info.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.
+ */
+
+/**
+ * This package contains the implementation of the driver side of the
+ * Group Communication Service using the tree/flat topology. The Service
+ * can be configured with many named Communication Groups and each
+ * Communication Group can be configured with many named operators. This
+ * configuration is typically done on the GroupCommDriver object injected
+ * into the driver. During this specification only the root nodes are
+ * specified.
+ *
+ * After the GroupCommDriver is configured the driver would want to submit
+ * tasks with the Communication Groups & their operators configured. To do
+ * that the user has to create a partial task configuration containing his
+ * set of configurations and add it to the relevant communication group.
+ * Based on whether the given task is a Master/Slave different roles for
+ * the operators are configured. Once added, the final Configuration containing
+ * operators and their roles encoded can be obtained by the
+ * CommunicationGroupDriver.getConfiguration() call. The topology is complete
+ * once the minimum number of tasks needed for the group to function have been
+ * added.
+ *
+ * The initial configuration dished out by the service
+ * creates a bunch of tasks that are not connected. Connections are
+ * established when the Tasks start running. Each operator defines its own
+ * topology and can have potentially different root nodes. Each node in the
+ * topology called a TaskNode is a logical representation of a running Task.
+ * Adding a task to a group creates its TaskNode with TaskState NOT_STARTED.
+ *
+ * The driver side of the service plays a major role in setting up the
+ * topology and making sure that topology is set-up only when the parties
+ * involved are ready to participate in the communication. A topology will
+ * not contain parties that are not active. Active status is given to parties
+ * who have acknowledged the presence of their neighbors and who have been
+ * acknowledged by their neighbors as present. The connection between two
+ * parties is initiated by the driver and the driver then expects the parties
+ * to ACK that their end of the connection has been set-up. Once a party has
+ * ACK its end of all connections and all its neighbors ACK the outgoing part
+ * of their connection to this party the driver sends a TopologySetup msg to
+ * indicate that the topology is usable by this party now. The driver also
+ * listens in on failure events and appropriately updates its state so that
+ * it does not wait for ACKs from failed tasks.
+ *
+ * There are two chains of control:
+ * 1. Driver Events (Running/Failed Tasks)
+ * 2. Tasks (Msgs sent by Task side of Group Communication Service)
+ *
+ * All events and msgs are funneled through the CommunicationGroupDriver so
+ * that all the topologies belonging to different operators configured on the
+ * group are in sync. Without this there is possibility of a deadlock between
+ * the tasks and the driver. So there is no finer level locking other than that
+ * in the CommunicationGroupDriver.
+ *
+ * 1. Driver Events
+ *  These are routed to all communication groups and each communication group
+ *  routes it to all topologies. The topology will then route this event to the
+ *  corresponding TaskNode which will process that event. When a task starts
+ *  running it is notified of its running neighbors and the running neighbors
+ *  are notified of its running. The TaskNodeStatus object keeps track of the
+ *  state of the local topology for this TaskNode. What msgs have been sent to
+ *  this node that need to be ACKed, the status of its neighbors and whether
+ *  this TaskNode is ready to accept data from a neighboring TaskNode when we
+ *  ask the neighbor to check if he was only waiting for this TaskNode to ACK
+ *  in order to send TopologySetup. So when we are sending (Parent|Child)(Add|Dead)
+ *  msgs we first note that we expect an ACK back for this. These ACK expectations
+ *  are then deleted if the node fails. Neighbor failures are also updated.
+ *  All the msg sending is done by the TaskNode. The TaskNodeStatus is only a
+ *  state manager to consult on the status of ACKs and neighbors. This is needed
+ *  by the chkAndSendTopSetup logic. These events also send msgs related to failure
+ *  of tasks so that any task in the toplogy that waited for a response from the
+ *  failed task can move on.
+ *
+ * 2. Tasks
+ *  We get ACK msgs from tasks and they update the status of ACK expectations.
+ *  Here the TaskNodeStatus acts as a bridge between the initiation of a link
+ *  between two parties and the final set-up of the link. Once all ACKs have
+ *  been received we ask the TaskNode to check if it is ready to send a
+ *  TopologySetup msg. Every ACK can also trigger the chkAndSendTopSetup for
+ *  a neighbor.
+ *
+ * The above concerns the topology set-up and fault notiifcations. However, the
+ * other major task that the driver helps with is in updating a topology. When
+ * requested for the update of a Topology using the UpdateTopology msg, the
+ * driver notifies all the parties that have to update their topologies by
+ * sending an UpdateTopology msg to the affected parties. The tasks then try to
+ * enter the UpdateToplogy phase and as soon as they can do(by acquiring a lock)
+ * they respond that they have done so. The driver will wait till all the affected
+ * parties do so and then sends the initiator a msg that Topology has been updated.
+ * The effect of this is that the unaffected regions of the topology can continue
+ * to work normally while the affected regions are healing. The affected regions
+ * heal their (local)topologies using the TopologySetup mechanism described above.
+ * Both update topology and inital set-up use the minimum number of tasks to define
+ * when the set-up is complete.
+ *
+ * The CommGroupDriver class also takes care of minor alterations to event ordering
+ * by the use of locks to coerce the events to occur in a definite way.
+ *
+ */
+package org.apache.reef.io.network.group.impl.driver;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
new file mode 100644
index 0000000..b8dd425
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastReceiver.java
@@ -0,0 +1,159 @@
+/**
+ * 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.reef.io.network.group.impl.operators;
+
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.api.task.CommGroupNetworkHandler;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupServiceClient;
+import org.apache.reef.io.network.group.api.task.OperatorTopology;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.parameters.CommunicationGroupName;
+import org.apache.reef.io.network.group.impl.config.parameters.DataCodec;
+import org.apache.reef.io.network.group.impl.config.parameters.OperatorName;
+import org.apache.reef.io.network.group.impl.config.parameters.TaskVersion;
+import org.apache.reef.io.network.group.impl.task.OperatorTopologyImpl;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+public class BroadcastReceiver<T> implements Broadcast.Receiver<T>, EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(BroadcastReceiver.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final CommGroupNetworkHandler commGroupNetworkHandler;
+  private final Codec<T> dataCodec;
+  private final NetworkService<GroupCommunicationMessage> netService;
+  private final Sender sender;
+
+  private final OperatorTopology topology;
+
+  private final AtomicBoolean init = new AtomicBoolean(false);
+
+  private final CommunicationGroupServiceClient commGroupClient;
+
+  private final int version;
+
+  @Inject
+  public BroadcastReceiver(@Parameter(CommunicationGroupName.class) final String groupName,
+                           @Parameter(OperatorName.class) final String operName,
+                           @Parameter(TaskConfigurationOptions.Identifier.class) final String selfId,
+                           @Parameter(DataCodec.class) final Codec<T> dataCodec,
+                           @Parameter(DriverIdentifier.class) final String driverId,
+                           @Parameter(TaskVersion.class) final int version,
+                           final CommGroupNetworkHandler commGroupNetworkHandler,
+                           final NetworkService<GroupCommunicationMessage> netService,
+                           final CommunicationGroupServiceClient commGroupClient) {
+    super();
+    this.version = version;
+    LOG.finest(operName + " has CommGroupHandler-" + commGroupNetworkHandler.toString());
+    this.groupName = Utils.getClass(groupName);
+    this.operName = Utils.getClass(operName);
+    this.dataCodec = dataCodec;
+    this.commGroupNetworkHandler = commGroupNetworkHandler;
+    this.netService = netService;
+    this.sender = new Sender(this.netService);
+    this.topology = new OperatorTopologyImpl(this.groupName, this.operName, selfId, driverId, sender, version);
+    this.commGroupNetworkHandler.register(this.operName, this);
+    this.commGroupClient = commGroupClient;
+  }
+
+  @Override
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void initialize() throws ParentDeadException {
+    topology.initialize();
+  }
+
+  @Override
+  public Class<? extends Name<String>> getOperName() {
+    return operName;
+  }
+
+  @Override
+  public Class<? extends Name<String>> getGroupName() {
+    return groupName;
+  }
+
+  @Override
+  public String toString() {
+    return "BroadcastReceiver:" + Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":" + version;
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    topology.handle(msg);
+  }
+
+  @Override
+  public T receive() throws NetworkException, InterruptedException {
+    LOG.entering("BroadcastReceiver", "receive", this);
+    LOG.fine("I am " + this);
+
+    if (init.compareAndSet(false, true)) {
+      LOG.fine(this + " Communication group initializing");
+      commGroupClient.initialize();
+      LOG.fine(this + " Communication group initialized");
+    }
+    // I am an intermediate node or leaf.
+
+    final T retVal;
+    // Wait for parent to send
+    LOG.fine(this + " Waiting to receive broadcast");
+    byte[] data;
+    try {
+      data = topology.recvFromParent();
+      // TODO: Should receive the identity element instead of null
+      if (data == null) {
+        LOG.fine(this + " Received null. Perhaps one of my ancestors is dead.");
+        retVal = null;
+      } else {
+        LOG.finest("Using " + dataCodec.getClass().getSimpleName() + " as codec");
+        retVal = dataCodec.decode(data);
+        LOG.finest("Decoded msg successfully");
+        LOG.fine(this + " Received: " + retVal);
+        LOG.finest(this + " Sending to children.");
+      }
+
+      topology.sendToChildren(data, ReefNetworkGroupCommProtos.GroupCommMessage.Type.Broadcast);
+    } catch (final ParentDeadException e) {
+      throw new RuntimeException("ParentDeadException", e);
+    }
+    LOG.exiting("BroadcastReceiver", "receive", Arrays.toString(new Object[]{retVal, this}));
+    return retVal;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
new file mode 100644
index 0000000..6f146e6
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/operators/BroadcastSender.java
@@ -0,0 +1,141 @@
+/**
+ * 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.reef.io.network.group.impl.operators;
+
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.api.task.CommGroupNetworkHandler;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupServiceClient;
+import org.apache.reef.io.network.group.api.task.OperatorTopology;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.parameters.CommunicationGroupName;
+import org.apache.reef.io.network.group.impl.config.parameters.DataCodec;
+import org.apache.reef.io.network.group.impl.config.parameters.OperatorName;
+import org.apache.reef.io.network.group.impl.config.parameters.TaskVersion;
+import org.apache.reef.io.network.group.impl.task.OperatorTopologyImpl;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+public class BroadcastSender<T> implements Broadcast.Sender<T>, EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(BroadcastSender.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final CommGroupNetworkHandler commGroupNetworkHandler;
+  private final Codec<T> dataCodec;
+  private final NetworkService<GroupCommunicationMessage> netService;
+  private final Sender sender;
+
+  private final OperatorTopology topology;
+
+  private final AtomicBoolean init = new AtomicBoolean(false);
+
+  private final CommunicationGroupServiceClient commGroupClient;
+
+  private final int version;
+
+  @Inject
+  public BroadcastSender(@Parameter(CommunicationGroupName.class) final String groupName,
+                         @Parameter(OperatorName.class) final String operName,
+                         @Parameter(TaskConfigurationOptions.Identifier.class) final String selfId,
+                         @Parameter(DataCodec.class) final Codec<T> dataCodec,
+                         @Parameter(DriverIdentifier.class) final String driverId,
+                         @Parameter(TaskVersion.class) final int version,
+                         final CommGroupNetworkHandler commGroupNetworkHandler,
+                         final NetworkService<GroupCommunicationMessage> netService,
+                         final CommunicationGroupServiceClient commGroupClient) {
+    super();
+    this.version = version;
+    LOG.finest(operName + "has CommGroupHandler-" + commGroupNetworkHandler.toString());
+    this.groupName = Utils.getClass(groupName);
+    this.operName = Utils.getClass(operName);
+    this.dataCodec = dataCodec;
+    this.commGroupNetworkHandler = commGroupNetworkHandler;
+    this.netService = netService;
+    this.sender = new Sender(this.netService);
+    this.topology = new OperatorTopologyImpl(this.groupName, this.operName, selfId, driverId, sender, version);
+    this.commGroupNetworkHandler.register(this.operName, this);
+    this.commGroupClient = commGroupClient;
+  }
+
+  @Override
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void initialize() throws ParentDeadException {
+    topology.initialize();
+  }
+
+  @Override
+  public Class<? extends Name<String>> getOperName() {
+    return operName;
+  }
+
+  @Override
+  public Class<? extends Name<String>> getGroupName() {
+    return groupName;
+  }
+
+  @Override
+  public String toString() {
+    return "BroadcastSender:" + Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + ":" + version;
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    topology.handle(msg);
+  }
+
+  @Override
+  public void send(final T element) throws NetworkException, InterruptedException {
+    LOG.entering("BroadcastSender", "send", new Object[]{this, element});
+    LOG.fine("I am " + this);
+
+    if (init.compareAndSet(false, true)) {
+      LOG.fine(this + " Communication group initializing");
+      commGroupClient.initialize();
+      LOG.fine(this + " Communication group initialized");
+    }
+
+    try {
+      LOG.fine(this + " Broadcasting " + element);
+      topology.sendToChildren(dataCodec.encode(element), ReefNetworkGroupCommProtos.GroupCommMessage.Type.Broadcast);
+    } catch (final ParentDeadException e) {
+      throw new RuntimeException("ParentDeadException", e);
+    }
+    LOG.exiting("BroadcastSender", "send", Arrays.toString(new Object[]{this, element}));
+  }
+
+}


[6/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/SlaveTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/SlaveTask.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/SlaveTask.java
new file mode 100644
index 0000000..dde0bc3
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/SlaveTask.java
@@ -0,0 +1,76 @@
+/**
+ * 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.reef.examples.group.broadcast;
+
+import org.apache.reef.examples.group.bgd.operatornames.ControlMessageBroadcaster;
+import org.apache.reef.examples.group.bgd.parameters.AllCommunicationGroup;
+import org.apache.reef.examples.group.broadcast.parameters.ModelBroadcaster;
+import org.apache.reef.examples.group.broadcast.parameters.ModelReceiveAckReducer;
+import org.apache.reef.examples.group.utils.math.Vector;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupClient;
+import org.apache.reef.io.network.group.api.task.GroupCommClient;
+import org.apache.reef.task.Task;
+
+import javax.inject.Inject;
+
+/**
+ *
+ */
+public class SlaveTask implements Task {
+  private final CommunicationGroupClient communicationGroupClient;
+  private final Broadcast.Receiver<ControlMessages> controlMessageBroadcaster;
+  private final Broadcast.Receiver<Vector> modelBroadcaster;
+  private final Reduce.Sender<Boolean> modelReceiveAckReducer;
+
+  @Inject
+  public SlaveTask(
+      final GroupCommClient groupCommClient) {
+    this.communicationGroupClient = groupCommClient.getCommunicationGroup(AllCommunicationGroup.class);
+    this.controlMessageBroadcaster = communicationGroupClient.getBroadcastReceiver(ControlMessageBroadcaster.class);
+    this.modelBroadcaster = communicationGroupClient.getBroadcastReceiver(ModelBroadcaster.class);
+    this.modelReceiveAckReducer = communicationGroupClient.getReduceSender(ModelReceiveAckReducer.class);
+  }
+
+  @Override
+  public byte[] call(final byte[] memento) throws Exception {
+    boolean stop = false;
+    while (!stop) {
+      final ControlMessages controlMessage = controlMessageBroadcaster.receive();
+      switch (controlMessage) {
+        case Stop:
+          stop = true;
+          break;
+
+        case ReceiveModel:
+          modelBroadcaster.receive();
+          if (Math.random() < 0.1) {
+            throw new RuntimeException("Simulated Failure");
+          }
+          modelReceiveAckReducer.send(true);
+          break;
+
+        default:
+          break;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/AllCommunicationGroup.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/AllCommunicationGroup.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/AllCommunicationGroup.java
new file mode 100644
index 0000000..9e8781e
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/AllCommunicationGroup.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+@NamedParameter()
+public final class AllCommunicationGroup implements Name<String> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ControlMessageBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ControlMessageBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ControlMessageBroadcaster.java
new file mode 100644
index 0000000..4e7607a
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ControlMessageBroadcaster.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter()
+public final class ControlMessageBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/Dimensions.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/Dimensions.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/Dimensions.java
new file mode 100644
index 0000000..e4814c6
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/Dimensions.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+@NamedParameter(doc = "Model dimensions", short_name = "dim")
+public class Dimensions implements Name<Integer> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/FailureProbability.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/FailureProbability.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/FailureProbability.java
new file mode 100644
index 0000000..911ad25
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/FailureProbability.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+@NamedParameter(doc = "Prob(failure)", default_value = "0.1")
+public class FailureProbability implements Name<Double> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelBroadcaster.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelBroadcaster.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelBroadcaster.java
new file mode 100644
index 0000000..0c673bf
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelBroadcaster.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter()
+public final class ModelBroadcaster implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelReceiveAckReducer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelReceiveAckReducer.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelReceiveAckReducer.java
new file mode 100644
index 0000000..635eeb7
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/ModelReceiveAckReducer.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter()
+public final class ModelReceiveAckReducer implements Name<String> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/NumberOfReceivers.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/NumberOfReceivers.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/NumberOfReceivers.java
new file mode 100644
index 0000000..6c7983e
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/broadcast/parameters/NumberOfReceivers.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.reef.examples.group.broadcast.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ *
+ */
+@NamedParameter(doc = "The number of receivers for the operators", short_name = "receivers")
+public class NumberOfReceivers implements Name<Integer> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractImmutableVector.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractImmutableVector.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractImmutableVector.java
new file mode 100644
index 0000000..9fa8498
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractImmutableVector.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.reef.examples.group.utils.math;
+
+
+import org.apache.reef.io.Tuple;
+
+import java.util.Formatter;
+import java.util.Locale;
+
+/**
+ * Base class for implementing ImmutableVector
+ */
+abstract class AbstractImmutableVector implements ImmutableVector {
+
+  @Override
+  public abstract double get(int i);
+
+  @Override
+  public abstract int size();
+
+  @Override
+  public double dot(final Vector that) {
+    assert (this.size() == that.size());
+
+    double result = 0.0;
+    for (int index = 0; index < this.size(); ++index) {
+      result += this.get(index) * that.get(index);
+    }
+    return result;
+  }
+
+
+  @Override
+  public double sum() {
+    double result = 0.0;
+    for (int i = 0; i < this.size(); ++i) {
+      result += this.get(i);
+    }
+    return result;
+  }
+
+  @Override
+  public double norm2() {
+    return Math.sqrt(dot((Vector) this));
+  }
+
+  @Override
+  public double norm2Sqr() {
+    return dot((Vector) this);
+  }
+
+  @SuppressWarnings("boxing")
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder("DenseVector(");
+    try (final Formatter formatter = new Formatter(b, Locale.US)) {
+      final int size = Math.min(25, this.size());
+      for (int i = 0; i < size; ++i) {
+        if (i < size - 1) {
+          formatter.format("%1.3f, ", this.get(i));
+        } else {
+          formatter.format("%1.3f ", this.get(i));
+        }
+      }
+      if (this.size() > 25) {
+        formatter.format("...");
+      }
+    }
+    b.append(')');
+    return b.toString();
+  }
+
+  @Override
+  public Tuple<Integer, Double> min() {
+    double min = get(0);
+    int minIdx = 0;
+    for (int i = 1; i < this.size(); ++i) {
+      final double curVal = get(i);
+      if (curVal < min) {
+        min = curVal;
+        minIdx = i;
+      }
+    }
+    return new Tuple<Integer, Double>(minIdx, min);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractVector.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractVector.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractVector.java
new file mode 100644
index 0000000..f2afcef
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/AbstractVector.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.reef.examples.group.utils.math;
+
+/**
+ * Abstract base class for {@link Vector} implementations.
+ * <p/>
+ * The only methods to be implemented by subclasses are get, set and size.
+ */
+public abstract class AbstractVector extends AbstractImmutableVector implements Vector {
+
+  @Override
+  public abstract void set(int i, double v);
+
+
+  @Override
+  public void add(final Vector that) {
+    for (int index = 0; index < this.size(); ++index) {
+      this.set(index, this.get(index) + that.get(index));
+    }
+  }
+
+  @Override
+  public void multAdd(final double factor, final ImmutableVector that) {
+    for (int index = 0; index < this.size(); ++index) {
+      this.set(index, this.get(index) + factor * that.get(index));
+    }
+  }
+
+  @Override
+  public void scale(final double factor) {
+    for (int index = 0; index < this.size(); ++index) {
+      this.set(index, this.get(index) * factor);
+    }
+  }
+
+
+  @Override
+  public void normalize() {
+    final double factor = 1.0 / this.norm2();
+    this.scale(factor);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/DenseVector.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/DenseVector.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/DenseVector.java
new file mode 100644
index 0000000..bc46521
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/DenseVector.java
@@ -0,0 +1,112 @@
+/**
+ * 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.reef.examples.group.utils.math;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Random;
+
+/**
+ * A dense {@link Vector} implementation backed by a double[]
+ */
+public class DenseVector extends AbstractVector implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+  private final double[] values;
+
+  /**
+   * Creates a dense vector of the given size
+   */
+  public DenseVector(final int size) {
+    this(new double[size]);
+  }
+
+  public DenseVector(final double[] values) {
+    this.values = values;
+  }
+
+  /**
+   * Instantiates a new DenseVector by copying the given other vector.
+   */
+  public DenseVector(final ImmutableVector other) {
+    final int size = other.size();
+    this.values = new double[size];
+    for (int i = 0; i < size; ++i) {
+      this.values[i] = other.get(i);
+    }
+  }
+
+  public DenseVector(final DenseVector other) {
+    this.values = Arrays.copyOf(other.values, other.values.length);
+  }
+
+  @Override
+  public void set(final int i, final double v) {
+    this.values[i] = v;
+  }
+
+  @Override
+  public double get(final int i) {
+    return this.values[i];
+  }
+
+  @Override
+  public int size() {
+    return this.values.length;
+  }
+
+  /**
+   * Access the underlying storage. This is unsafe.
+   */
+  public double[] getValues() {
+    return this.values;
+  }
+
+  /**
+   * Creates a random Vector of size 'size' where each element is individually
+   * drawn from Math.random()
+   *
+   * @return a random Vector of the given size where each element is
+   * individually drawn from Math.random()
+   */
+  public static DenseVector rand(final int size) {
+    return rand(size, new Random());
+  }
+
+  /**
+   * Creates a random Vector of size 'size' where each element is individually
+   * drawn from Math.random()
+   *
+   * @param random the random number generator to use.
+   * @return a random Vector of the given size where each element is
+   * individually drawn from Math.random()
+   */
+  public static DenseVector rand(final int size, final Random random) {
+    final DenseVector vec = new DenseVector(size);
+    for (int i = 0; i < size; ++i) {
+      vec.values[i] = random.nextDouble();
+    }
+    return vec;
+  }
+
+  @Override
+  public Vector newInstance() {
+    return new DenseVector(size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/ImmutableVector.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/ImmutableVector.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/ImmutableVector.java
new file mode 100644
index 0000000..cfd2e8d
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/ImmutableVector.java
@@ -0,0 +1,78 @@
+/**
+ * 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.reef.examples.group.utils.math;
+
+
+import org.apache.reef.io.Tuple;
+
+/**
+ * Represents an immutable vector.
+ */
+public interface ImmutableVector {
+  /**
+   * Access the value of the Vector at dimension i
+   *
+   * @param i index
+   * @return the value at index i
+   */
+  double get(int i);
+
+  /**
+   * The size (dimensionality) of the Vector
+   *
+   * @return the size of the Vector.
+   */
+  int size();
+
+  /**
+   * Computes the inner product with another Vector.
+   *
+   * @param that
+   * @return the inner product between two Vectors.
+   */
+  double dot(Vector that);
+
+  /**
+   * Computes the computeSum of all entries in the Vector.
+   *
+   * @return the computeSum of all entries in this Vector
+   */
+  double sum();
+
+  /**
+   * Computes the L2 norm of this Vector.
+   *
+   * @return the L2 norm of this Vector.
+   */
+  double norm2();
+
+  /**
+   * Computes the square of the L2 norm of this Vector.
+   *
+   * @return the square of the L2 norm of this Vector.
+   */
+  double norm2Sqr();
+
+  /**
+   * Computes the min of all entries in the Vector
+   *
+   * @return the min of all entries in this Vector
+   */
+  Tuple<Integer, Double> min();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/SparseVector.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/SparseVector.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/SparseVector.java
new file mode 100644
index 0000000..48a3b21
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/SparseVector.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.group.utils.math;
+
+
+/**
+ * A sparse vector represented by an index and value array.
+ */
+public final class SparseVector extends AbstractImmutableVector {
+
+  private final double[] values;
+  private final int[] indices;
+  private final int size;
+
+
+  public SparseVector(final double[] values, final int[] indices, final int size) {
+    this.values = values;
+    this.indices = indices;
+    this.size = size;
+  }
+
+  public SparseVector(final double[] values, final int[] indices) {
+    this(values, indices, -1);
+  }
+
+
+  @Override
+  public double get(final int index) {
+    for (int i = 0; i < indices.length; ++i) {
+      if (indices[i] == index) {
+        return values[i];
+      }
+    }
+    return 0;
+  }
+
+  @Override
+  public int size() {
+    return this.size;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Vector.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Vector.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Vector.java
new file mode 100644
index 0000000..021a666
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Vector.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.reef.examples.group.utils.math;
+
+import java.io.Serializable;
+
+/**
+ * An interface for Linear Alebra Vectors.
+ */
+public interface Vector extends ImmutableVector, Serializable {
+
+  /**
+   * Set dimension i of the Vector to value v
+   *
+   * @param i the index
+   * @param v value
+   */
+  public void set(final int i, final double v);
+
+  /**
+   * Adds the Vector that to this one in place: this += that.
+   *
+   * @param that
+   */
+  public void add(final Vector that);
+
+  /**
+   * this += factor * that.
+   *
+   * @param factor
+   * @param that
+   */
+  public void multAdd(final double factor, final ImmutableVector that);
+
+  /**
+   * Scales this Vector: this *= factor.
+   *
+   * @param factor the scaling factor.
+   */
+  public void scale(final double factor);
+
+
+  /**
+   * Normalizes the Vector according to the L2 norm.
+   */
+  public void normalize();
+
+  /**
+   * Create a new instance of the current type
+   * with elements being zero
+   *
+   * @return zero vector of current dimensionality
+   */
+  public Vector newInstance();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/VectorCodec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/VectorCodec.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/VectorCodec.java
new file mode 100644
index 0000000..e7ad7e9
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/VectorCodec.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.reef.examples.group.utils.math;
+
+import org.apache.reef.wake.remote.Codec;
+
+import javax.inject.Inject;
+import java.io.*;
+
+/**
+ * Codec for the Vector type Uses Data*Stream
+ *
+ * @author shravan
+ */
+public class VectorCodec implements Codec<Vector> {
+  /**
+   * This class is instantiated by TANG
+   */
+  @Inject
+  public VectorCodec() {
+    // Intentionally blank
+  }
+
+  @Override
+  public Vector decode(byte[] data) {
+    ByteArrayInputStream bais = new ByteArrayInputStream(data);
+    Vector result;
+    try (DataInputStream dais = new DataInputStream(bais)) {
+      int size = dais.readInt();
+      result = new DenseVector(size);
+      for (int i = 0; i < size; i++)
+        result.set(i, dais.readDouble());
+    } catch (IOException e) {
+      throw new RuntimeException(e.getCause());
+    }
+    return result;
+  }
+
+  @Override
+  public byte[] encode(Vector vec) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(vec.size()
+        * (Double.SIZE / Byte.SIZE));
+    try (DataOutputStream daos = new DataOutputStream(baos)) {
+      daos.writeInt(vec.size());
+      for (int i = 0; i < vec.size(); i++) {
+        daos.writeDouble(vec.get(i));
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e.getCause());
+    }
+    return baos.toByteArray();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Window.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Window.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Window.java
new file mode 100644
index 0000000..659e118
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/math/Window.java
@@ -0,0 +1,76 @@
+/**
+ * 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.reef.examples.group.utils.math;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+// TODO: Document
+public class Window {
+
+  private final int maxSize;
+  private final List<Double> list;
+
+  public Window(int size) {
+    this.maxSize = size;
+    list = new ArrayList<>(size);
+  }
+
+  public void add(double d) {
+    if (list.size() < maxSize) {
+      list.add(d);
+      return;
+    }
+    list.remove(0);
+    list.add(d);
+  }
+
+  public double avg() {
+    if (list.size() == 0)
+      return 0;
+    double retVal = 0;
+    for (double d : list) {
+      retVal += d;
+    }
+    return retVal / list.size();
+  }
+
+  public double avgIfAdded(double d) {
+    if (list.isEmpty())
+      return d;
+    int start = (list.size() < maxSize) ? 0 : 1;
+    int numElems = (list.size() < maxSize) ? list.size() + 1 : maxSize;
+    for (int i = start; i < list.size(); i++)
+      d += list.get(i);
+    return d / numElems;
+  }
+
+  public static void main(String[] args) {
+    final Logger log = Logger.getLogger(Window.class.getName());
+    final Window w = new Window(3);
+    for (int i = 1; i < 10; i++) {
+      final double exp = w.avgIfAdded(i);
+      w.add(i);
+      final double act = w.avg();
+      log.log(Level.INFO, "OUT: Exp: {0} Act: {1}", new Object[] {exp, act});
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/timer/Timer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/timer/Timer.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/timer/Timer.java
new file mode 100644
index 0000000..729ddf6
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/utils/timer/Timer.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.reef.examples.group.utils.timer;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class Timer implements AutoCloseable {
+
+  private static final Logger LOG = Logger.getLogger(Timer.class.getName());
+
+  public static final int MINUTES = 60 * 1000;  // ms
+  public static final int HOURS = 60 * MINUTES;
+
+  private final Logger log;
+  private final Level level;
+  private final String description;
+  private final long timeStart;
+
+  public Timer(final Logger log, final String description) {
+    this(log, Level.INFO, description);
+  }
+
+  public Timer(final String description) {
+    this(LOG, Level.INFO, description);
+  }
+
+  public Timer(final Logger log, final Level level, final String description) {
+    this.log = log;
+    this.level = level;
+    this.description = description;
+    this.timeStart = System.currentTimeMillis();
+    this.log.log(this.level, "TIMER Start: {0}", this.description);
+  }
+
+  @Override
+  public void close() {
+    final long timeEnd = System.currentTimeMillis();
+    this.log.log(this.level, "TIMER End: {0} took {1} sec.",
+        new Object[]{this.description, (timeEnd - this.timeStart) / 1000.0});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/scheduler/Scheduler.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/scheduler/Scheduler.java
index 960f297..544f333 100644
--- a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/scheduler/Scheduler.java
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/scheduler/Scheduler.java
@@ -28,7 +28,10 @@ import org.apache.reef.tang.Tang;
 
 import javax.annotation.concurrent.ThreadSafe;
 import javax.inject.Inject;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Queue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicInteger;
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/BlockingEventHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/BlockingEventHandler.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/BlockingEventHandler.java
index e6a5d6c..eefdcde 100644
--- a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/BlockingEventHandler.java
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/BlockingEventHandler.java
@@ -33,8 +33,8 @@ import java.util.List;
 public final class BlockingEventHandler<T> implements EventHandler<T> {
 
   private final int expectedSize;
-  private final EventHandler<Iterable<T>> destination;
   private List<T> events = new ArrayList<>();
+  private final EventHandler<Iterable<T>> destination;
 
   public BlockingEventHandler(final int expectedSize, final EventHandler<Iterable<T>> destination) {
     this.expectedSize = expectedSize;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/LoggingEventHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/LoggingEventHandler.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/LoggingEventHandler.java
index 9d2d5b8..804e67c 100644
--- a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/LoggingEventHandler.java
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/utils/wake/LoggingEventHandler.java
@@ -31,28 +31,27 @@ import java.util.logging.Logger;
  */
 public class LoggingEventHandler<T> implements EventHandler<T> {
 
+  private static final Logger LOG = Logger.getLogger(LoggingEventHandler.class.getName());
+
   private final EventHandler<T> downstreamEventHandler;
-  private final String prefix;
-  private final String suffix;
+  private final String format;
 
   /**
-   * @param prefix                 to be logged before the event
    * @param downstreamEventHandler the event handler to hand the event to
-   * @param suffix                 to be logged after the event
+   * @param format                 Format string to log the event, e.g. "Event {0} received".
    */
-  public LoggingEventHandler(final String prefix, EventHandler<T> downstreamEventHandler, final String suffix) {
+  public LoggingEventHandler(EventHandler<T> downstreamEventHandler, final String format) {
     this.downstreamEventHandler = downstreamEventHandler;
-    this.prefix = prefix;
-    this.suffix = suffix;
+    this.format = format;
   }
 
   public LoggingEventHandler(final EventHandler<T> downstreamEventHandler) {
-    this("", downstreamEventHandler, "");
+    this(downstreamEventHandler, "{0}");
   }
 
   @Override
   public void onNext(final T value) {
-    Logger.getLogger(LoggingEventHandler.class.getName()).log(Level.INFO, prefix + value.toString() + suffix);
+    LOG.log(Level.INFO, this.format, value);
     this.downstreamEventHandler.onNext(value);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/pom.xml
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/pom.xml b/lang/java/reef-io/pom.xml
index 90bdf1d..b44cb77 100644
--- a/lang/java/reef-io/pom.xml
+++ b/lang/java/reef-io/pom.xml
@@ -45,6 +45,11 @@ under the License.
                                     <arg value="--java_out=target/generated-sources/proto"/>
                                     <arg value="src/main/proto/ns_protocol.proto"/>
                                 </exec>
+                                <exec executable="protoc">
+                                    <arg value="--proto_path=src/main/proto/"/>
+                                    <arg value="--java_out=target/generated-sources/proto"/>
+                                    <arg value="src/main/proto/group_comm_protocol.proto"/>
+                                </exec>
                             </tasks>
                             <sourceRoot>target/generated-sources/proto</sourceRoot>
                         </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/GroupChanges.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/GroupChanges.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/GroupChanges.java
new file mode 100644
index 0000000..1758646
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/GroupChanges.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.reef.io.network.group.api;
+
+import org.apache.reef.annotations.audience.TaskSide;
+
+/**
+ * Represents the changes in Topology that happened in a communication group
+ * from the last time the user asked for topology changes
+ */
+@TaskSide
+public interface GroupChanges {
+
+  boolean exist();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/config/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/config/OperatorSpec.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/config/OperatorSpec.java
new file mode 100644
index 0000000..eabd23c
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/config/OperatorSpec.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.reef.io.network.group.api.config;
+
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.io.serialization.Codec;
+
+/**
+ * The specification of an operator submitted by the user
+ * while configuring the communication group.
+ */
+@DriverSide
+@Private
+public interface OperatorSpec {
+
+  /**
+   * @return The codec class to be used to serialize & deserialize data
+   * in the group communication operators.
+   */
+  Class<? extends Codec> getDataCodecClass();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/CommunicationGroupDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/CommunicationGroupDriver.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/CommunicationGroupDriver.java
new file mode 100644
index 0000000..ca486a3
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/CommunicationGroupDriver.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.reef.io.network.group.api.driver;
+
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.io.network.group.api.task.GroupCommClient;
+import org.apache.reef.io.network.group.impl.config.BroadcastOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.ReduceOperatorSpec;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.annotations.Name;
+
+/**
+ * The driver side interface of a Communication Group
+ * Lets one add opertaors and tasks.
+ * Main function is to extract the configuration related
+ * to the Group Communication for a task in the comm group
+ */
+@DriverSide
+public interface CommunicationGroupDriver {
+
+  /**
+   * Add the broadcast operator specified by the
+   * 'spec' with name 'operatorName' into this
+   * Communication Group
+   *
+   * @param operatorName
+   * @param spec
+   * @return
+   */
+  public CommunicationGroupDriver addBroadcast(Class<? extends Name<String>> operatorName, BroadcastOperatorSpec spec);
+
+  /**
+   * Add the reduce operator specified by the
+   * 'spec' with name 'operatorName' into this
+   * Communication Group
+   *
+   * @param operatorName
+   * @param spec
+   * @return
+   */
+  public CommunicationGroupDriver addReduce(Class<? extends Name<String>> operatorName, ReduceOperatorSpec spec);
+
+  /**
+   * This signals to the service that no more
+   * operator specs will be added to this communication
+   * group and an attempt to do that will throw an
+   * IllegalStateException
+   */
+  public void finalise();
+
+  /**
+   * Returns a configuration that includes the partial task
+   * configuration passed in as 'taskConf' and makes the
+   * current communication group and the operators configured
+   * on it available on the Task side. Provides for injection
+   * of {@link GroupCommClient}
+   *
+   * @param taskConf
+   * @return
+   */
+  public Configuration getTaskConfiguration(Configuration taskConf);
+
+  /**
+   * Add the task represented by this configuration to this
+   * communication group. The configuration needs to contain
+   * the id of the Task that will be used
+   *
+   * @param partialTaskConf
+   */
+  public void addTask(Configuration partialTaskConf);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommDriver.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommDriver.java
new file mode 100644
index 0000000..fc7e919
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommDriver.java
@@ -0,0 +1,76 @@
+/**
+ * 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.reef.io.network.group.api.driver;
+
+import org.apache.reef.annotations.Provided;
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.io.network.group.impl.driver.GroupCommDriverImpl;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.annotations.Name;
+
+/**
+ * The driver side interface of Group Communication
+ * which is the entry point for the service
+ */
+@DriverSide
+@Provided
+@DefaultImplementation(value = GroupCommDriverImpl.class)
+public interface GroupCommDriver {
+
+  /**
+   * Create a new communication group with the specified name
+   * and the minimum number of tasks needed in this group before
+   * communication can start
+   *
+   * @param groupName
+   * @param numberOfTasks
+   * @return
+   */
+  CommunicationGroupDriver newCommunicationGroup(Class<? extends Name<String>> groupName, int numberOfTasks);
+
+  /**
+   * Tests whether the activeContext is a context configured
+   * using the Group Communication Service
+   *
+   * @param activeContext
+   * @return
+   */
+  boolean isConfigured(ActiveContext activeContext);
+
+  /**
+   * @return Configuration needed for a Context that should have
+   * Group Communication Service enabled
+   */
+  Configuration getContextConfiguration();
+
+  /**
+   * @return Configuration needed to enable
+   * Group Communication as a Service
+   */
+  Configuration getServiceConfiguration();
+
+  /**
+   * @return Configuration needed for a Task that should have
+   * Group Communication Service enabled
+   */
+  Configuration getTaskConfiguration(Configuration partialTaskConf);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommServiceDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommServiceDriver.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommServiceDriver.java
new file mode 100644
index 0000000..a097075
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/GroupCommServiceDriver.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.reef.io.network.group.api.driver;
+
+
+import org.apache.reef.annotations.Provided;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.io.network.group.impl.driver.GroupCommDriverImpl;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.wake.EStage;
+
+@Private
+@Provided
+@DefaultImplementation(value = GroupCommDriverImpl.class)
+public interface GroupCommServiceDriver extends GroupCommDriver {
+
+  /**
+   * Not user facing but used the Group Communication Service class
+   *
+   * @return The running task stage that will handle the RunningTask
+   * events
+   */
+  EStage<RunningTask> getGroupCommRunningTaskStage();
+
+  /**
+   * Not user facing but used the Group Communication Service class
+   *
+   * @return The running task stage that will handle the FailedTask
+   * events
+   */
+  EStage<FailedTask> getGroupCommFailedTaskStage();
+
+  /**
+   * Not user facing but used the Group Communication Service class
+   *
+   * @return The running task stage that will handle the FailedEvaluator
+   * events
+   */
+  EStage<FailedEvaluator> getGroupCommFailedEvaluatorStage();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNode.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNode.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNode.java
new file mode 100644
index 0000000..d16d4ea
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNode.java
@@ -0,0 +1,94 @@
+/**
+ * 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.reef.io.network.group.api.driver;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+
+/**
+ * A node in the Topology representing a Task on the driver
+ * Impl should maintain state relating to whether task is running/dead and
+ * status of neighboring nodes and send ctrl msgs to the tasks indicating
+ * topology changing events
+ */
+public interface TaskNode {
+
+  String getTaskId();
+
+  int getVersion();
+
+  int getNumberOfChildren();
+
+  TaskNode getParent();
+
+  void setParent(TaskNode parent);
+
+  void addChild(TaskNode child);
+
+  void removeChild(TaskNode taskNode);
+
+  boolean isRunning();
+
+  void onRunningTask();
+
+  void onFailedTask();
+
+  boolean hasChanges();
+
+  boolean isNeighborActive(String neighborId);
+
+  void onReceiptOfAcknowledgement(GroupCommunicationMessage msg);
+
+  void onParentRunning();
+
+  void onParentDead();
+
+  void onChildRunning(String childId);
+
+  void onChildDead(String childId);
+
+  /**
+   * Check if this node is ready for sending
+   * TopologySetup
+   */
+  void checkAndSendTopologySetupMessage();
+
+  /**
+   * Check if the neighbor node with id source
+   * is ready for sending TopologySetup
+   * @param source
+   */
+  void checkAndSendTopologySetupMessageFor(String source);
+
+  /**
+   * reset topology setup ensures that update topology is not sent to someone
+   * who is already updating topology which is usually when they are just
+   * (re)starting
+   *
+   * @return
+   */
+  boolean resetTopologySetupSent();
+
+  void waitForTopologySetupOrFailure();
+
+  void setSibling(TaskNode leaf);
+
+  TaskNode successor();
+
+  void updatingTopology ();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNodeStatus.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNodeStatus.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNodeStatus.java
new file mode 100644
index 0000000..93955e1
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/TaskNodeStatus.java
@@ -0,0 +1,81 @@
+/**
+ * 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.reef.io.network.group.api.driver;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos.GroupCommMessage.Type;
+
+/**
+ * Tracks the Status of the ctrl msgs sent to a
+ * task node in the topology -
+ *   what msgs have been sent to this node and
+ *   what msgs have been ACKed as received by this node
+ *   Status of neighbors
+ * This is used to see whether the local topology
+ * of a Task is completely set-up
+ * It also offers convenience methods for waiting
+ * on receiving ACKs from the task.
+ */
+public interface TaskNodeStatus {
+
+  boolean hasChanges();
+
+  void onTopologySetupMessageSent();
+
+  boolean isActive(String neighborId);
+
+  /**
+   * Process the msg that was received and update
+   * state accordingly
+   */
+  void processAcknowledgement(GroupCommunicationMessage msg);
+
+  /**
+   * To be called before sending a ctrl msg to the task
+   * represented by this node. All ctrl msgs sent to this
+   * node need to be ACKed.
+   * Ctrl msgs will be sent        from a src &
+   * ACK sent from the task will be for a src.
+   * As this is called from the TaskNodeImpl use srcId of msg
+   * In TaskNodeImpl while processMsg        use dstId of msg
+   */
+  public void expectAckFor(final Type msgType, final String srcId);
+
+  /**
+   * Used when the task has failed to clear all
+   * the state that is associated with this task
+   * Also should release the locks held for implementing
+   * the convenience wait* methods
+   */
+  void clearStateAndReleaseLocks();
+
+  /**
+   * This should remove state concerning neighboring tasks
+   * that have failed
+   */
+  void updateFailureOf(String taskId);
+
+  void waitForTopologySetup();
+
+  /**
+   * Called to denote that a UpdateTopology msg will
+   * be sent
+   */
+  void updatingTopology ();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/Topology.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/Topology.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/Topology.java
new file mode 100644
index 0000000..b393d7f
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/driver/Topology.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.api.driver;
+
+import org.apache.reef.io.network.group.api.config.OperatorSpec;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.tang.Configuration;
+
+/**
+ * A topology should implement the following
+ * interface so that it can work with the
+ * elastic group communication framework
+ * Currently we have two implementations
+ * 1. Flat 2. Tree
+ */
+public interface Topology {
+
+  /**
+   * Get the version of the Task 'taskId'
+   * that belongs to this topology
+   *
+   * @param taskId
+   * @return
+   */
+  int getNodeVersion(String taskId);
+
+  /**
+   * Get the id of the root task
+   *
+   * @return
+   */
+  String getRootId();
+
+  /**
+   * Set task with id 'senderId' as
+   * the root of this topology
+   *
+   * @param senderId
+   */
+  void setRootTask(String senderId);
+
+  /**
+   * Add task with id 'taskId' to
+   * the topology
+   *
+   * @param taskId
+   */
+  void addTask(String taskId);
+
+  /**
+   * Remove task with id 'taskId' from
+   * the topology
+   *
+   * @param taskId
+   */
+  void removeTask(String taskId);
+
+  /**
+   * Update state on receipt of RunningTask
+   * event for task with id 'id'
+   *
+   * @param id
+   */
+  void onRunningTask(String id);
+
+  /**
+   * Update state on receipt of FailedTask
+   * event for task with id 'id'
+   *
+   * @param id
+   */
+  void onFailedTask(String id);
+
+  /**
+   * Set operator specification of the operator
+   * that is the owner of this topology instance
+   *
+   * @param spec
+   */
+  void setOperatorSpecification(OperatorSpec spec);
+
+  /**
+   * Get the topology portion of the Configuration
+   * for the task 'taskId' that belongs to this
+   * topology
+   *
+   * @param taskId
+   * @return
+   */
+  Configuration getTaskConfiguration(String taskId);
+
+  /**
+   * Update state on receipt of a message
+   * from the tasks
+   *
+   * @param msg
+   */
+  void onReceiptOfMessage(GroupCommunicationMessage msg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AbstractGroupCommOperator.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AbstractGroupCommOperator.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AbstractGroupCommOperator.java
new file mode 100644
index 0000000..ac89338
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AbstractGroupCommOperator.java
@@ -0,0 +1,44 @@
+/**
+ * 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.reef.io.network.group.api.operators;
+
+import org.apache.reef.tang.annotations.Name;
+
+public abstract class AbstractGroupCommOperator implements GroupCommOperator {
+
+  @Override
+  public Class<? extends Name<String>> getOperName() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Class<? extends Name<String>> getGroupName() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void initialize() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getVersion() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllGather.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllGather.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllGather.java
new file mode 100644
index 0000000..8c2d047
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllGather.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.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.wake.Identifier;
+
+import java.util.List;
+
+/**
+ * MPI AllGather Operator.
+ * <p/>
+ * Each task applies this operator on an element of type T. The result will be
+ * a list of elements constructed using the elements all-gathered at each
+ * task.
+ */
+public interface AllGather<T> extends GroupCommOperator {
+
+  /**
+   * Apply the operation on element.
+   *
+   * @return List of all elements on which the operation was applied using default order
+   */
+  List<T> apply(T element) throws NetworkException,
+      InterruptedException;
+
+  /**
+   * Apply the operation on element.
+   *
+   * @return List of all elements on which the operation was applied using order specified
+   */
+  List<T> apply(T element, List<? extends Identifier> order)
+      throws NetworkException, InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllReduce.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllReduce.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllReduce.java
new file mode 100644
index 0000000..fd4bda5
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/AllReduce.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.wake.Identifier;
+
+import java.util.List;
+
+/**
+ * MPI All Reduce Operator. Each task applies this operator on an element of
+ * type T. The result will be an element which is result of applying a reduce
+ * function on the list of all elements on which this operator has been applied
+ */
+public interface AllReduce<T> extends GroupCommOperator {
+
+  /**
+   * Apply the operation on element.
+   *
+   * @return result of all-reduce on all elements operation was applied on.
+   * Reduce function is applied based on default order.
+   */
+  T apply(T aElement) throws InterruptedException, NetworkException;
+
+  /**
+   * Apply the operation on element.
+   *
+   * @return result of all-reduce on all elements operation was applied on.
+   * Reduce function is applied based on specified order.
+   */
+  T apply(T element, List<? extends Identifier> order) throws InterruptedException, NetworkException;
+
+  /**
+   * Get the {@link org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction} configured.
+   *
+   * @return {@link org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction}
+   */
+  Reduce.ReduceFunction<T> getReduceFunction();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Broadcast.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Broadcast.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Broadcast.java
new file mode 100644
index 0000000..ae007ac
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Broadcast.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.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.group.impl.operators.BroadcastReceiver;
+import org.apache.reef.io.network.group.impl.operators.BroadcastSender;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+
+/**
+ * MPI Broadcast operator.
+ * <p/>
+ * The sender or root send's an element that is received by all the receivers or other tasks.
+ * <p/>
+ * This is an asymmetric operation and hence the differentiation b/w Sender and Receiver.
+ */
+public interface Broadcast {
+
+  /**
+   * Sender or Root.
+   */
+  @DefaultImplementation(BroadcastSender.class)
+  static interface Sender<T> extends GroupCommOperator {
+
+    /**
+     * Send element to all receivers.
+     */
+    void send(T element) throws NetworkException, InterruptedException;
+  }
+
+  /**
+   * Receivers or Non-roots
+   */
+  @DefaultImplementation(BroadcastReceiver.class)
+  static interface Receiver<T> extends GroupCommOperator {
+
+    /**
+     * Receiver the element broadcasted by sender.
+     *
+     * @return the element broadcasted by sender
+     */
+    T receive() throws NetworkException, InterruptedException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Gather.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Gather.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Gather.java
new file mode 100644
index 0000000..a04b02e
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Gather.java
@@ -0,0 +1,64 @@
+/**
+ * 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.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.wake.Identifier;
+
+import java.util.List;
+
+/**
+ * MPI Gather Operator.
+ * <p/>
+ * This is an operator where the root is a receiver and there are multiple senders.
+ * The root or receiver gathers all the elements sent by the senders in a List.
+ */
+public interface Gather {
+
+  /**
+   * Senders or non-roots.
+   */
+  static interface Sender<T> extends GroupCommOperator {
+
+    /**
+     * Send the element to the root/receiver.
+     */
+    void send(T element) throws InterruptedException, NetworkException;
+  }
+
+  /**
+   * Receiver or Root
+   */
+  static interface Receiver<T> extends GroupCommOperator {
+
+    /**
+     * Receive the elements sent by the senders in default order.
+     *
+     * @return elements sent by senders as a List in default order
+     */
+    List<T> receive() throws InterruptedException, NetworkException;
+
+    /**
+     * Receive the elements sent by the senders in specified order
+     *
+     * @return elements sent by senders as a List in specified order
+     */
+    List<T> receive(List<? extends Identifier> order) throws InterruptedException, NetworkException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/GroupCommOperator.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/GroupCommOperator.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/GroupCommOperator.java
new file mode 100644
index 0000000..ae90d99
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/GroupCommOperator.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.reef.io.network.group.api.operators;
+
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.tang.annotations.Name;
+
+public interface GroupCommOperator {
+
+  Class<? extends Name<String>> getOperName();
+
+  Class<? extends Name<String>> getGroupName();
+
+  void initialize() throws ParentDeadException;
+
+  int getVersion();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Reduce.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Reduce.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Reduce.java
new file mode 100644
index 0000000..b638348
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Reduce.java
@@ -0,0 +1,99 @@
+/**
+ * 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.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.group.impl.operators.ReduceReceiver;
+import org.apache.reef.io.network.group.impl.operators.ReduceSender;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.wake.Identifier;
+
+import java.util.List;
+
+/**
+ * MPI Reduce operator.
+ * <p/>
+ * This is another operator with root being receiver All senders send an element
+ * to the receiver. These elements are passed through a reduce function and its
+ * result is made available at the root
+ */
+public interface Reduce {
+
+  /**
+   * Receiver or Root
+   */
+  @DefaultImplementation(ReduceReceiver.class)
+  static interface Receiver<T> extends GroupCommOperator {
+
+    /**
+     * Receive values sent by senders and pass them through the reduce
+     * function in default order.
+     *
+     * @return Result of applying reduce function on the elements gathered in default order.
+     */
+    T reduce() throws InterruptedException, NetworkException;
+
+    /**
+     * Receive values sent by senders and pass them through the reduce
+     * function in specified order.
+     *
+     * @return Result of applying reduce function on the elements gathered in specified order.
+     */
+    T reduce(List<? extends Identifier> order) throws InterruptedException, NetworkException;
+
+    /**
+     * The reduce function to be applied on the set of received values
+     *
+     * @return {@link ReduceFunction}
+     */
+    Reduce.ReduceFunction<T> getReduceFunction();
+  }
+
+  /**
+   * Senders or non roots
+   */
+  @DefaultImplementation(ReduceSender.class)
+  static interface Sender<T> extends GroupCommOperator {
+
+    /**
+     * Send the element to the root.
+     */
+    void send(T element) throws NetworkException, InterruptedException;
+
+    /**
+     * The {@link ReduceFunction} to be applied on the set of received values.
+     *
+     * @return {@link ReduceFunction}
+     */
+    Reduce.ReduceFunction<T> getReduceFunction();
+  }
+
+  /**
+   * Interface for a Reduce Function takes in an {@link Iterable} returns an
+   * aggregate value computed from the {@link Iterable}
+   */
+  static interface ReduceFunction<T> {
+    /**
+     * Apply the function on elements.
+     *
+     * @return aggregate value computed from elements.
+     */
+    T apply(Iterable<T> elements);
+  }
+}


[4/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CommunicationGroupDriverImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CommunicationGroupDriverImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CommunicationGroupDriverImpl.java
new file mode 100644
index 0000000..d9a4cd9
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CommunicationGroupDriverImpl.java
@@ -0,0 +1,451 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.driver.task.TaskConfigurationOptions;
+import org.apache.reef.io.network.group.api.config.OperatorSpec;
+import org.apache.reef.io.network.group.api.driver.CommunicationGroupDriver;
+import org.apache.reef.io.network.group.api.driver.Topology;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.BroadcastOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.ReduceOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.parameters.CommunicationGroupName;
+import org.apache.reef.io.network.group.impl.config.parameters.OperatorName;
+import org.apache.reef.io.network.group.impl.config.parameters.SerializedOperConfigs;
+import org.apache.reef.io.network.group.impl.utils.BroadcastingEventHandler;
+import org.apache.reef.io.network.group.impl.utils.CountingSemaphore;
+import org.apache.reef.io.network.group.impl.utils.SetMap;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.wake.EStage;
+
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+@DriverSide
+@Private
+public class CommunicationGroupDriverImpl implements CommunicationGroupDriver {
+
+  private static final Logger LOG = Logger.getLogger(CommunicationGroupDriverImpl.class.getName());
+
+  private final Class<? extends Name<String>> groupName;
+  private final ConcurrentMap<Class<? extends Name<String>>, OperatorSpec> operatorSpecs = new ConcurrentHashMap<>();
+  private final ConcurrentMap<Class<? extends Name<String>>, Topology> topologies = new ConcurrentHashMap<>();
+  private final Map<String, TaskState> perTaskState = new HashMap<>();
+  private boolean finalised = false;
+  private final ConfigurationSerializer confSerializer;
+  private final EStage<GroupCommunicationMessage> senderStage;
+  private final String driverId;
+  private final int numberOfTasks;
+
+  private final CountingSemaphore allTasksAdded;
+
+  private final Object topologiesLock = new Object();
+  private final Object configLock = new Object();
+  private final AtomicBoolean initializing = new AtomicBoolean(true);
+
+  private final Object yetToRunLock = new Object();
+  private final Object toBeRemovedLock = new Object();
+
+  private final SetMap<MsgKey, IndexedMsg> msgQue = new SetMap<>();
+
+  private final int fanOut;
+
+  public CommunicationGroupDriverImpl(final Class<? extends Name<String>> groupName,
+                                      final ConfigurationSerializer confSerializer,
+                                      final EStage<GroupCommunicationMessage> senderStage,
+                                      final BroadcastingEventHandler<RunningTask> commGroupRunningTaskHandler,
+                                      final BroadcastingEventHandler<FailedTask> commGroupFailedTaskHandler,
+                                      final BroadcastingEventHandler<FailedEvaluator> commGroupFailedEvaluatorHandler,
+                                      final BroadcastingEventHandler<GroupCommunicationMessage> commGroupMessageHandler,
+                                      final String driverId, final int numberOfTasks, final int fanOut) {
+    super();
+    this.groupName = groupName;
+    this.numberOfTasks = numberOfTasks;
+    this.driverId = driverId;
+    this.confSerializer = confSerializer;
+    this.senderStage = senderStage;
+    this.fanOut = fanOut;
+    this.allTasksAdded = new CountingSemaphore(numberOfTasks, getQualifiedName(), topologiesLock);
+
+    final TopologyRunningTaskHandler topologyRunningTaskHandler = new TopologyRunningTaskHandler(this);
+    commGroupRunningTaskHandler.addHandler(topologyRunningTaskHandler);
+    final TopologyFailedTaskHandler topologyFailedTaskHandler = new TopologyFailedTaskHandler(this);
+    commGroupFailedTaskHandler.addHandler(topologyFailedTaskHandler);
+    final TopologyFailedEvaluatorHandler topologyFailedEvaluatorHandler = new TopologyFailedEvaluatorHandler(this);
+    commGroupFailedEvaluatorHandler.addHandler(topologyFailedEvaluatorHandler);
+    final TopologyMessageHandler topologyMessageHandler = new TopologyMessageHandler(this);
+    commGroupMessageHandler.addHandler(topologyMessageHandler);
+  }
+
+  @Override
+  public CommunicationGroupDriver addBroadcast(final Class<? extends Name<String>> operatorName,
+                                               final BroadcastOperatorSpec spec) {
+    LOG.entering("CommunicationGroupDriverImpl", "addBroadcast", new Object[]{getQualifiedName(), Utils.simpleName(operatorName), spec});
+    if (finalised) {
+      throw new IllegalStateException("Can't add more operators to a finalised spec");
+    }
+    operatorSpecs.put(operatorName, spec);
+    final Topology topology = new TreeTopology(senderStage, groupName, operatorName, driverId, numberOfTasks, fanOut);
+    topology.setRootTask(spec.getSenderId());
+    topology.setOperatorSpecification(spec);
+    topologies.put(operatorName, topology);
+    LOG.exiting("CommunicationGroupDriverImpl", "addBroadcast", Arrays.toString(new Object[]{getQualifiedName(), Utils.simpleName(operatorName), " added"}));
+    return this;
+  }
+
+  @Override
+  public CommunicationGroupDriver addReduce(final Class<? extends Name<String>> operatorName,
+                                            final ReduceOperatorSpec spec) {
+    LOG.entering("CommunicationGroupDriverImpl", "addReduce", new Object[]{getQualifiedName(), Utils.simpleName(operatorName), spec});
+    if (finalised) {
+      throw new IllegalStateException("Can't add more operators to a finalised spec");
+    }
+    LOG.finer(getQualifiedName() + "Adding reduce operator to tree topology: " + spec);
+    operatorSpecs.put(operatorName, spec);
+    final Topology topology = new TreeTopology(senderStage, groupName, operatorName, driverId, numberOfTasks, fanOut);
+    topology.setRootTask(spec.getReceiverId());
+    topology.setOperatorSpecification(spec);
+    topologies.put(operatorName, topology);
+    LOG.exiting("CommunicationGroupDriverImpl", "addReduce", Arrays.toString(new Object[]{getQualifiedName(), Utils.simpleName(operatorName), " added"}));
+    return this;
+  }
+
+  @Override
+  public Configuration getTaskConfiguration(final Configuration taskConf) {
+    LOG.entering("CommunicationGroupDriverImpl", "getTaskConfiguration", new Object[]{getQualifiedName(), confSerializer.toString(taskConf)});
+    final JavaConfigurationBuilder jcb = Tang.Factory.getTang().newConfigurationBuilder();
+    final String taskId = taskId(taskConf);
+    if (perTaskState.containsKey(taskId)) {
+      jcb.bindNamedParameter(DriverIdentifier.class, driverId);
+      jcb.bindNamedParameter(CommunicationGroupName.class, groupName.getName());
+      LOG.finest(getQualifiedName() + "Task has been added. Waiting to acquire configLock");
+      synchronized (configLock) {
+        LOG.finest(getQualifiedName() + "Acquired configLock");
+        while (cantGetConfig(taskId)) {
+          LOG.finest(getQualifiedName() + "Need to wait for failure");
+          try {
+            configLock.wait();
+          } catch (final InterruptedException e) {
+            throw new RuntimeException(getQualifiedName() + "InterruptedException while waiting on configLock", e);
+          }
+        }
+        LOG.finest(getQualifiedName() + taskId + " - Will fetch configuration now.");
+        LOG.finest(getQualifiedName() + "Released configLock. Waiting to acquire topologiesLock");
+      }
+      synchronized (topologiesLock) {
+        LOG.finest(getQualifiedName() + "Acquired topologiesLock");
+        for (final Map.Entry<Class<? extends Name<String>>, OperatorSpec> operSpecEntry : operatorSpecs.entrySet()) {
+          final Class<? extends Name<String>> operName = operSpecEntry.getKey();
+          final Topology topology = topologies.get(operName);
+          final JavaConfigurationBuilder jcbInner = Tang.Factory.getTang()
+              .newConfigurationBuilder(topology.getTaskConfiguration(taskId));
+          jcbInner.bindNamedParameter(DriverIdentifier.class, driverId);
+          jcbInner.bindNamedParameter(OperatorName.class, operName.getName());
+          jcb.bindSetEntry(SerializedOperConfigs.class, confSerializer.toString(jcbInner.build()));
+        }
+        LOG.finest(getQualifiedName() + "Released topologiesLock");
+      }
+    } else {
+      return null;
+    }
+    final Configuration configuration = jcb.build();
+    LOG.exiting("CommunicationGroupDriverImpl", "getTaskConfiguration", Arrays.toString(new Object[]{getQualifiedName(), confSerializer.toString(configuration)}));
+    return configuration;
+  }
+
+  private boolean cantGetConfig(final String taskId) {
+    LOG.entering("CommunicationGroupDriverImpl", "cantGetConfig", new Object[]{getQualifiedName(), taskId});
+    final TaskState taskState = perTaskState.get(taskId);
+    if (!taskState.equals(TaskState.NOT_STARTED)) {
+      LOG.finest(getQualifiedName() + taskId + " has started.");
+      if (taskState.equals(TaskState.RUNNING)) {
+        LOG.exiting("CommunicationGroupDriverImpl", "cantGetConfig", Arrays.toString(new Object[]{true, getQualifiedName(), taskId, " is running. We can't get config"}));
+        return true;
+      } else {
+        LOG.exiting("CommunicationGroupDriverImpl", "cantGetConfig", Arrays.toString(new Object[]{false, getQualifiedName(), taskId, " has failed. We can get config"}));
+        return false;
+      }
+    } else {
+      LOG.exiting("CommunicationGroupDriverImpl", "cantGetConfig", Arrays.toString(new Object[]{false, getQualifiedName(), taskId, " has not started. We can get config"}));
+      return false;
+    }
+  }
+
+  @Override
+  public void finalise() {
+    finalised = true;
+  }
+
+  @Override
+  public void addTask(final Configuration partialTaskConf) {
+    LOG.entering("CommunicationGroupDriverImpl", "addTask", new Object[]{getQualifiedName(), confSerializer.toString(partialTaskConf)});
+    final String taskId = taskId(partialTaskConf);
+    LOG.finest(getQualifiedName() + "AddTask(" + taskId + "). Waiting to acquire toBeRemovedLock");
+    synchronized (toBeRemovedLock) {
+      LOG.finest(getQualifiedName() + "Acquired toBeRemovedLock");
+      while (perTaskState.containsKey(taskId)) {
+        LOG.finest(getQualifiedName() + "Trying to add an existing task. Will wait for removeTask");
+        try {
+          toBeRemovedLock.wait();
+        } catch (final InterruptedException e) {
+          throw new RuntimeException(getQualifiedName() + "InterruptedException while waiting on toBeRemovedLock", e);
+        }
+      }
+      LOG.finest(getQualifiedName() + "Released toBeRemovedLock. Waiting to acquire topologiesLock");
+    }
+    synchronized (topologiesLock) {
+      LOG.finest(getQualifiedName() + "Acquired topologiesLock");
+      for (final Class<? extends Name<String>> operName : operatorSpecs.keySet()) {
+        final Topology topology = topologies.get(operName);
+        topology.addTask(taskId);
+      }
+      perTaskState.put(taskId, TaskState.NOT_STARTED);
+      LOG.finest(getQualifiedName() + "Released topologiesLock");
+    }
+    LOG.fine(getQualifiedName() + "Added " + taskId + " to topology");
+    LOG.exiting("CommunicationGroupDriverImpl", "addTask", Arrays.toString(new Object[]{getQualifiedName(), "Added task: ", taskId}));
+  }
+
+  public void removeTask(final String taskId) {
+    LOG.entering("CommunicationGroupDriverImpl", "removeTask", new Object[]{getQualifiedName(), taskId});
+    LOG.info(getQualifiedName() + "Removing Task " + taskId +
+        " as the evaluator has failed.");
+    LOG.finest(getQualifiedName() + "Remove Task(" + taskId +
+        "): Waiting to acquire topologiesLock");
+    synchronized (topologiesLock) {
+      LOG.finest(getQualifiedName() + "Acquired topologiesLock");
+      for (final Class<? extends Name<String>> operName : operatorSpecs.keySet()) {
+        final Topology topology = topologies.get(operName);
+        topology.removeTask(taskId);
+      }
+      perTaskState.remove(taskId);
+      LOG.finest(getQualifiedName() + "Released topologiesLock. Waiting to acquire toBeRemovedLock");
+    }
+    synchronized (toBeRemovedLock) {
+      LOG.finest(getQualifiedName() + "Acquired toBeRemovedLock");
+      LOG.finest(getQualifiedName() + "Removed Task " + taskId + " Notifying waiting threads");
+      toBeRemovedLock.notifyAll();
+      LOG.finest(getQualifiedName() + "Released toBeRemovedLock");
+    }
+    LOG.fine(getQualifiedName() + "Removed " + taskId + " to topology");
+    LOG.exiting("CommunicationGroupDriverImpl", "removeTask", Arrays.toString(new Object[]{getQualifiedName(), "Removed task: ", taskId}));
+  }
+
+  public void runTask(final String id) {
+    LOG.entering("CommunicationGroupDriverImpl", "runTask", new Object[]{getQualifiedName(), id});
+    LOG.finest(getQualifiedName() + "Task-" + id + " running. Waiting to acquire topologiesLock");
+    LOG.fine(getQualifiedName() + "Got running Task: " + id);
+
+    boolean nonMember = false;
+    synchronized (topologiesLock) {
+      if (perTaskState.containsKey(id)) {
+        LOG.finest(getQualifiedName() + "Acquired topologiesLock");
+        for (final Class<? extends Name<String>> operName : operatorSpecs.keySet()) {
+          final Topology topology = topologies.get(operName);
+          topology.onRunningTask(id);
+        }
+        allTasksAdded.decrement();
+        perTaskState.put(id, TaskState.RUNNING);
+        LOG.finest(getQualifiedName() + "Released topologiesLock. Waiting to acquire yetToRunLock");
+      } else {
+        nonMember = true;
+      }
+    }
+    synchronized (yetToRunLock) {
+      LOG.finest(getQualifiedName() + "Acquired yetToRunLock");
+      yetToRunLock.notifyAll();
+      LOG.finest(getQualifiedName() + "Released yetToRunLock");
+    }
+    if (nonMember) {
+      LOG.exiting("CommunicationGroupDriverImpl", "runTask", getQualifiedName() + id + " does not belong to this communication group. Ignoring");
+    } else {
+      LOG.fine(getQualifiedName() + "Status of task " + id + " changed to RUNNING");
+      LOG.exiting("CommunicationGroupDriverImpl", "runTask", Arrays.toString(new Object[]{getQualifiedName(), "Set running complete on task ", id}));
+    }
+  }
+
+  public void failTask(final String id) {
+    LOG.entering("CommunicationGroupDriverImpl", "failTask", new Object[]{getQualifiedName(), id});
+    LOG.finest(getQualifiedName() + "Task-" + id + " failed. Waiting to acquire yetToRunLock");
+    LOG.fine(getQualifiedName() + "Got failed Task: " + id);
+    synchronized (yetToRunLock) {
+      LOG.finest(getQualifiedName() + "Acquired yetToRunLock");
+      while (cantFailTask(id)) {
+        LOG.finest(getQualifiedName() + "Need to wait for it run");
+        try {
+          yetToRunLock.wait();
+        } catch (final InterruptedException e) {
+          throw new RuntimeException(getQualifiedName() + "InterruptedException while waiting on yetToRunLock", e);
+        }
+      }
+      LOG.finest(getQualifiedName() + id + " - Can safely set failure.");
+      LOG.finest(getQualifiedName() + "Released yetToRunLock. Waiting to acquire topologiesLock");
+    }
+    synchronized (topologiesLock) {
+      LOG.finest(getQualifiedName() + "Acquired topologiesLock");
+      for (final Class<? extends Name<String>> operName : operatorSpecs.keySet()) {
+        final Topology topology = topologies.get(operName);
+        topology.onFailedTask(id);
+      }
+      allTasksAdded.increment();
+      perTaskState.put(id, TaskState.FAILED);
+      LOG.finest(getQualifiedName() + "Removing msgs associated with dead task " + id + " from msgQue.");
+      final Set<MsgKey> keys = msgQue.keySet();
+      final List<MsgKey> keysToBeRemoved = new ArrayList<>();
+      for (final MsgKey msgKey : keys) {
+        if (msgKey.getSrc().equals(id)) {
+          keysToBeRemoved.add(msgKey);
+        }
+      }
+      LOG.finest(getQualifiedName() + keysToBeRemoved + " keys that will be removed");
+      for (final MsgKey key : keysToBeRemoved) {
+        msgQue.remove(key);
+      }
+      LOG.finest(getQualifiedName() + "Released topologiesLock. Waiting to acquire configLock");
+    }
+    synchronized (configLock) {
+      LOG.finest(getQualifiedName() + "Acquired configLock");
+      configLock.notifyAll();
+      LOG.finest(getQualifiedName() + "Released configLock");
+    }
+    LOG.fine(getQualifiedName() + "Status of task " + id + " changed to FAILED");
+    LOG.exiting("CommunicationGroupDriverImpl", "failTask", Arrays.toString(new Object[]{getQualifiedName(), "Set failed complete on task ", id}));
+  }
+
+  private boolean cantFailTask(final String taskId) {
+    LOG.entering("CommunicationGroupDriverImpl", "cantFailTask", new Object[]{getQualifiedName(), taskId});
+    final TaskState taskState = perTaskState.get(taskId);
+    if (!taskState.equals(TaskState.NOT_STARTED)) {
+      LOG.finest(getQualifiedName() + taskId + " has started.");
+      if (!taskState.equals(TaskState.RUNNING)) {
+        LOG.exiting("CommunicationGroupDriverImpl", "cantFailTask", Arrays.toString(new Object[]{true, getQualifiedName(), taskId, " is not running yet. Can't set failure"}));
+        return true;
+      } else {
+        LOG.exiting("CommunicationGroupDriverImpl", "cantFailTask", Arrays.toString(new Object[]{false, getQualifiedName(), taskId, " is running. Can set failure"}));
+        return false;
+      }
+    } else {
+      LOG.exiting("CommunicationGroupDriverImpl", "cantFailTask", Arrays.toString(new Object[]{true, getQualifiedName(), taskId, " has not started. We can't fail a task that hasn't started"}));
+      return true;
+    }
+  }
+
+  public void queNProcessMsg(final GroupCommunicationMessage msg) {
+    LOG.entering("CommunicationGroupDriverImpl", "queNProcessMsg", new Object[]{getQualifiedName(), msg});
+    final IndexedMsg indMsg = new IndexedMsg(msg);
+    final Class<? extends Name<String>> operName = indMsg.getOperName();
+    final MsgKey key = new MsgKey(msg);
+    if (msgQue.contains(key, indMsg)) {
+      throw new RuntimeException(getQualifiedName() + "MsgQue already contains " + msg.getType() + " msg for " + key + " in "
+          + Utils.simpleName(operName));
+    }
+    LOG.finest(getQualifiedName() + "Adding msg to que");
+    msgQue.add(key, indMsg);
+    if (msgQue.count(key) == topologies.size()) {
+      LOG.finest(getQualifiedName() + "MsgQue for " + key + " contains " + msg.getType() + " msgs from: "
+          + msgQue.get(key));
+      for (final IndexedMsg innerIndMsg : msgQue.remove(key)) {
+        topologies.get(innerIndMsg.getOperName()).onReceiptOfMessage(innerIndMsg.getMsg());
+      }
+      LOG.finest(getQualifiedName() + "All msgs processed and removed");
+    }
+    LOG.exiting("CommunicationGroupDriverImpl", "queNProcessMsg", Arrays.toString(new Object[]{getQualifiedName(), "Que & Process done for: ", msg}));
+  }
+
+  private boolean isMsgVersionOk(final GroupCommunicationMessage msg) {
+    LOG.entering("CommunicationGroupDriverImpl", "isMsgVersionOk", new Object[]{getQualifiedName(), msg});
+    if (msg.hasVersion()) {
+      final String srcId = msg.getSrcid();
+      final int rcvSrcVersion = msg.getSrcVersion();
+      final int expSrcVersion = topologies.get(Utils.getClass(msg.getOperatorname())).getNodeVersion(srcId);
+
+      final boolean srcVersionChk = chkVersion(rcvSrcVersion, expSrcVersion, "Src Version Check: ");
+      LOG.exiting("CommunicationGroupDriverImpl", "isMsgVersionOk", Arrays.toString(new Object[]{srcVersionChk, getQualifiedName(), msg}));
+      return srcVersionChk;
+    } else {
+      throw new RuntimeException(getQualifiedName() + "can only deal with versioned msgs");
+    }
+  }
+
+  private boolean chkVersion(final int rcvVersion, final int version, final String msg) {
+    if (rcvVersion < version) {
+      LOG.warning(getQualifiedName() + msg + "received a ver-" + rcvVersion + " msg while expecting ver-" + version);
+      return false;
+    }
+    if (rcvVersion > version) {
+      LOG.warning(getQualifiedName() + msg + "received a HIGHER ver-" + rcvVersion + " msg while expecting ver-"
+          + version + ". Something fishy!!!");
+      return false;
+    }
+    return true;
+  }
+
+  public void processMsg(final GroupCommunicationMessage msg) {
+    LOG.entering("CommunicationGroupDriverImpl", "processMsg", new Object[]{getQualifiedName(), msg});
+    LOG.finest(getQualifiedName() + "ProcessMsg: " + msg + ". Waiting to acquire topologiesLock");
+    synchronized (topologiesLock) {
+      LOG.finest(getQualifiedName() + "Acquired topologiesLock");
+      if (!isMsgVersionOk(msg)) {
+        LOG.finer(getQualifiedName() + "Discarding msg. Released topologiesLock");
+        return;
+      }
+      if (initializing.get() || msg.getType().equals(ReefNetworkGroupCommProtos.GroupCommMessage.Type.UpdateTopology)) {
+        LOG.fine(getQualifiedName() + msg.getSimpleOperName() + ": Waiting for all required(" + allTasksAdded.getInitialCount() +
+            ") nodes to run");
+        allTasksAdded.await();
+        LOG.fine(getQualifiedName() + msg.getSimpleOperName() + ": All required(" + allTasksAdded.getInitialCount() +
+            ") nodes are running");
+        initializing.compareAndSet(true, false);
+      }
+      queNProcessMsg(msg);
+      LOG.finest(getQualifiedName() + "Released topologiesLock");
+    }
+    LOG.exiting("CommunicationGroupDriverImpl", "processMsg", Arrays.toString(new Object[]{getQualifiedName(), "ProcessMsg done for: ", msg}));
+  }
+
+  private String taskId(final Configuration partialTaskConf) {
+    try {
+      final Injector injector = Tang.Factory.getTang().newInjector(partialTaskConf);
+      return injector.getNamedInstance(TaskConfigurationOptions.Identifier.class);
+    } catch (final InjectionException e) {
+      throw new RuntimeException(getQualifiedName() + "Injection exception while extracting taskId from partialTaskConf", e);
+    }
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + " - ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CtrlMsgSender.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CtrlMsgSender.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CtrlMsgSender.java
new file mode 100644
index 0000000..5535511
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/CtrlMsgSender.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.reef.io.network.group.impl.driver;
+
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.network.Connection;
+import org.apache.reef.io.network.impl.NetworkService;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.Identifier;
+import org.apache.reef.wake.IdentifierFactory;
+
+import java.util.logging.Logger;
+
+/**
+ * Event handler that receives ctrl msgs and
+ * dispatched them using network service
+ */
+public class CtrlMsgSender implements EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(CtrlMsgSender.class.getName());
+  private final IdentifierFactory idFac;
+  private final NetworkService<GroupCommunicationMessage> netService;
+
+  public CtrlMsgSender(final IdentifierFactory idFac, final NetworkService<GroupCommunicationMessage> netService) {
+    this.idFac = idFac;
+    this.netService = netService;
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage srcCtrlMsg) {
+    LOG.entering("CtrlMsgSender", "onNext", srcCtrlMsg);
+    final Identifier id = idFac.getNewInstance(srcCtrlMsg.getDestid());
+    final Connection<GroupCommunicationMessage> link = netService.newConnection(id);
+    try {
+      link.open();
+      link.write(srcCtrlMsg);
+    } catch (final NetworkException e) {
+      throw new RuntimeException("Unable to send ctrl task msg to parent " + id, e);
+    }
+    LOG.exiting("CtrlMsgSender", "onNext", srcCtrlMsg);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/ExceptionHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/ExceptionHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/ExceptionHandler.java
new file mode 100644
index 0000000..38a1df6
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/ExceptionHandler.java
@@ -0,0 +1,56 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+/**
+ *
+ */
+public class ExceptionHandler implements EventHandler<Exception> {
+  private static final Logger LOG = Logger.getLogger(ExceptionHandler.class.getName());
+  List<Exception> exceptions = new ArrayList<>();
+
+  @Inject
+  public ExceptionHandler() {
+  }
+
+  @Override
+  public synchronized void onNext(final Exception ex) {
+    LOG.entering("ExceptionHandler", "onNext", new Object[]{ex});
+    exceptions.add(ex);
+    LOG.finest("Got an exception. Added it to list(" + exceptions.size() + ")");
+    LOG.exiting("ExceptionHandler", "onNext");
+  }
+
+  public synchronized boolean hasExceptions() {
+    LOG.entering("ExceptionHandler", "hasExceptions");
+    final boolean ret = !exceptions.isEmpty();
+    LOG.finest("There are " + exceptions.size() + " exceptions. Clearing now");
+    exceptions.clear();
+    LOG.exiting("ExceptionHandler", "hasExceptions", ret);
+    return ret;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/FlatTopology.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/FlatTopology.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/FlatTopology.java
new file mode 100644
index 0000000..70670a2
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/FlatTopology.java
@@ -0,0 +1,307 @@
+/**
+ * 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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.api.operators.GroupCommOperator;
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.network.group.api.config.OperatorSpec;
+import org.apache.reef.io.network.group.api.driver.TaskNode;
+import org.apache.reef.io.network.group.api.driver.Topology;
+import org.apache.reef.io.network.group.impl.GroupChangesCodec;
+import org.apache.reef.io.network.group.impl.GroupChangesImpl;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.config.BroadcastOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.ReduceOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.parameters.DataCodec;
+import org.apache.reef.io.network.group.impl.config.parameters.ReduceFunctionParam;
+import org.apache.reef.io.network.group.impl.config.parameters.TaskVersion;
+import org.apache.reef.io.network.group.impl.operators.BroadcastReceiver;
+import org.apache.reef.io.network.group.impl.operators.BroadcastSender;
+import org.apache.reef.io.network.group.impl.operators.ReduceReceiver;
+import org.apache.reef.io.network.group.impl.operators.ReduceSender;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EStage;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.impl.SingleThreadStage;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.logging.Logger;
+
+/**
+ * Implements a one level Tree Topology
+ */
+public class FlatTopology implements Topology {
+
+  private static final Logger LOG = Logger.getLogger(FlatTopology.class.getName());
+
+  private final EStage<GroupCommunicationMessage> senderStage;
+  private final Class<? extends Name<String>> groupName;
+  private final Class<? extends Name<String>> operName;
+  private final String driverId;
+  private String rootId;
+  private OperatorSpec operatorSpec;
+
+  private TaskNode root;
+  private final ConcurrentMap<String, TaskNode> nodes = new ConcurrentSkipListMap<>();
+
+  public FlatTopology(final EStage<GroupCommunicationMessage> senderStage,
+                      final Class<? extends Name<String>> groupName, final Class<? extends Name<String>> operatorName,
+                      final String driverId, final int numberOfTasks) {
+    this.senderStage = senderStage;
+    this.groupName = groupName;
+    this.operName = operatorName;
+    this.driverId = driverId;
+  }
+
+  @Override
+  public void setRootTask(final String rootId) {
+    this.rootId = rootId;
+  }
+
+  /**
+   * @return the rootId
+   */
+  @Override
+  public String getRootId() {
+    return rootId;
+  }
+
+  @Override
+  public void setOperatorSpecification(final OperatorSpec spec) {
+    this.operatorSpec = spec;
+  }
+
+  @Override
+  public Configuration getTaskConfiguration(final String taskId) {
+    LOG.finest(getQualifiedName() + "Getting config for task " + taskId);
+    final TaskNode taskNode = nodes.get(taskId);
+    if (taskNode == null) {
+      throw new RuntimeException(getQualifiedName() + taskId + " does not exist");
+    }
+
+    final int version;
+    version = getNodeVersion(taskId);
+    final JavaConfigurationBuilder jcb = Tang.Factory.getTang().newConfigurationBuilder();
+    jcb.bindNamedParameter(DataCodec.class, operatorSpec.getDataCodecClass());
+    jcb.bindNamedParameter(TaskVersion.class, Integer.toString(version));
+    if (operatorSpec instanceof BroadcastOperatorSpec) {
+      final BroadcastOperatorSpec broadcastOperatorSpec = (BroadcastOperatorSpec) operatorSpec;
+      if (taskId.equals(broadcastOperatorSpec.getSenderId())) {
+        jcb.bindImplementation(GroupCommOperator.class, BroadcastSender.class);
+      } else {
+        jcb.bindImplementation(GroupCommOperator.class, BroadcastReceiver.class);
+      }
+    }
+    if (operatorSpec instanceof ReduceOperatorSpec) {
+      final ReduceOperatorSpec reduceOperatorSpec = (ReduceOperatorSpec) operatorSpec;
+      jcb.bindNamedParameter(ReduceFunctionParam.class, reduceOperatorSpec.getRedFuncClass());
+      if (taskId.equals(reduceOperatorSpec.getReceiverId())) {
+        jcb.bindImplementation(GroupCommOperator.class, ReduceReceiver.class);
+      } else {
+        jcb.bindImplementation(GroupCommOperator.class, ReduceSender.class);
+      }
+    }
+    return jcb.build();
+  }
+
+  @Override
+  public int getNodeVersion(final String taskId) {
+    final TaskNode node = nodes.get(taskId);
+    if (node == null) {
+      throw new RuntimeException(getQualifiedName() + taskId + " is not available on the nodes map");
+    }
+    final int version = node.getVersion();
+    return version;
+  }
+
+  @Override
+  public void removeTask(final String taskId) {
+    if (!nodes.containsKey(taskId)) {
+      LOG.warning("Trying to remove a non-existent node in the task graph");
+      return;
+    }
+    if (taskId.equals(rootId)) {
+      unsetRootNode(taskId);
+    } else {
+      removeChild(taskId);
+    }
+  }
+
+  @Override
+  public void addTask(final String taskId) {
+    if (nodes.containsKey(taskId)) {
+      LOG.warning("Got a request to add a task that is already in the graph");
+      LOG.warning("We need to block this request till the delete finishes");
+    }
+    if (taskId.equals(rootId)) {
+      setRootNode(taskId);
+    } else {
+      addChild(taskId);
+    }
+  }
+
+  /**
+   * @param taskId
+   */
+  private void addChild(final String taskId) {
+    LOG.finest(getQualifiedName() + "Adding leaf " + taskId);
+    final TaskNode node = new TaskNodeImpl(senderStage, groupName, operName, taskId, driverId, false);
+    final TaskNode leaf = node;
+    if (root != null) {
+      leaf.setParent(root);
+      root.addChild(leaf);
+    }
+    nodes.put(taskId, leaf);
+  }
+
+  /**
+   * @param taskId
+   */
+  private void removeChild(final String taskId) {
+    LOG.finest(getQualifiedName() + "Removing leaf " + taskId);
+    if (root != null) {
+      root.removeChild(nodes.get(taskId));
+    }
+    nodes.remove(taskId);
+  }
+
+  private void setRootNode(final String rootId) {
+    LOG.finest(getQualifiedName() + "Setting " + rootId + " as root");
+    final TaskNode node = new TaskNodeImpl(senderStage, groupName, operName, rootId, driverId, true);
+    this.root = node;
+
+    for (final Map.Entry<String, TaskNode> nodeEntry : nodes.entrySet()) {
+      final String id = nodeEntry.getKey();
+
+      final TaskNode leaf = nodeEntry.getValue();
+      root.addChild(leaf);
+      leaf.setParent(root);
+    }
+    nodes.put(rootId, root);
+  }
+
+  /**
+   * @param taskId
+   */
+  private void unsetRootNode(final String taskId) {
+    LOG.finest(getQualifiedName() + "Unsetting " + rootId + " as root");
+    nodes.remove(rootId);
+
+    for (final Map.Entry<String, TaskNode> nodeEntry : nodes.entrySet()) {
+      final String id = nodeEntry.getKey();
+      final TaskNode leaf = nodeEntry.getValue();
+      leaf.setParent(null);
+    }
+  }
+
+  @Override
+  public void onFailedTask(final String id) {
+    LOG.finest(getQualifiedName() + "Task-" + id + " failed");
+    final TaskNode taskNode = nodes.get(id);
+    if (taskNode == null) {
+      throw new RuntimeException(getQualifiedName() + id + " does not exist");
+    }
+
+    taskNode.onFailedTask();
+  }
+
+  @Override
+  public void onRunningTask(final String id) {
+    LOG.finest(getQualifiedName() + "Task-" + id + " is running");
+    final TaskNode taskNode = nodes.get(id);
+    if (taskNode == null) {
+      throw new RuntimeException(getQualifiedName() + id + " does not exist");
+    }
+
+    taskNode.onRunningTask();
+  }
+
+  @Override
+  public void onReceiptOfMessage(final GroupCommunicationMessage msg) {
+    LOG.finest(getQualifiedName() + "processing " + msg.getType() + " from " + msg.getSrcid());
+    if (msg.getType().equals(ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyChanges)) {
+      processTopologyChanges(msg);
+      return;
+    }
+    if (msg.getType().equals(ReefNetworkGroupCommProtos.GroupCommMessage.Type.UpdateTopology)) {
+      processUpdateTopology(msg);
+      return;
+    }
+    final String id = msg.getSrcid();
+    nodes.get(id).onReceiptOfAcknowledgement(msg);
+  }
+
+  private void processUpdateTopology(final GroupCommunicationMessage msg) {
+    final String dstId = msg.getSrcid();
+    final int version = getNodeVersion(dstId);
+
+    LOG.finest(getQualifiedName() + "Creating NodeTopologyUpdateWaitStage to wait on nodes to be updated");
+    final EventHandler<List<TaskNode>> topoUpdateWaitHandler = new TopologyUpdateWaitHandler(senderStage, groupName,
+        operName, driverId, 0,
+        dstId, version,
+        getQualifiedName());
+    final EStage<List<TaskNode>> nodeTopologyUpdateWaitStage = new SingleThreadStage<>("NodeTopologyUpdateWaitStage",
+        topoUpdateWaitHandler,
+        nodes.size());
+
+    final List<TaskNode> toBeUpdatedNodes = new ArrayList<>(nodes.size());
+    LOG.finest(getQualifiedName() + "Checking which nodes need to be updated");
+    for (final TaskNode node : nodes.values()) {
+      if (node.isRunning() && node.hasChanges() && node.resetTopologySetupSent()) {
+        toBeUpdatedNodes.add(node);
+      }
+    }
+    for (final TaskNode node : toBeUpdatedNodes) {
+      node.updatingTopology();
+      senderStage.onNext(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.UpdateTopology, driverId, 0, node.getTaskId(),
+          node.getVersion(), Utils.EmptyByteArr));
+    }
+    nodeTopologyUpdateWaitStage.onNext(toBeUpdatedNodes);
+  }
+
+  private void processTopologyChanges(final GroupCommunicationMessage msg) {
+    final String dstId = msg.getSrcid();
+    boolean hasTopologyChanged = false;
+    LOG.finest(getQualifiedName() + "Checking which nodes need to be updated");
+    for (final TaskNode node : nodes.values()) {
+      if (!node.isRunning() || node.hasChanges()) {
+        hasTopologyChanged = true;
+        break;
+      }
+    }
+    final GroupChanges changes = new GroupChangesImpl(hasTopologyChanged);
+    final Codec<GroupChanges> changesCodec = new GroupChangesCodec();
+    senderStage.onNext(Utils.bldVersionedGCM(groupName, operName, ReefNetworkGroupCommProtos.GroupCommMessage.Type.TopologyChanges, driverId, 0, dstId, getNodeVersion(dstId),
+        changesCodec.encode(changes)));
+  }
+
+  private String getQualifiedName() {
+    return Utils.simpleName(groupName) + ":" + Utils.simpleName(operName) + " - ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.java
new file mode 100644
index 0000000..8c01b31
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommDriverImpl.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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.ContextConfiguration;
+import org.apache.reef.driver.context.ServiceConfiguration;
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.driver.parameters.DriverIdentifier;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.io.network.Message;
+import org.apache.reef.io.network.impl.*;
+import org.apache.reef.io.network.naming.NameServer;
+import org.apache.reef.io.network.naming.NameServerImpl;
+import org.apache.reef.io.network.naming.NameServerParameters;
+import org.apache.reef.io.network.group.api.driver.CommunicationGroupDriver;
+import org.apache.reef.io.network.group.api.driver.GroupCommServiceDriver;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessageCodec;
+import org.apache.reef.io.network.group.impl.config.parameters.SerializedGroupConfigs;
+import org.apache.reef.io.network.group.impl.config.parameters.TreeTopologyFanOut;
+import org.apache.reef.io.network.group.impl.task.GroupCommNetworkHandlerImpl;
+import org.apache.reef.io.network.group.impl.utils.BroadcastingEventHandler;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.util.StringIdentifierFactory;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.util.SingletonAsserter;
+import org.apache.reef.wake.EStage;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.IdentifierFactory;
+import org.apache.reef.wake.impl.LoggingEventHandler;
+import org.apache.reef.wake.impl.SingleThreadStage;
+import org.apache.reef.wake.impl.SyncStage;
+import org.apache.reef.wake.impl.ThreadPoolStage;
+import org.apache.reef.wake.remote.NetUtils;
+
+import javax.inject.Inject;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+/**
+ * Sets up various stages to handle REEF events and adds the per communication
+ * group stages to them whenever a new communication group is created.
+ * <p/>
+ * Also starts the NameService & the NetworkService on the driver
+ */
+public class GroupCommDriverImpl implements GroupCommServiceDriver {
+  private static final Logger LOG = Logger.getLogger(GroupCommDriverImpl.class.getName());
+  /**
+   * TANG instance
+   */
+  private static final Tang tang = Tang.Factory.getTang();
+
+  private final AtomicInteger contextIds = new AtomicInteger(0);
+
+  private final IdentifierFactory idFac = new StringIdentifierFactory();
+
+  private final NameServer nameService = new NameServerImpl(0, idFac);
+
+  private final String nameServiceAddr;
+  private final int nameServicePort;
+
+  private final Map<Class<? extends Name<String>>, CommunicationGroupDriver> commGroupDrivers = new HashMap<>();
+
+  private final ConfigurationSerializer confSerializer;
+
+  private final NetworkService<GroupCommunicationMessage> netService;
+
+  private final EStage<GroupCommunicationMessage> senderStage;
+
+  private final String driverId;
+  private final BroadcastingEventHandler<RunningTask> groupCommRunningTaskHandler;
+  private final EStage<RunningTask> groupCommRunningTaskStage;
+  private final BroadcastingEventHandler<FailedTask> groupCommFailedTaskHandler;
+  private final EStage<FailedTask> groupCommFailedTaskStage;
+  private final BroadcastingEventHandler<FailedEvaluator> groupCommFailedEvaluatorHandler;
+  private final EStage<FailedEvaluator> groupCommFailedEvaluatorStage;
+  private final GroupCommMessageHandler groupCommMessageHandler;
+  private final EStage<GroupCommunicationMessage> groupCommMessageStage;
+  private final int fanOut;
+
+  @Inject
+  public GroupCommDriverImpl(final ConfigurationSerializer confSerializer,
+                             @Parameter(DriverIdentifier.class) final String driverId,
+                             @Parameter(TreeTopologyFanOut.class) final int fanOut) {
+    assert (SingletonAsserter.assertSingleton(getClass()));
+    this.driverId = driverId;
+    this.fanOut = fanOut;
+    this.nameServiceAddr = NetUtils.getLocalAddress();
+    this.nameServicePort = nameService.getPort();
+    this.confSerializer = confSerializer;
+    this.groupCommRunningTaskHandler = new BroadcastingEventHandler<>();
+    this.groupCommRunningTaskStage = new SyncStage<>("GroupCommRunningTaskStage", groupCommRunningTaskHandler);
+    this.groupCommFailedTaskHandler = new BroadcastingEventHandler<>();
+    this.groupCommFailedTaskStage = new SyncStage<>("GroupCommFailedTaskStage", groupCommFailedTaskHandler);
+    this.groupCommFailedEvaluatorHandler = new BroadcastingEventHandler<>();
+    this.groupCommFailedEvaluatorStage = new SyncStage<>("GroupCommFailedEvaluatorStage",
+        groupCommFailedEvaluatorHandler);
+    this.groupCommMessageHandler = new GroupCommMessageHandler();
+    this.groupCommMessageStage = new SingleThreadStage<>("GroupCommMessageStage", groupCommMessageHandler, 100 * 1000);
+    this.netService = new NetworkService<>(idFac, 0, nameServiceAddr, nameServicePort,
+        new GroupCommunicationMessageCodec(), new MessagingTransportFactory(),
+        new EventHandler<Message<GroupCommunicationMessage>>() {
+
+          @Override
+          public void onNext(final Message<GroupCommunicationMessage> msg) {
+            groupCommMessageStage.onNext(Utils.getGCM(msg));
+          }
+        }, new LoggingEventHandler<Exception>());
+    this.netService.registerId(idFac.getNewInstance(driverId));
+    this.senderStage = new ThreadPoolStage<>("SrcCtrlMsgSender", new CtrlMsgSender(idFac, netService), 5);
+  }
+
+  @Override
+  public CommunicationGroupDriver newCommunicationGroup(final Class<? extends Name<String>> groupName,
+                                                        final int numberOfTasks) {
+    LOG.entering("GroupCommDriverImpl", "newCommunicationGroup", new Object[]{Utils.simpleName(groupName), numberOfTasks});
+    final BroadcastingEventHandler<RunningTask> commGroupRunningTaskHandler = new BroadcastingEventHandler<>();
+    final BroadcastingEventHandler<FailedTask> commGroupFailedTaskHandler = new BroadcastingEventHandler<>();
+    final BroadcastingEventHandler<FailedEvaluator> commGroupFailedEvaluatorHandler = new BroadcastingEventHandler<>();
+    final BroadcastingEventHandler<GroupCommunicationMessage> commGroupMessageHandler = new BroadcastingEventHandler<>();
+    final CommunicationGroupDriver commGroupDriver = new CommunicationGroupDriverImpl(groupName, confSerializer,
+        senderStage,
+        commGroupRunningTaskHandler,
+        commGroupFailedTaskHandler,
+        commGroupFailedEvaluatorHandler,
+        commGroupMessageHandler,
+        driverId, numberOfTasks, fanOut);
+    commGroupDrivers.put(groupName, commGroupDriver);
+    groupCommRunningTaskHandler.addHandler(commGroupRunningTaskHandler);
+    groupCommFailedTaskHandler.addHandler(commGroupFailedTaskHandler);
+    groupCommMessageHandler.addHandler(groupName, commGroupMessageHandler);
+    LOG.exiting("GroupCommDriverImpl", "newCommunicationGroup", "Created communication group: " + Utils.simpleName(groupName));
+    return commGroupDriver;
+  }
+
+  @Override
+  public boolean isConfigured(final ActiveContext activeContext) {
+    LOG.entering("GroupCommDriverImpl", "isConfigured", activeContext.getId());
+    final boolean retVal = activeContext.getId().startsWith("GroupCommunicationContext-");
+    LOG.exiting("GroupCommDriverImpl", "isConfigured", retVal);
+    return retVal;
+  }
+
+  @Override
+  public Configuration getContextConfiguration() {
+    LOG.entering("GroupCommDriverImpl", "getContextConf");
+    final Configuration retVal = ContextConfiguration.CONF.set(ContextConfiguration.IDENTIFIER,
+        "GroupCommunicationContext-" + contextIds.getAndIncrement()).build();
+    LOG.exiting("GroupCommDriverImpl", "getContextConf", confSerializer.toString(retVal));
+    return retVal;
+  }
+
+  @Override
+  public Configuration getServiceConfiguration() {
+    LOG.entering("GroupCommDriverImpl", "getServiceConf");
+    final Configuration serviceConfiguration = ServiceConfiguration.CONF.set(ServiceConfiguration.SERVICES,
+        NetworkService.class)
+        .set(ServiceConfiguration.SERVICES,
+            GroupCommNetworkHandlerImpl.class)
+        .set(ServiceConfiguration.ON_CONTEXT_STOP,
+            NetworkServiceClosingHandler.class)
+        .set(ServiceConfiguration.ON_TASK_STARTED,
+            BindNSToTask.class)
+        .set(ServiceConfiguration.ON_TASK_STOP,
+            UnbindNSFromTask.class).build();
+    final Configuration retVal = tang.newConfigurationBuilder(serviceConfiguration)
+        .bindNamedParameter(NetworkServiceParameters.NetworkServiceCodec.class,
+            GroupCommunicationMessageCodec.class)
+        .bindNamedParameter(NetworkServiceParameters.NetworkServiceHandler.class,
+            GroupCommNetworkHandlerImpl.class)
+        .bindNamedParameter(NetworkServiceParameters.NetworkServiceExceptionHandler.class,
+            ExceptionHandler.class)
+        .bindNamedParameter(NameServerParameters.NameServerAddr.class, nameServiceAddr)
+        .bindNamedParameter(NameServerParameters.NameServerPort.class, Integer.toString(nameServicePort))
+        .bindNamedParameter(NetworkServiceParameters.NetworkServicePort.class, "0").build();
+    LOG.exiting("GroupCommDriverImpl", "getServiceConf", confSerializer.toString(retVal));
+    return retVal;
+  }
+
+  @Override
+  public Configuration getTaskConfiguration(final Configuration partialTaskConf) {
+    LOG.entering("GroupCommDriverImpl", "getTaskConfiguration", new Object[]{confSerializer.toString(partialTaskConf)});
+    final JavaConfigurationBuilder jcb = Tang.Factory.getTang().newConfigurationBuilder(partialTaskConf);
+    for (final CommunicationGroupDriver commGroupDriver : commGroupDrivers.values()) {
+      final Configuration commGroupConf = commGroupDriver.getTaskConfiguration(partialTaskConf);
+      if (commGroupConf != null) {
+        jcb.bindSetEntry(SerializedGroupConfigs.class, confSerializer.toString(commGroupConf));
+      }
+    }
+    final Configuration retVal = jcb.build();
+    LOG.exiting("GroupCommDriverImpl", "getTaskConfiguration", confSerializer.toString(retVal));
+    return retVal;
+  }
+
+  /**
+   * @return the groupCommRunningTaskStage
+   */
+  @Override
+  public EStage<RunningTask> getGroupCommRunningTaskStage() {
+    LOG.entering("GroupCommDriverImpl", "getGroupCommRunningTaskStage");
+    LOG.exiting("GroupCommDriverImpl", "getGroupCommRunningTaskStage", "Returning GroupCommRunningTaskStage");
+    return groupCommRunningTaskStage;
+  }
+
+  /**
+   * @return the groupCommFailedTaskStage
+   */
+  @Override
+  public EStage<FailedTask> getGroupCommFailedTaskStage() {
+    LOG.entering("GroupCommDriverImpl", "getGroupCommFailedTaskStage");
+    LOG.exiting("GroupCommDriverImpl", "getGroupCommFailedTaskStage", "Returning GroupCommFailedTaskStage");
+    return groupCommFailedTaskStage;
+  }
+
+  /**
+   * @return the groupCommFailedEvaluatorStage
+   */
+  @Override
+  public EStage<FailedEvaluator> getGroupCommFailedEvaluatorStage() {
+    LOG.entering("GroupCommDriverImpl", "getGroupCommFailedEvaluatorStage");
+    LOG.exiting("GroupCommDriverImpl", "getGroupCommFailedEvaluatorStage", "Returning GroupCommFaileEvaluatorStage");
+    return groupCommFailedEvaluatorStage;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommMessageHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommMessageHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommMessageHandler.java
new file mode 100644
index 0000000..b466205
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommMessageHandler.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.BroadcastingEventHandler;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EventHandler;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Logger;
+
+/**
+ * The network handler for the group communcation service on the driver side
+ */
+public class GroupCommMessageHandler implements EventHandler<GroupCommunicationMessage> {
+
+  private static final Logger LOG = Logger.getLogger(GroupCommMessageHandler.class.getName());
+
+  private final Map<Class<? extends Name<String>>, BroadcastingEventHandler<GroupCommunicationMessage>>
+      commGroupMessageHandlers = new HashMap<>();
+
+  public void addHandler(final Class<? extends Name<String>> groupName,
+                         final BroadcastingEventHandler<GroupCommunicationMessage> handler) {
+    LOG.entering("GroupCommMessageHandler", "addHandler", new Object[]{Utils.simpleName(groupName), handler});
+    commGroupMessageHandlers.put(groupName, handler);
+    LOG.exiting("GroupCommMessageHandler", "addHandler", Utils.simpleName(groupName));
+  }
+
+  @Override
+  public void onNext(final GroupCommunicationMessage msg) {
+    LOG.entering("GroupCommMessageHandler", "onNext", msg);
+    final Class<? extends Name<String>> groupName = Utils.getClass(msg.getGroupname());
+    commGroupMessageHandlers.get(groupName).onNext(msg);
+    LOG.exiting("GroupCommMessageHandler", "onNext", msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommService.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommService.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommService.java
new file mode 100644
index 0000000..0915fc5
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/GroupCommService.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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.driver.evaluator.FailedEvaluator;
+import org.apache.reef.driver.parameters.EvaluatorDispatcherThreads;
+import org.apache.reef.driver.parameters.ServiceEvaluatorFailedHandlers;
+import org.apache.reef.driver.parameters.ServiceTaskFailedHandlers;
+import org.apache.reef.driver.parameters.TaskRunningHandlers;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.io.network.group.api.driver.GroupCommServiceDriver;
+import org.apache.reef.io.network.group.impl.config.parameters.TreeTopologyFanOut;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Unit;
+import org.apache.reef.tang.formats.AvroConfigurationSerializer;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.logging.Logger;
+
+/**
+ * The Group Communication Service
+ */
+@Unit
+public class GroupCommService {
+
+  private static final Logger LOG = Logger.getLogger(GroupCommService.class.getName());
+  private static final ConfigurationSerializer confSer = new AvroConfigurationSerializer();
+
+  private final GroupCommServiceDriver groupCommDriver;
+
+  @Inject
+  public GroupCommService(final GroupCommServiceDriver groupCommDriver) {
+    this.groupCommDriver = groupCommDriver;
+  }
+
+  public static Configuration getConfiguration() {
+    LOG.entering("GroupCommService", "getConfiguration");
+    final JavaConfigurationBuilder jcb = Tang.Factory.getTang().newConfigurationBuilder();
+    jcb.bindSetEntry(TaskRunningHandlers.class, RunningTaskHandler.class);
+    jcb.bindSetEntry(ServiceTaskFailedHandlers.class, FailedTaskHandler.class);
+    jcb.bindSetEntry(ServiceEvaluatorFailedHandlers.class, FailedEvaluatorHandler.class);
+    jcb.bindNamedParameter(EvaluatorDispatcherThreads.class, "1");
+    final Configuration retVal = jcb.build();
+    LOG.exiting("GroupCommService", "getConfiguration", confSer.toString(retVal));
+    return retVal;
+  }
+
+  public static Configuration getConfiguration(final int fanOut) {
+    LOG.entering("GroupCommService", "getConfiguration", fanOut);
+    final Configuration baseConf = getConfiguration();
+    final Configuration retConf = Tang.Factory.getTang().newConfigurationBuilder(baseConf)
+        .bindNamedParameter(TreeTopologyFanOut.class, Integer.toString(fanOut)).build();
+    LOG.exiting("GroupCommService", "getConfiguration", confSer.toString(retConf));
+    return retConf;
+  }
+
+  public class FailedEvaluatorHandler implements EventHandler<FailedEvaluator> {
+
+    @Override
+    public void onNext(final FailedEvaluator failedEvaluator) {
+      LOG.entering("GroupCommService.FailedEvaluatorHandler", "onNext", failedEvaluator.getId());
+      groupCommDriver.getGroupCommFailedEvaluatorStage().onNext(failedEvaluator);
+      LOG.exiting("GroupCommService.FailedEvaluatorHandler", "onNext", failedEvaluator.getId());
+    }
+
+  }
+
+  public class RunningTaskHandler implements EventHandler<RunningTask> {
+
+    @Override
+    public void onNext(final RunningTask runningTask) {
+      LOG.entering("GroupCommService.RunningTaskHandler", "onNext", runningTask.getId());
+      groupCommDriver.getGroupCommRunningTaskStage().onNext(runningTask);
+      LOG.exiting("GroupCommService.RunningTaskHandler", "onNext", runningTask.getId());
+    }
+
+  }
+
+  public class FailedTaskHandler implements EventHandler<FailedTask> {
+
+    @Override
+    public void onNext(final FailedTask failedTask) {
+      LOG.entering("GroupCommService.FailedTaskHandler", "onNext", failedTask.getId());
+      groupCommDriver.getGroupCommFailedTaskStage().onNext(failedTask);
+      LOG.exiting("GroupCommService.FailedTaskHandler", "onNext", failedTask.getId());
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/IndexedMsg.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/IndexedMsg.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/IndexedMsg.java
new file mode 100644
index 0000000..b72979c
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/IndexedMsg.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.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.tang.annotations.Name;
+
+/**
+ * Helper class to wrap msg and the operator name in the msg
+ */
+public class IndexedMsg {
+  private final Class<? extends Name<String>> operName;
+  private final GroupCommunicationMessage msg;
+
+  public IndexedMsg(final GroupCommunicationMessage msg) {
+    super();
+    this.operName = Utils.getClass(msg.getOperatorname());
+    this.msg = msg;
+  }
+
+  public Class<? extends Name<String>> getOperName() {
+    return operName;
+  }
+
+  public GroupCommunicationMessage getMsg() {
+    return msg;
+  }
+
+  @Override
+  public int hashCode() {
+    return operName.getName().hashCode();
+  }
+
+  @Override
+  public boolean equals(final Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof IndexedMsg)) {
+      return false;
+    }
+    final IndexedMsg that = (IndexedMsg) obj;
+    if (this.operName == that.operName) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return operName.getSimpleName();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/MsgKey.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/MsgKey.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/MsgKey.java
new file mode 100644
index 0000000..9ac021e
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/driver/MsgKey.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.impl.driver;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+
+/**
+ * The key object used in map to aggregate msgs from
+ * all the operators before updating state on driver
+ */
+public class MsgKey {
+  private final String src;
+  private final String dst;
+  private final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType;
+
+  public MsgKey (final String src, final String dst, final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) {
+    this.src = src;
+    this.dst = dst;
+    this.msgType = msgType;
+  }
+
+  public MsgKey (final GroupCommunicationMessage msg) {
+    this.src = msg.getSrcid() + ":" + msg.getSrcVersion();
+    this.dst = msg.getDestid() + ":" + msg.getVersion();
+    this.msgType = msg.getType();
+  }
+
+  public String getSrc () {
+    return src.split(":",2)[0];
+  }
+
+  public String getDst () {
+    return dst.split(":",2)[0];
+  }
+
+  public ReefNetworkGroupCommProtos.GroupCommMessage.Type getMsgType () {
+    return msgType;
+  }
+
+  @Override
+  public String toString () {
+    return "(" + src + "," + dst + "," + msgType + ")";
+  }
+
+  @Override
+  public boolean equals (final Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof MsgKey)) {
+      return false;
+    }
+    final MsgKey that = (MsgKey) obj;
+    if (!this.src.equals(that.src)) {
+      return false;
+    }
+    if (!this.dst.equals(that.dst)) {
+      return false;
+    }
+    if (!this.msgType.equals(that.msgType)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode () {
+    int result = src.hashCode();
+    result = 31 * result + dst.hashCode();
+    result = 31 * result + msgType.hashCode();
+    return result;
+  }
+}


[5/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/ReduceScatter.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/ReduceScatter.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/ReduceScatter.java
new file mode 100644
index 0000000..792b655
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/ReduceScatter.java
@@ -0,0 +1,67 @@
+/**
+ * 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.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.wake.Identifier;
+
+import java.util.List;
+
+/**
+ * MPI Reduce Scatter operator.
+ * <p/>
+ * Each task has a list of elements. Assume that each task reduces
+ * each element in the list to form a list of reduced elements at a dummy root.
+ * The dummy root then keeps the portion of the list assigned to it and
+ * scatters the remaining among the other tasks
+ */
+public interface ReduceScatter<T> extends GroupCommOperator {
+
+  /**
+   * Apply this operation on elements where counts specify the distribution of
+   * elements to each task. Ordering is assumed to be default.
+   * <p/>
+   * Here counts is of the same size as the entire group not just children.
+   *
+   * @return List of values that result from applying reduce function on
+   * corresponding elements of each list received as a result of
+   * applying scatter.
+   */
+  List<T> apply(List<T> elements, List<Integer> counts) throws InterruptedException, NetworkException;
+
+  /**
+   * Apply this operation on elements where counts specify the distribution of
+   * elements to each task. Ordering is specified using order
+   * <p/>
+   * Here counts is of the same size as the entire group not just children
+   *
+   * @return List of values that result from applying reduce function on
+   * corresponding elements of each list received as a result of
+   * applying scatter.
+   */
+  List<T> apply(List<T> elements, List<Integer> counts,
+                List<? extends Identifier> order) throws InterruptedException, NetworkException;
+
+  /**
+   * get {@link org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction} configured
+   *
+   * @return {@link org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction}
+   */
+  Reduce.ReduceFunction<T> getReduceFunction();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Scatter.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Scatter.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Scatter.java
new file mode 100644
index 0000000..6c58d61
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/Scatter.java
@@ -0,0 +1,74 @@
+/**
+ * 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.reef.io.network.group.api.operators;
+
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.wake.Identifier;
+
+import java.util.List;
+
+/**
+ * MPI Scatter operator
+ * <p/>
+ * Scatter a list of elements to the receivers The receivers will receive a
+ * sub-list of elements targeted for them. Supports non-uniform distribution
+ * through the specification of counts
+ */
+public interface Scatter {
+
+  /**
+   * Sender or Root.
+   */
+  static interface Sender<T> extends GroupCommOperator {
+
+    /**
+     * Distributes evenly across task ids sorted lexicographically.
+     */
+    void send(List<T> elements) throws NetworkException, InterruptedException;
+
+    /**
+     * Distributes as per counts across task ids sorted lexicographically.
+     */
+    void send(List<T> elements, Integer... counts) throws NetworkException, InterruptedException;
+
+    /**
+     * Distributes evenly across task ids sorted using order.
+     */
+    void send(List<T> elements, List<? extends Identifier> order)
+        throws NetworkException, InterruptedException;
+
+    /**
+     * Distributes as per counts across task ids sorted using order.
+     */
+    void send(List<T> elements, List<Integer> counts,
+              List<? extends Identifier> order) throws NetworkException, InterruptedException;
+  }
+
+  /**
+   * Receiver or non-roots.
+   */
+  static interface Receiver<T> extends GroupCommOperator {
+    /**
+     * Receive the sub-list of elements targeted for the current receiver.
+     *
+     * @return list of elements targeted for the current receiver.
+     */
+    List<T> receive() throws InterruptedException, NetworkException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/package-info.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/package-info.java
new file mode 100644
index 0000000..d2e600c
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/operators/package-info.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.
+ */
+
+/**
+ * Provides the interfaces for MPI style group communication operations.
+ * The interface is asymmetric for asymmetric operations and symmetric
+ * for symmetric operations unlike MPI which provides symmetric operations
+ * for all operations.
+ *
+ * The interface is asymmetric in the sense that Senders & Receivers are
+ * separated out for operations like Scatter and Gather. All participants
+ * do not execute the same function. A sender sends & a receiver receives.
+ *
+ * The interface only concentrates on the data part because we are on the
+ * data-plane of things in REEF. The control information is embedded in the
+ * {@link org.apache.reef.tang.Configuration} used to instantiate these
+ * operators. It is the responsibility of the Driver, the primary agent in
+ * the control-plane to configure these operators, that is, denote who is
+ * the sender, who are the receivers, what {@link org.apache.reef.io.serialization.Codec}
+ * need to be used and so on for an operation like Scatter with the root node
+ * acting as a sender and the other nodes as receivers.
+ *
+ * One thing implicit in MPI operations is the ordering of processors based
+ * on their ranks which determines the order of operations. For ex., if we
+ * scatter an array of 10 elements into 10 processors, then which processor
+ * gets the 1st entry & so on is based on the rank.
+ *
+ * In our case we do not have any ranks associated with tasks. Instead,
+ * by default we use the lexicographic order of the task ids. These can
+ * also be over-ridden in the send/receive/apply function calls
+ */
+package org.apache.reef.io.network.group.api.operators;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommGroupNetworkHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommGroupNetworkHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommGroupNetworkHandler.java
new file mode 100644
index 0000000..45b1839
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommGroupNetworkHandler.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.reef.io.network.group.api.task;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.task.CommGroupNetworkHandlerImpl;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EventHandler;
+
+/**
+ * The EventHandler that receives the GroupCommunicationMsg
+ * pertaining to a specific Communication Group
+ */
+@DefaultImplementation(value = CommGroupNetworkHandlerImpl.class)
+public interface CommGroupNetworkHandler extends EventHandler<GroupCommunicationMessage> {
+
+  void register(Class<? extends Name<String>> operName, EventHandler<GroupCommunicationMessage> handler);
+
+  void addTopologyElement(Class<? extends Name<String>> operName);
+
+  GroupCommunicationMessage waitForTopologyUpdate(Class<? extends Name<String>> operName);
+
+  byte[] waitForTopologyChanges(Class<? extends Name<String>> operName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupClient.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupClient.java
new file mode 100644
index 0000000..a1370e5
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupClient.java
@@ -0,0 +1,97 @@
+/**
+ * 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.reef.io.network.group.api.task;
+
+import org.apache.reef.annotations.audience.TaskSide;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.network.group.impl.task.CommunicationGroupClientImpl;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.annotations.Name;
+
+/**
+ * The Task side interface of a communication group
+ * Lets one get the operators configured for this task
+ * and use them for communication between tasks configured
+ * in this communication group
+ */
+@TaskSide
+@DefaultImplementation(value = CommunicationGroupClientImpl.class)
+public interface CommunicationGroupClient {
+
+  /**
+   * @return The name configured on this communication group
+   */
+  Class<? extends Name<String>> getName();
+
+  /**
+   * The broadcast sender configured on this communication group
+   * with the given oepratorName
+   *
+   * @param operatorName
+   * @return
+   */
+  Broadcast.Sender getBroadcastSender(Class<? extends Name<String>> operatorName);
+
+  /**
+   * The broadcast receiver configured on this communication group
+   * with the given oepratorName
+   *
+   * @param operatorName
+   * @return
+   */
+  Broadcast.Receiver getBroadcastReceiver(Class<? extends Name<String>> operatorName);
+
+  /**
+   * The reduce receiver configured on this communication group
+   * with the given oepratorName
+   *
+   * @param operatorName
+   * @return
+   */
+  Reduce.Receiver getReduceReceiver(Class<? extends Name<String>> operatorName);
+
+  /**
+   * The reduce sender configured on this communication group
+   * with the given oepratorName
+   *
+   * @param operatorName
+   * @return
+   */
+  Reduce.Sender getReduceSender(Class<? extends Name<String>> operatorName);
+
+  /**
+   * @return Changes in topology of this communication group since the last time
+   * this method was called
+   */
+  GroupChanges getTopologyChanges();
+
+  /**
+   * Asks the driver to update the topology of this communication group. This can
+   * be an expensive call depending on what the minimum number of tasks is for this
+   * group to function as this first tells the driver, driver then tells the affected
+   * tasks and the driver gives a green only after affected tasks have had a chance
+   * to be sure that their topology will be updated before the next message is
+   * communicated
+   */
+  void updateTopology();
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupServiceClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupServiceClient.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupServiceClient.java
new file mode 100644
index 0000000..b75af08
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/CommunicationGroupServiceClient.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.reef.io.network.group.api.task;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.io.network.group.impl.task.CommunicationGroupClientImpl;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+
+@Private
+@DefaultImplementation(value = CommunicationGroupClientImpl.class)
+public interface CommunicationGroupServiceClient extends CommunicationGroupClient {
+  /**
+   * Should not be used by user code
+   * Used for initialization of the
+   * communication group
+   */
+  void initialize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommClient.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommClient.java
new file mode 100644
index 0000000..14c811b
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommClient.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.api.task;
+
+import org.apache.reef.annotations.Provided;
+import org.apache.reef.annotations.audience.TaskSide;
+import org.apache.reef.io.network.group.impl.task.GroupCommClientImpl;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.annotations.Name;
+
+
+/**
+ * The task side interface for the Group Communication Service
+ */
+@TaskSide
+@Provided
+@DefaultImplementation(value = GroupCommClientImpl.class)
+public interface GroupCommClient {
+
+  /**
+   * @param string
+   * @return The communication group client with the given name that gives access
+   * to the operators configured on it that will be used to do group communication
+   */
+  CommunicationGroupClient getCommunicationGroup(Class<? extends Name<String>> groupName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommNetworkHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommNetworkHandler.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommNetworkHandler.java
new file mode 100644
index 0000000..ecea973
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/GroupCommNetworkHandler.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.reef.io.network.group.api.task;
+
+import org.apache.reef.annotations.audience.TaskSide;
+import org.apache.reef.io.network.Message;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.task.GroupCommNetworkHandlerImpl;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.wake.EventHandler;
+
+/**
+ * The global EventHandler that receives the GroupCommunicationMsg
+ * and routes it to the relevant communication group
+ */
+@TaskSide
+@DefaultImplementation(value = GroupCommNetworkHandlerImpl.class)
+public interface GroupCommNetworkHandler extends EventHandler<Message<GroupCommunicationMessage>> {
+
+  void register(Class<? extends Name<String>> groupName, EventHandler<GroupCommunicationMessage> commGroupNetworkHandler);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/NodeStruct.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/NodeStruct.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/NodeStruct.java
new file mode 100644
index 0000000..08554f8
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/NodeStruct.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.io.network.group.api.task;
+
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+
+/**
+ * The actual node that is part of the operator topology
+ *
+ * Receives data from the handlers & provides them to the
+ * operators/OperatorTopologyStruct when they need it.
+ *
+ * This implementation decouples the send & receive.
+ */
+public interface NodeStruct {
+
+  String getId();
+
+  int getVersion();
+
+  void setVersion(int version);
+
+  byte[] getData();
+
+  void addData(GroupCommunicationMessage msg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopology.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopology.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopology.java
new file mode 100644
index 0000000..62b6934
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopology.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.reef.io.network.group.api.task;
+
+import org.apache.reef.io.network.exception.ParentDeadException;
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+
+/**
+ * Represents the local topology of tasks for an operator. It
+ * provides methods to send/rcv from parents & children
+ * <p/>
+ * Every operator is an EventHandler<GroupCommunicationMessage>
+ * and it will use an instance of this type to delegate the
+ * handling of the message and also uses it to communicate
+ * with its parents and children
+ * <p/>
+ * This is an operator facing interface. The actual topology is
+ * maintained in OperatorTopologyStruct. Current strategy is to
+ * maintain two versions of the topology and current operations
+ * are always delegated to effectiveTopology and the baseTopology
+ * is updated while initialization & when user calls updateTopology.
+ * So this is only a wrapper around the two versions of topologies
+ * and manages when to create/update them based on the messages from
+ * the driver.
+ */
+public interface OperatorTopology {
+
+  void handle(GroupCommunicationMessage msg);
+
+  void sendToParent(byte[] encode, ReefNetworkGroupCommProtos.GroupCommMessage.Type reduce) throws ParentDeadException;
+
+  byte[] recvFromParent() throws ParentDeadException;
+
+  void sendToChildren(byte[] data, ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType) throws ParentDeadException;
+
+  <T> T recvFromChildren(ReduceFunction<T> redFunc, Codec<T> dataCodec) throws ParentDeadException;
+
+  void initialize() throws ParentDeadException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopologyStruct.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopologyStruct.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopologyStruct.java
new file mode 100644
index 0000000..dd262c9
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/api/task/OperatorTopologyStruct.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.reef.io.network.group.api.task;
+
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.group.impl.GroupCommunicationMessage;
+import org.apache.reef.io.network.group.impl.operators.Sender;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+
+import java.util.Collection;
+import java.util.Set;
+
+/**
+ * The actual local topology maintaining the
+ * children and parent that reacts to update
+ * and data msgs. The actual nodes are represented
+ * by NodeStruct and it handles receiving &
+ * providing data
+ */
+public interface OperatorTopologyStruct {
+
+  Class<? extends Name<String>> getGroupName();
+
+  Class<? extends Name<String>> getOperName();
+
+  String getSelfId();
+
+  int getVersion();
+
+  NodeStruct getParent();
+
+  Collection<? extends NodeStruct> getChildren();
+
+  String getDriverId();
+
+  Sender getSender();
+
+  boolean hasChanges();
+
+  void setChanges(boolean b);
+
+  void addAsData(GroupCommunicationMessage msg);
+
+  void update(Set<GroupCommunicationMessage> deletionDeltas);
+
+  void update(GroupCommunicationMessage msg);
+
+  void sendToParent(byte[] data, ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType);
+
+  byte[] recvFromParent();
+
+  void sendToChildren(byte[] data, ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType);
+
+  <T> T recvFromChildren(ReduceFunction<T> redFunc, Codec<T> dataCodec);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesCodec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesCodec.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesCodec.java
new file mode 100644
index 0000000..46eb79e
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesCodec.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.reef.io.network.group.impl;
+
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.serialization.Codec;
+
+import javax.inject.Inject;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class GroupChangesCodec implements Codec<GroupChanges> {
+
+  @Inject
+  public GroupChangesCodec() {
+  }
+
+  @Override
+  public GroupChanges decode(final byte[] changeBytes) {
+    return new GroupChangesImpl(changeBytes[0] == 1);
+  }
+
+  @Override
+  public byte[] encode(final GroupChanges changes) {
+    final byte[] retVal = new byte[1];
+    if (changes.exist()) {
+      retVal[0] = 1;
+    }
+    return retVal;
+  }
+
+  public static void main(final String[] args) {
+    GroupChanges changes = new GroupChangesImpl(false);
+    final GroupChangesCodec changesCodec = new GroupChangesCodec();
+    GroupChanges changes1 = changesCodec.decode(changesCodec.encode(changes));
+    test(changes, changes1);
+    changes = new GroupChangesImpl(true);
+    changes1 = changesCodec.decode(changesCodec.encode(changes));
+    test(changes, changes1);
+  }
+
+  private static void test(final GroupChanges changes, final GroupChanges changes1) {
+
+    final Logger LOG = Logger.getLogger(GroupChangesCodec.class.getName());
+
+    final boolean c1 = changes.exist();
+    final boolean c2 = changes1.exist();
+
+    if (c1 != c2) {
+      LOG.log(Level.SEVERE, "Something is wrong: {0} != {1}", new Object[] {c1, c2});
+    } else {
+      LOG.log(Level.INFO, "Codec is fine");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesImpl.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesImpl.java
new file mode 100644
index 0000000..48da7e7
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupChangesImpl.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.reef.io.network.group.impl;
+
+import org.apache.reef.io.network.group.api.GroupChanges;
+
+import javax.annotation.concurrent.Immutable;
+import javax.annotation.concurrent.ThreadSafe;
+
+@Immutable
+@ThreadSafe
+public class GroupChangesImpl implements GroupChanges {
+
+  private final boolean changes;
+
+  public GroupChangesImpl(final boolean changes) {
+    this.changes = changes;
+  }
+
+  @Override
+  public boolean exist() {
+    return changes;
+  }
+
+  @Override
+  public String toString() {
+    return "Changes: " + changes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessage.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessage.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessage.java
new file mode 100644
index 0000000..c02d6af
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessage.java
@@ -0,0 +1,167 @@
+/**
+ * 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.reef.io.network.group.impl;
+
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos;
+
+import java.util.Arrays;
+
+/**
+ *
+ */
+public class GroupCommunicationMessage {
+  private final String groupName;
+  private final String operName;
+  private final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType;
+  private final String from;
+  private final int srcVersion;
+  private final String to;
+  private final int dstVersion;
+  private final byte[][] data;
+
+  private final String simpleGroupName;
+  private final String simpleOperName;
+
+  public GroupCommunicationMessage(
+      final String groupName,
+      final String operName,
+      final ReefNetworkGroupCommProtos.GroupCommMessage.Type msgType,
+      final String from, final int srcVersion,
+      final String to, final int dstVersion,
+      final byte[][] data) {
+    super();
+    this.groupName = groupName;
+    this.operName = operName;
+    this.msgType = msgType;
+    this.from = from;
+    this.srcVersion = srcVersion;
+    this.to = to;
+    this.dstVersion = dstVersion;
+    this.data = data;
+    this.simpleGroupName = Utils.simpleName(Utils.getClass(groupName));
+    this.simpleOperName = Utils.simpleName(Utils.getClass(operName));
+  }
+
+  public String getGroupname() {
+    return groupName;
+  }
+
+  public String getOperatorname() {
+    return operName;
+  }
+
+  public String getSimpleOperName() {
+    return simpleOperName;
+  }
+
+  public ReefNetworkGroupCommProtos.GroupCommMessage.Type getType() {
+    return msgType;
+  }
+
+  public String getSrcid() {
+    return from;
+  }
+
+  public int getSrcVersion() {
+    return srcVersion;
+  }
+
+  public String getDestid() {
+    return to;
+  }
+
+  public int getVersion() {
+    return dstVersion;
+  }
+
+  public String getSource() {
+    return "(" + getSrcid() + "," + getSrcVersion() + ")";
+  }
+
+  public String getDestination() {
+    return "(" + getDestid() + "," + getVersion() + ")";
+  }
+
+  public byte[][] getData() {
+    return data;
+  }
+
+  public int getMsgsCount() {
+    return data.length;
+  }
+
+  public boolean hasVersion() {
+    return true;
+  }
+
+  public boolean hasSrcVersion() {
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "[" + msgType + " from " + getSource() + " to " + getDestination() + " for " + simpleGroupName + ":" + simpleOperName + "]";
+  }
+
+  @Override
+  public boolean equals(final Object obj) {
+    if (this != obj) {
+      if (obj instanceof GroupCommunicationMessage) {
+        final GroupCommunicationMessage that = (GroupCommunicationMessage) obj;
+        if (!this.groupName.equals(that.groupName)) {
+          return false;
+        }
+        if (!this.operName.equals(that.operName)) {
+          return false;
+        }
+        if (!this.from.equals(that.from)) {
+          return false;
+        }
+        if (this.srcVersion != that.srcVersion) {
+          return false;
+        }
+        if (!this.to.equals(that.to)) {
+          return false;
+        }
+        if (this.dstVersion != that.dstVersion) {
+          return false;
+        }
+        if (!this.msgType.equals(that.msgType)) {
+          return false;
+        }
+        if (this.data.length != that.data.length) {
+          return false;
+        }
+        for (int i = 0; i < data.length; i++) {
+          if (!Arrays.equals(this.data[i], that.data[i])) {
+            return false;
+          }
+        }
+
+        return true;
+      } else {
+        return false;
+      }
+    } else {
+      return true;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessageCodec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessageCodec.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessageCodec.java
new file mode 100644
index 0000000..8b5225d
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/GroupCommunicationMessageCodec.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.reef.io.network.group.impl;
+
+
+import org.apache.reef.io.network.impl.StreamingCodec;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos.GroupCommMessage;
+import org.apache.reef.io.network.proto.ReefNetworkGroupCommProtos.GroupCommMessage.Type;
+
+import javax.inject.Inject;
+import java.io.*;
+
+/**
+ * Codec for {@link GroupCommMessage}
+ */
+public class GroupCommunicationMessageCodec implements StreamingCodec<GroupCommunicationMessage> {
+
+  @Inject
+  public GroupCommunicationMessageCodec() {
+    // Intentionally Blank
+  }
+
+  @Override
+  public GroupCommunicationMessage decode(final byte[] data) {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(data)) {
+      try (DataInputStream dais = new DataInputStream(bais)) {
+        return decodeFromStream(dais);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException("IOException", e);
+    }
+  }
+
+  @Override
+  public GroupCommunicationMessage decodeFromStream(final DataInputStream stream) {
+    try {
+      final String groupName = stream.readUTF();
+      final String operName = stream.readUTF();
+      final Type msgType = Type.valueOf(stream.readInt());
+      final String from = stream.readUTF();
+      final int srcVersion = stream.readInt();
+      final String to = stream.readUTF();
+      final int dstVersion = stream.readInt();
+      final byte[][] gcmData = new byte[stream.readInt()][];
+      for (int i = 0; i < gcmData.length; i++) {
+        gcmData[i] = new byte[stream.readInt()];
+        stream.readFully(gcmData[i]);
+      }
+      return new GroupCommunicationMessage(
+          groupName,
+          operName,
+          msgType,
+          from,
+          srcVersion,
+          to,
+          dstVersion,
+          gcmData);
+    } catch (final IOException e) {
+      throw new RuntimeException("IOException", e);
+    }
+  }
+
+  @Override
+  public byte[] encode(final GroupCommunicationMessage msg) {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+      try (DataOutputStream daos = new DataOutputStream(baos)) {
+        encodeToStream(msg, daos);
+      }
+      return baos.toByteArray();
+    } catch (final IOException e) {
+      throw new RuntimeException("IOException", e);
+    }
+  }
+
+  @Override
+  public void encodeToStream(final GroupCommunicationMessage msg, final DataOutputStream stream) {
+    try {
+      stream.writeUTF(msg.getGroupname());
+      stream.writeUTF(msg.getOperatorname());
+      stream.writeInt(msg.getType().getNumber());
+      stream.writeUTF(msg.getSrcid());
+      stream.writeInt(msg.getSrcVersion());
+      stream.writeUTF(msg.getDestid());
+      stream.writeInt(msg.getVersion());
+      stream.writeInt(msg.getMsgsCount());
+      for (final byte[] b : msg.getData()) {
+        stream.writeInt(b.length);
+        stream.write(b);
+      }
+    } catch (final IOException e) {
+      throw new RuntimeException("IOException", e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/BroadcastOperatorSpec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/BroadcastOperatorSpec.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/BroadcastOperatorSpec.java
new file mode 100644
index 0000000..11514b1
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/BroadcastOperatorSpec.java
@@ -0,0 +1,86 @@
+/**
+ * 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.reef.io.network.group.impl.config;
+
+import org.apache.reef.io.network.group.api.config.OperatorSpec;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.serialization.Codec;
+
+
+/**
+ * The specification for the broadcast operator
+ */
+public class BroadcastOperatorSpec implements OperatorSpec {
+  private final String senderId;
+
+  /**
+   * Codec to be used to serialize data
+   */
+  private final Class<? extends Codec> dataCodecClass;
+
+
+  public BroadcastOperatorSpec(final String senderId,
+                               final Class<? extends Codec> dataCodecClass) {
+    super();
+    this.senderId = senderId;
+    this.dataCodecClass = dataCodecClass;
+  }
+
+  public String getSenderId() {
+    return senderId;
+  }
+
+  @Override
+  public Class<? extends Codec> getDataCodecClass() {
+    return dataCodecClass;
+  }
+
+  @Override
+  public String toString() {
+    return "Broadcast Operator Spec: [sender=" + senderId + "] [dataCodecClass=" + Utils.simpleName(dataCodecClass)
+        + "]";
+  }
+
+  public static Builder newBuilder() {
+    return new BroadcastOperatorSpec.Builder();
+  }
+
+  public static class Builder implements org.apache.reef.util.Builder<BroadcastOperatorSpec> {
+    private String senderId;
+
+    private Class<? extends Codec> dataCodecClass;
+
+
+    public Builder setSenderId(final String senderId) {
+      this.senderId = senderId;
+      return this;
+    }
+
+    public Builder setDataCodecClass(final Class<? extends Codec> codecClazz) {
+      this.dataCodecClass = codecClazz;
+      return this;
+    }
+
+    @Override
+    public BroadcastOperatorSpec build() {
+      return new BroadcastOperatorSpec(senderId, dataCodecClass);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/ReduceOperatorSpec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/ReduceOperatorSpec.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/ReduceOperatorSpec.java
new file mode 100644
index 0000000..386ad13
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/ReduceOperatorSpec.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.reef.io.network.group.impl.config;
+
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.group.api.config.OperatorSpec;
+import org.apache.reef.io.network.group.impl.utils.Utils;
+import org.apache.reef.io.serialization.Codec;
+
+/**
+ * The specification for the Reduce operator
+ */
+public class ReduceOperatorSpec implements OperatorSpec {
+
+  private final String receiverId;
+
+  /**
+   * Codec to be used to serialize data
+   */
+  private final Class<? extends Codec> dataCodecClass;
+
+  /**
+   * The reduce function to be used for operations that do reduction
+   */
+  public final Class<? extends ReduceFunction> redFuncClass;
+
+
+  public ReduceOperatorSpec(final String receiverId,
+                            final Class<? extends Codec> dataCodecClass,
+                            final Class<? extends ReduceFunction> redFuncClass) {
+    super();
+    this.receiverId = receiverId;
+    this.dataCodecClass = dataCodecClass;
+    this.redFuncClass = redFuncClass;
+  }
+
+  public String getReceiverId() {
+    return receiverId;
+  }
+
+  /**
+   * @return the redFuncClass
+   */
+  public Class<? extends ReduceFunction> getRedFuncClass() {
+    return redFuncClass;
+  }
+
+  @Override
+  public Class<? extends Codec> getDataCodecClass() {
+    return dataCodecClass;
+  }
+
+  @Override
+  public String toString() {
+    return "Reduce Operator Spec: [receiver=" + receiverId + "] [dataCodecClass=" + Utils.simpleName(dataCodecClass)
+        + "] [reduceFunctionClass=" + Utils.simpleName(redFuncClass) + "]";
+  }
+
+  public static Builder newBuilder() {
+    return new ReduceOperatorSpec.Builder();
+  }
+
+  public static class Builder implements org.apache.reef.util.Builder<ReduceOperatorSpec> {
+
+    private String receiverId;
+
+    private Class<? extends Codec> dataCodecClass;
+
+    private Class<? extends ReduceFunction> redFuncClass;
+
+    public Builder setReceiverId(final String receiverId) {
+      this.receiverId = receiverId;
+      return this;
+    }
+
+    public Builder setDataCodecClass(final Class<? extends Codec> codecClazz) {
+      this.dataCodecClass = codecClazz;
+      return this;
+    }
+
+    public Builder setReduceFunctionClass(final Class<? extends ReduceFunction> redFuncClass) {
+      this.redFuncClass = redFuncClass;
+      return this;
+    }
+
+    @Override
+    public ReduceOperatorSpec build() {
+      return new ReduceOperatorSpec(receiverId, dataCodecClass, redFuncClass);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/CommunicationGroupName.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/CommunicationGroupName.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/CommunicationGroupName.java
new file mode 100644
index 0000000..49dab96
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/CommunicationGroupName.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter(doc = "Name of the comm group")
+public final class CommunicationGroupName implements Name<String> {
+  private CommunicationGroupName() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/DataCodec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/DataCodec.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/DataCodec.java
new file mode 100644
index 0000000..82613a7
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/DataCodec.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter(doc = "Codec used to serialize and deserialize data in operators")
+public final class DataCodec implements Name<Codec> {
+  private DataCodec() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/OperatorName.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/OperatorName.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/OperatorName.java
new file mode 100644
index 0000000..9098577
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/OperatorName.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter(doc = "Name of the operator")
+public final class OperatorName implements Name<String> {
+  private OperatorName() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/ReduceFunctionParam.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/ReduceFunctionParam.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/ReduceFunctionParam.java
new file mode 100644
index 0000000..d37408b
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/ReduceFunctionParam.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter(doc = "The reduce function class that is associated with a reduce operator")
+public final class ReduceFunctionParam implements Name<ReduceFunction> {
+  private ReduceFunctionParam() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedGroupConfigs.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedGroupConfigs.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedGroupConfigs.java
new file mode 100644
index 0000000..1844766
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedGroupConfigs.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+import java.util.Set;
+
+@NamedParameter(doc = "Serialized communication group configurations")
+public final class SerializedGroupConfigs implements Name<Set<String>> {
+  private SerializedGroupConfigs() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedOperConfigs.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedOperConfigs.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedOperConfigs.java
new file mode 100644
index 0000000..e0103f5
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/SerializedOperConfigs.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+import java.util.Set;
+
+@NamedParameter(doc = "Serialized operator configurations")
+public final class SerializedOperConfigs implements Name<Set<String>> {
+  private SerializedOperConfigs() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TaskVersion.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TaskVersion.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TaskVersion.java
new file mode 100644
index 0000000..b1caa84
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TaskVersion.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter(doc = "The version that this task is assigned")
+public final class TaskVersion implements Name<Integer> {
+  private TaskVersion() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TreeTopologyFanOut.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TreeTopologyFanOut.java b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TreeTopologyFanOut.java
new file mode 100644
index 0000000..c759130
--- /dev/null
+++ b/lang/java/reef-io/src/main/java/org/apache/reef/io/network/group/impl/config/parameters/TreeTopologyFanOut.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.reef.io.network.group.impl.config.parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+@NamedParameter(doc = "The fan out for the tree topology", default_value = "2", short_name = "fanout")
+public final class TreeTopologyFanOut implements Name<Integer> {
+  private TreeTopologyFanOut() {
+  }
+}


[8/8] incubator-reef git commit: [REEF-118] Add Shimoga library for elastic group communication.

Posted by we...@apache.org.
[REEF-118] Add Shimoga library for elastic group communication.

   Shimoga is REEF library for elastic group communication. It provides
   MPI-style operators like Broadcast and Reduce for inter-task messaging.

JIRA:
  [REEF-118](https://issues.apache.org/jira/browse/REEF-118)

Pull Request:
  This closes #63


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

Branch: refs/heads/master
Commit: 6c6ad33674c6e61e44015e0632023e776b07536e
Parents: 0911c08
Author: Sergiy Matusevych <mo...@apache.org>
Authored: Thu Feb 12 14:22:58 2015 -0800
Committer: Markus Weimer <we...@apache.org>
Committed: Thu Mar 5 17:52:55 2015 -0800

----------------------------------------------------------------------
 lang/java/reef-examples/pom.xml                 |  24 -
 .../reef/examples/group/bgd/BGDClient.java      | 134 +++++
 .../reef/examples/group/bgd/BGDDriver.java      | 376 ++++++++++++
 .../reef/examples/group/bgd/BGDLocal.java       |  53 ++
 .../apache/reef/examples/group/bgd/BGDYarn.java |  52 ++
 .../examples/group/bgd/ControlMessages.java     |  30 +
 .../reef/examples/group/bgd/ExampleList.java    |  72 +++
 .../group/bgd/LineSearchReduceFunction.java     |  51 ++
 .../bgd/LossAndGradientReduceFunction.java      |  55 ++
 .../reef/examples/group/bgd/MasterTask.java     | 246 ++++++++
 .../reef/examples/group/bgd/SlaveTask.java      | 204 +++++++
 .../reef/examples/group/bgd/data/Example.java   |  52 ++
 .../examples/group/bgd/data/SparseExample.java  |  68 +++
 .../examples/group/bgd/data/parser/Parser.java  |  32 +
 .../group/bgd/data/parser/SVMLightParser.java   |  98 ++++
 .../group/bgd/loss/LogisticLossFunction.java    |  50 ++
 .../examples/group/bgd/loss/LossFunction.java   |  46 ++
 .../bgd/loss/SquaredErrorLossFunction.java      |  49 ++
 .../bgd/loss/WeightedLogisticLossFunction.java  |  74 +++
 .../ControlMessageBroadcaster.java              |  29 +
 .../DescentDirectionBroadcaster.java            |  29 +
 .../LineSearchEvaluationsReducer.java           |  29 +
 .../operatornames/LossAndGradientReducer.java   |  29 +
 .../bgd/operatornames/MinEtaBroadcaster.java    |  26 +
 .../ModelAndDescentDirectionBroadcaster.java    |  29 +
 .../bgd/operatornames/ModelBroadcaster.java     |  29 +
 .../group/bgd/operatornames/package-info.java   |  23 +
 .../bgd/parameters/AllCommunicationGroup.java   |  26 +
 .../bgd/parameters/BGDControlParameters.java    | 126 ++++
 .../group/bgd/parameters/BGDLossType.java       |  61 ++
 .../group/bgd/parameters/EnableRampup.java      |  29 +
 .../reef/examples/group/bgd/parameters/Eps.java |  30 +
 .../reef/examples/group/bgd/parameters/Eta.java |  30 +
 .../group/bgd/parameters/EvaluatorMemory.java   |  29 +
 .../examples/group/bgd/parameters/InputDir.java |  29 +
 .../group/bgd/parameters/Iterations.java        |  29 +
 .../examples/group/bgd/parameters/Lambda.java   |  29 +
 .../group/bgd/parameters/LossFunctionType.java  |  30 +
 .../examples/group/bgd/parameters/MinParts.java |  29 +
 .../group/bgd/parameters/ModelDimensions.java   |  30 +
 .../group/bgd/parameters/NumSplits.java         |  30 +
 .../group/bgd/parameters/NumberOfReceivers.java |  30 +
 .../bgd/parameters/ProbabilityOfFailure.java    |  30 +
 .../ProbabilityOfSuccesfulIteration.java        |  30 +
 .../examples/group/bgd/parameters/Timeout.java  |  28 +
 .../examples/group/bgd/utils/StepSizes.java     |  59 ++
 .../group/bgd/utils/SubConfiguration.java       |  73 +++
 .../group/broadcast/BroadcastDriver.java        | 285 +++++++++
 .../examples/group/broadcast/BroadcastREEF.java | 148 +++++
 .../group/broadcast/ControlMessages.java        |  26 +
 .../examples/group/broadcast/MasterTask.java    |  97 ++++
 .../ModelReceiveAckReduceFunction.java          |  39 ++
 .../examples/group/broadcast/SlaveTask.java     |  76 +++
 .../parameters/AllCommunicationGroup.java       |  30 +
 .../parameters/ControlMessageBroadcaster.java   |  26 +
 .../group/broadcast/parameters/Dimensions.java  |  30 +
 .../parameters/FailureProbability.java          |  30 +
 .../broadcast/parameters/ModelBroadcaster.java  |  26 +
 .../parameters/ModelReceiveAckReducer.java      |  26 +
 .../broadcast/parameters/NumberOfReceivers.java |  30 +
 .../utils/math/AbstractImmutableVector.java     | 103 ++++
 .../group/utils/math/AbstractVector.java        |  61 ++
 .../examples/group/utils/math/DenseVector.java  | 112 ++++
 .../group/utils/math/ImmutableVector.java       |  78 +++
 .../examples/group/utils/math/SparseVector.java |  57 ++
 .../reef/examples/group/utils/math/Vector.java  |  72 +++
 .../examples/group/utils/math/VectorCodec.java  |  70 +++
 .../reef/examples/group/utils/math/Window.java  |  76 +++
 .../reef/examples/group/utils/timer/Timer.java  |  58 ++
 .../reef/examples/scheduler/Scheduler.java      |   5 +-
 .../utils/wake/BlockingEventHandler.java        |   2 +-
 .../utils/wake/LoggingEventHandler.java         |  17 +-
 lang/java/reef-io/pom.xml                       |   5 +
 .../reef/io/network/group/api/GroupChanges.java |  31 +
 .../network/group/api/config/OperatorSpec.java  |  38 ++
 .../api/driver/CommunicationGroupDriver.java    |  87 +++
 .../group/api/driver/GroupCommDriver.java       |  76 +++
 .../api/driver/GroupCommServiceDriver.java      |  59 ++
 .../io/network/group/api/driver/TaskNode.java   |  94 +++
 .../group/api/driver/TaskNodeStatus.java        |  81 +++
 .../io/network/group/api/driver/Topology.java   | 115 ++++
 .../operators/AbstractGroupCommOperator.java    |  44 ++
 .../network/group/api/operators/AllGather.java  |  50 ++
 .../network/group/api/operators/AllReduce.java  |  55 ++
 .../network/group/api/operators/Broadcast.java  |  60 ++
 .../io/network/group/api/operators/Gather.java  |  64 ++
 .../group/api/operators/GroupCommOperator.java  |  33 ++
 .../io/network/group/api/operators/Reduce.java  |  99 ++++
 .../group/api/operators/ReduceScatter.java      |  67 +++
 .../io/network/group/api/operators/Scatter.java |  74 +++
 .../group/api/operators/package-info.java       |  48 ++
 .../group/api/task/CommGroupNetworkHandler.java |  41 ++
 .../api/task/CommunicationGroupClient.java      |  97 ++++
 .../task/CommunicationGroupServiceClient.java   |  34 ++
 .../network/group/api/task/GroupCommClient.java |  42 ++
 .../group/api/task/GroupCommNetworkHandler.java |  38 ++
 .../io/network/group/api/task/NodeStruct.java   |  42 ++
 .../group/api/task/OperatorTopology.java        |  58 ++
 .../group/api/task/OperatorTopologyStruct.java  |  73 +++
 .../network/group/impl/GroupChangesCodec.java   |  71 +++
 .../io/network/group/impl/GroupChangesImpl.java |  45 ++
 .../group/impl/GroupCommunicationMessage.java   | 167 ++++++
 .../impl/GroupCommunicationMessageCodec.java    | 111 ++++
 .../impl/config/BroadcastOperatorSpec.java      |  86 +++
 .../group/impl/config/ReduceOperatorSpec.java   | 107 ++++
 .../parameters/CommunicationGroupName.java      |  28 +
 .../group/impl/config/parameters/DataCodec.java |  29 +
 .../impl/config/parameters/OperatorName.java    |  28 +
 .../config/parameters/ReduceFunctionParam.java  |  29 +
 .../parameters/SerializedGroupConfigs.java      |  30 +
 .../parameters/SerializedOperConfigs.java       |  30 +
 .../impl/config/parameters/TaskVersion.java     |  28 +
 .../config/parameters/TreeTopologyFanOut.java   |  28 +
 .../driver/CommunicationGroupDriverImpl.java    | 451 +++++++++++++++
 .../group/impl/driver/CtrlMsgSender.java        |  61 ++
 .../group/impl/driver/ExceptionHandler.java     |  56 ++
 .../network/group/impl/driver/FlatTopology.java | 307 ++++++++++
 .../group/impl/driver/GroupCommDriverImpl.java  | 250 ++++++++
 .../impl/driver/GroupCommMessageHandler.java    |  55 ++
 .../group/impl/driver/GroupCommService.java     | 111 ++++
 .../network/group/impl/driver/IndexedMsg.java   |  71 +++
 .../io/network/group/impl/driver/MsgKey.java    |  90 +++
 .../network/group/impl/driver/TaskNodeImpl.java | 476 +++++++++++++++
 .../group/impl/driver/TaskNodeStatusImpl.java   | 267 +++++++++
 .../io/network/group/impl/driver/TaskState.java |  23 +
 .../driver/TopologyFailedEvaluatorHandler.java  |  50 ++
 .../impl/driver/TopologyFailedTaskHandler.java  |  45 ++
 .../impl/driver/TopologyMessageHandler.java     |  44 ++
 .../impl/driver/TopologyRunningTaskHandler.java |  44 ++
 .../impl/driver/TopologyUpdateWaitHandler.java  |  94 +++
 .../network/group/impl/driver/TreeTopology.java | 345 +++++++++++
 .../network/group/impl/driver/package-info.java | 116 ++++
 .../group/impl/operators/BroadcastReceiver.java | 159 +++++
 .../group/impl/operators/BroadcastSender.java   | 141 +++++
 .../group/impl/operators/ReduceReceiver.java    | 155 +++++
 .../group/impl/operators/ReduceSender.java      | 161 ++++++
 .../io/network/group/impl/operators/Sender.java |  59 ++
 .../group/impl/task/ChildNodeStruct.java        |  42 ++
 .../impl/task/CommGroupNetworkHandlerImpl.java  | 102 ++++
 .../impl/task/CommunicationGroupClientImpl.java | 296 ++++++++++
 .../group/impl/task/GroupCommClientImpl.java    |  85 +++
 .../impl/task/GroupCommNetworkHandlerImpl.java  |  68 +++
 .../io/network/group/impl/task/InitHandler.java |  54 ++
 .../network/group/impl/task/NodeStructImpl.java |  98 ++++
 .../group/impl/task/OperatorTopologyImpl.java   | 466 +++++++++++++++
 .../impl/task/OperatorTopologyStructImpl.java   | 579 +++++++++++++++++++
 .../group/impl/task/ParentNodeStruct.java       |  45 ++
 .../impl/utils/BroadcastingEventHandler.java    |  44 ++
 .../group/impl/utils/ConcurrentCountingMap.java | 134 +++++
 .../network/group/impl/utils/CountingMap.java   |  98 ++++
 .../group/impl/utils/CountingSemaphore.java     | 103 ++++
 .../impl/utils/ResettingCountDownLatch.java     |  57 ++
 .../io/network/group/impl/utils/SetMap.java     |  95 +++
 .../reef/io/network/group/impl/utils/Utils.java |  80 +++
 .../reef/io/network/group/package-info.java     |  33 ++
 .../reef/io/network/naming/NameServer.java      |  20 +-
 .../reef/io/network/naming/NameServerImpl.java  |   1 -
 .../org/apache/reef/io/network/util/Utils.java  | 119 ++++
 .../org/apache/reef/io/storage/ram/RamMap.java  |   6 +-
 .../src/main/proto/group_comm_protocol.proto    |  64 ++
 .../GroupCommunicationMessageCodecTest.java     |  72 +++
 .../apache/reef/io/network/util/TestUtils.java  |  60 ++
 .../services/network/NetworkServiceTest.java    |  26 +-
 163 files changed, 13180 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/pom.xml
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/pom.xml b/lang/java/reef-examples/pom.xml
index f910a77..68c4693 100644
--- a/lang/java/reef-examples/pom.xml
+++ b/lang/java/reef-examples/pom.xml
@@ -214,30 +214,6 @@ under the License.
                 </plugins>
             </build>
         </profile>
-        <profile>
-            <id>MatMult</id>
-            <build>
-                <defaultGoal>exec:exec</defaultGoal>
-                <plugins>
-                    <plugin>
-                        <groupId>org.codehaus.mojo</groupId>
-                        <artifactId>exec-maven-plugin</artifactId>
-                        <configuration>
-                            <executable>java</executable>
-                            <arguments>
-                                <argument>-classpath</argument>
-                                <classpath/>
-                                <argument>-Djava.util.logging.config.class=org.apache.reef.util.logging.Config
-                                </argument>
-                                <argument>-Dcom.microsoft.reef.runtime.local.folder=${project.build.directory}
-                                </argument>
-                                <argument>org.apache.reef.examples.groupcomm.matmul.MatMultREEF</argument>
-                            </arguments>
-                        </configuration>
-                    </plugin>
-                </plugins>
-            </build>
-        </profile>
 
         <profile>
             <id>RetainedEval</id>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDClient.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDClient.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDClient.java
new file mode 100644
index 0000000..84865e8
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDClient.java
@@ -0,0 +1,134 @@
+/**
+ * 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.reef.examples.group.bgd;
+
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.reef.client.DriverConfiguration;
+import org.apache.reef.client.DriverLauncher;
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.client.REEF;
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.apache.reef.examples.group.bgd.parameters.*;
+import org.apache.reef.io.data.loading.api.DataLoadingRequestBuilder;
+import org.apache.reef.io.network.group.impl.config.parameters.TreeTopologyFanOut;
+import org.apache.reef.io.network.group.impl.driver.GroupCommService;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.JavaConfigurationBuilder;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.tang.formats.CommandLine;
+import org.apache.reef.util.EnvironmentUtils;
+
+import javax.inject.Inject;
+
+/**
+ * A client to submit BGD Jobs
+ */
+public class BGDClient {
+  private final String input;
+  private final int numSplits;
+  private final int memory;
+
+  private final BGDControlParameters bgdControlParameters;
+  private final int fanOut;
+
+  @Inject
+  public BGDClient(final @Parameter(InputDir.class) String input,
+                   final @Parameter(NumSplits.class) int numSplits,
+                   final @Parameter(EvaluatorMemory.class) int memory,
+                   final @Parameter(TreeTopologyFanOut.class) int fanOut,
+                   final BGDControlParameters bgdControlParameters) {
+    this.input = input;
+    this.fanOut = fanOut;
+    this.bgdControlParameters = bgdControlParameters;
+    this.numSplits = numSplits;
+    this.memory = memory;
+  }
+
+  /**
+   * Runs BGD on the given runtime.
+   *
+   * @param runtimeConfiguration the runtime to run on.
+   * @param jobName              the name of the job on the runtime.
+   * @return
+   */
+  public void submit(final Configuration runtimeConfiguration, final String jobName) throws Exception {
+    final Configuration driverConfiguration = getDriverConfiguration(jobName);
+    Tang.Factory.getTang().newInjector(runtimeConfiguration).getInstance(REEF.class).submit(driverConfiguration);
+  }
+
+  /**
+   * Runs BGD on the given runtime - with timeout.
+   *
+   * @param runtimeConfiguration the runtime to run on.
+   * @param jobName              the name of the job on the runtime.
+   * @param timeout              the time after which the job will be killed if not completed, in ms
+   * @return job completion status
+   */
+  public LauncherStatus run(final Configuration runtimeConfiguration,
+                            final String jobName, final int timeout) throws Exception {
+    final Configuration driverConfiguration = getDriverConfiguration(jobName);
+    return DriverLauncher.getLauncher(runtimeConfiguration).run(driverConfiguration, timeout);
+  }
+
+  private final Configuration getDriverConfiguration(final String jobName) {
+    return Configurations.merge(
+        getDataLoadConfiguration(jobName),
+        GroupCommService.getConfiguration(fanOut),
+        this.bgdControlParameters.getConfiguration());
+  }
+
+  private Configuration getDataLoadConfiguration(final String jobName) {
+    final EvaluatorRequest computeRequest = EvaluatorRequest.newBuilder()
+        .setNumber(1)
+        .setMemory(memory)
+        .build();
+    final Configuration dataLoadConfiguration = new DataLoadingRequestBuilder()
+        .setMemoryMB(memory)
+        .setInputFormatClass(TextInputFormat.class)
+        .setInputPath(input)
+        .setNumberOfDesiredSplits(numSplits)
+        .setComputeRequest(computeRequest)
+        .renewFailedEvaluators(false)
+        .setDriverConfigurationModule(EnvironmentUtils
+            .addClasspath(DriverConfiguration.CONF, DriverConfiguration.GLOBAL_LIBRARIES)
+            .set(DriverConfiguration.DRIVER_MEMORY, Integer.toString(memory))
+            .set(DriverConfiguration.ON_CONTEXT_ACTIVE, BGDDriver.ContextActiveHandler.class)
+            .set(DriverConfiguration.ON_TASK_RUNNING, BGDDriver.TaskRunningHandler.class)
+            .set(DriverConfiguration.ON_TASK_FAILED, BGDDriver.TaskFailedHandler.class)
+            .set(DriverConfiguration.ON_TASK_COMPLETED, BGDDriver.TaskCompletedHandler.class)
+            .set(DriverConfiguration.DRIVER_IDENTIFIER, jobName))
+        .build();
+    return dataLoadConfiguration;
+  }
+
+  public static final BGDClient fromCommandLine(final String[] args) throws Exception {
+    final JavaConfigurationBuilder configurationBuilder = Tang.Factory.getTang().newConfigurationBuilder();
+    final CommandLine commandLine = new CommandLine(configurationBuilder)
+        .registerShortNameOfClass(InputDir.class)
+        .registerShortNameOfClass(Timeout.class)
+        .registerShortNameOfClass(EvaluatorMemory.class)
+        .registerShortNameOfClass(NumSplits.class)
+        .registerShortNameOfClass(TreeTopologyFanOut.class);
+    BGDControlParameters.registerShortNames(commandLine);
+    commandLine.processCommandLine(args);
+    return Tang.Factory.getTang().newInjector(configurationBuilder.build()).getInstance(BGDClient.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDDriver.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDDriver.java
new file mode 100644
index 0000000..2a80581
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDDriver.java
@@ -0,0 +1,376 @@
+/**
+ * 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.reef.examples.group.bgd;
+
+import org.apache.reef.annotations.audience.DriverSide;
+import org.apache.reef.driver.context.ActiveContext;
+import org.apache.reef.driver.context.ServiceConfiguration;
+import org.apache.reef.driver.task.CompletedTask;
+import org.apache.reef.driver.task.FailedTask;
+import org.apache.reef.driver.task.RunningTask;
+import org.apache.reef.driver.task.TaskConfiguration;
+import org.apache.reef.evaluator.context.parameters.ContextIdentifier;
+import org.apache.reef.examples.group.bgd.data.parser.Parser;
+import org.apache.reef.examples.group.bgd.data.parser.SVMLightParser;
+import org.apache.reef.examples.group.bgd.loss.LossFunction;
+import org.apache.reef.examples.group.bgd.operatornames.*;
+import org.apache.reef.examples.group.bgd.parameters.AllCommunicationGroup;
+import org.apache.reef.examples.group.bgd.parameters.BGDControlParameters;
+import org.apache.reef.examples.group.bgd.parameters.ModelDimensions;
+import org.apache.reef.examples.group.bgd.parameters.ProbabilityOfFailure;
+import org.apache.reef.io.data.loading.api.DataLoadingService;
+import org.apache.reef.io.network.group.api.driver.CommunicationGroupDriver;
+import org.apache.reef.io.network.group.api.driver.GroupCommDriver;
+import org.apache.reef.io.network.group.impl.config.BroadcastOperatorSpec;
+import org.apache.reef.io.network.group.impl.config.ReduceOperatorSpec;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.io.serialization.SerializableCodec;
+import org.apache.reef.poison.PoisonedConfiguration;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.annotations.Unit;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.tang.formats.ConfigurationSerializer;
+import org.apache.reef.wake.EventHandler;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+@DriverSide
+@Unit
+public class BGDDriver {
+
+  private static final Logger LOG = Logger.getLogger(BGDDriver.class.getName());
+
+  private static final Tang TANG = Tang.Factory.getTang();
+
+  private static final double STARTUP_FAILURE_PROB = 0.01;
+
+  private final DataLoadingService dataLoadingService;
+  private final GroupCommDriver groupCommDriver;
+  private final ConfigurationSerializer confSerializer;
+  private final CommunicationGroupDriver communicationsGroup;
+  private final AtomicBoolean masterSubmitted = new AtomicBoolean(false);
+  private final AtomicInteger slaveIds = new AtomicInteger(0);
+  private final Map<String, RunningTask> runningTasks = new HashMap<>();
+  private final AtomicBoolean jobComplete = new AtomicBoolean(false);
+  private final Codec<ArrayList<Double>> lossCodec = new SerializableCodec<>();
+  private final BGDControlParameters bgdControlParameters;
+
+  private String communicationsGroupMasterContextId;
+
+  @Inject
+  public BGDDriver(final DataLoadingService dataLoadingService,
+                   final GroupCommDriver groupCommDriver,
+                   final ConfigurationSerializer confSerializer,
+                   final BGDControlParameters bgdControlParameters) {
+    this.dataLoadingService = dataLoadingService;
+    this.groupCommDriver = groupCommDriver;
+    this.confSerializer = confSerializer;
+    this.bgdControlParameters = bgdControlParameters;
+
+    final int minNumOfPartitions =
+        bgdControlParameters.isRampup()
+            ? bgdControlParameters.getMinParts()
+            : dataLoadingService.getNumberOfPartitions();
+
+    final int numParticipants = minNumOfPartitions + 1;
+
+    this.communicationsGroup = this.groupCommDriver.newCommunicationGroup(
+        AllCommunicationGroup.class, // NAME
+        numParticipants);            // Number of participants
+
+    LOG.log(Level.INFO,
+        "Obtained entire communication group: start with {0} partitions", numParticipants);
+
+    this.communicationsGroup
+        .addBroadcast(ControlMessageBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .addBroadcast(ModelBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .addReduce(LossAndGradientReducer.class,
+            ReduceOperatorSpec.newBuilder()
+                .setReceiverId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .setReduceFunctionClass(LossAndGradientReduceFunction.class)
+                .build())
+        .addBroadcast(ModelAndDescentDirectionBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .addBroadcast(DescentDirectionBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .addReduce(LineSearchEvaluationsReducer.class,
+            ReduceOperatorSpec.newBuilder()
+                .setReceiverId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .setReduceFunctionClass(LineSearchReduceFunction.class)
+                .build())
+        .addBroadcast(MinEtaBroadcaster.class,
+            BroadcastOperatorSpec.newBuilder()
+                .setSenderId(MasterTask.TASK_ID)
+                .setDataCodecClass(SerializableCodec.class)
+                .build())
+        .finalise();
+
+    LOG.log(Level.INFO, "Added operators to communicationsGroup");
+  }
+
+  final class ContextActiveHandler implements EventHandler<ActiveContext> {
+
+    @Override
+    public void onNext(final ActiveContext activeContext) {
+      LOG.log(Level.INFO, "Got active context: {0}", activeContext.getId());
+      if (jobRunning(activeContext)) {
+        if (!groupCommDriver.isConfigured(activeContext)) {
+          // The Context is not configured with the group communications service let's do that.
+          submitGroupCommunicationsService(activeContext);
+        } else {
+          // The group communications service is already active on this context. We can submit the task.
+          submitTask(activeContext);
+        }
+      }
+    }
+
+    /**
+     * @param activeContext a context to be configured with group communications.
+     */
+    private void submitGroupCommunicationsService(final ActiveContext activeContext) {
+      final Configuration contextConf = groupCommDriver.getContextConfiguration();
+      final String contextId = getContextId(contextConf);
+      final Configuration serviceConf;
+      if (!dataLoadingService.isDataLoadedContext(activeContext)) {
+        communicationsGroupMasterContextId = contextId;
+        serviceConf = groupCommDriver.getServiceConfiguration();
+      } else {
+        final Configuration parsedDataServiceConf = ServiceConfiguration.CONF
+            .set(ServiceConfiguration.SERVICES, ExampleList.class)
+            .build();
+        serviceConf = Tang.Factory.getTang()
+            .newConfigurationBuilder(groupCommDriver.getServiceConfiguration(), parsedDataServiceConf)
+            .bindImplementation(Parser.class, SVMLightParser.class)
+            .build();
+      }
+
+      LOG.log(Level.FINEST, "Submit GCContext conf: {0} and Service conf: {1}", new Object[]{
+          confSerializer.toString(contextConf), confSerializer.toString(serviceConf)});
+
+      activeContext.submitContextAndService(contextConf, serviceConf);
+    }
+
+    private void submitTask(final ActiveContext activeContext) {
+
+      assert (groupCommDriver.isConfigured(activeContext));
+
+      final Configuration partialTaskConfiguration;
+      if (activeContext.getId().equals(communicationsGroupMasterContextId) && !masterTaskSubmitted()) {
+        partialTaskConfiguration = getMasterTaskConfiguration();
+        LOG.info("Submitting MasterTask conf");
+      } else {
+        partialTaskConfiguration = getSlaveTaskConfiguration(getSlaveId(activeContext));
+        // partialTaskConfiguration = Configurations.merge(
+        //     getSlaveTaskConfiguration(getSlaveId(activeContext)),
+        //     getTaskPoisonConfiguration());
+        LOG.info("Submitting SlaveTask conf");
+      }
+      communicationsGroup.addTask(partialTaskConfiguration);
+      final Configuration taskConfiguration = groupCommDriver.getTaskConfiguration(partialTaskConfiguration);
+      LOG.log(Level.FINEST, "{0}", confSerializer.toString(taskConfiguration));
+      activeContext.submitTask(taskConfiguration);
+    }
+
+    private boolean jobRunning(final ActiveContext activeContext) {
+      synchronized (runningTasks) {
+        if (!jobComplete.get()) {
+          return true;
+        } else {
+          LOG.log(Level.INFO, "Job complete. Not submitting any task. Closing context {0}", activeContext);
+          activeContext.close();
+          return false;
+        }
+      }
+    }
+  }
+
+  final class TaskRunningHandler implements EventHandler<RunningTask> {
+
+    @Override
+    public void onNext(final RunningTask runningTask) {
+      synchronized (runningTasks) {
+        if (!jobComplete.get()) {
+          LOG.log(Level.INFO, "Job has not completed yet. Adding to runningTasks: {0}", runningTask);
+          runningTasks.put(runningTask.getId(), runningTask);
+        } else {
+          LOG.log(Level.INFO, "Job complete. Closing context: {0}", runningTask.getActiveContext().getId());
+          runningTask.getActiveContext().close();
+        }
+      }
+    }
+  }
+
+  final class TaskFailedHandler implements EventHandler<FailedTask> {
+
+    @Override
+    public void onNext(final FailedTask failedTask) {
+
+      final String failedTaskId = failedTask.getId();
+
+      LOG.log(Level.WARNING, "Got failed Task: " + failedTaskId);
+
+      if (jobRunning(failedTaskId)) {
+
+        final ActiveContext activeContext = failedTask.getActiveContext().get();
+        final Configuration partialTaskConf = getSlaveTaskConfiguration(failedTaskId);
+
+        // Do not add the task back:
+        // allCommGroup.addTask(partialTaskConf);
+
+        final Configuration taskConf = groupCommDriver.getTaskConfiguration(partialTaskConf);
+        LOG.log(Level.FINEST, "Submit SlaveTask conf: {0}", confSerializer.toString(taskConf));
+
+        activeContext.submitTask(taskConf);
+      }
+    }
+
+    private boolean jobRunning(final String failedTaskId) {
+      synchronized (runningTasks) {
+        if (!jobComplete.get()) {
+          return true;
+        } else {
+          final RunningTask rTask = runningTasks.remove(failedTaskId);
+          LOG.log(Level.INFO, "Job has completed. Not resubmitting");
+          if (rTask != null) {
+            LOG.log(Level.INFO, "Closing activecontext");
+            rTask.getActiveContext().close();
+          } else {
+            LOG.log(Level.INFO, "Master must have closed my context");
+          }
+          return false;
+        }
+      }
+    }
+  }
+
+  final class TaskCompletedHandler implements EventHandler<CompletedTask> {
+
+    @Override
+    public void onNext(final CompletedTask task) {
+      LOG.log(Level.INFO, "Got CompletedTask: {0}", task.getId());
+      final byte[] retVal = task.get();
+      if (retVal != null) {
+        final List<Double> losses = BGDDriver.this.lossCodec.decode(retVal);
+        for (final Double loss : losses) {
+          LOG.log(Level.INFO, "OUT: LOSS = {0}", loss);
+        }
+      }
+      synchronized (runningTasks) {
+        LOG.log(Level.INFO, "Acquired lock on runningTasks. Removing {0}", task.getId());
+        final RunningTask rTask = runningTasks.remove(task.getId());
+        if (rTask != null) {
+          LOG.log(Level.INFO, "Closing active context: {0}", task.getActiveContext().getId());
+          task.getActiveContext().close();
+        } else {
+          LOG.log(Level.INFO, "Master must have closed active context already for task {0}", task.getId());
+        }
+
+        if (MasterTask.TASK_ID.equals(task.getId())) {
+          jobComplete.set(true);
+          LOG.log(Level.INFO, "Master(=>Job) complete. Closing other running tasks: {0}", runningTasks.values());
+          for (final RunningTask runTask : runningTasks.values()) {
+            runTask.getActiveContext().close();
+          }
+          LOG.finest("Clearing runningTasks");
+          runningTasks.clear();
+        }
+      }
+    }
+  }
+
+  /**
+   * @return Configuration for the MasterTask
+   */
+  public Configuration getMasterTaskConfiguration() {
+    return Configurations.merge(
+        TaskConfiguration.CONF
+            .set(TaskConfiguration.IDENTIFIER, MasterTask.TASK_ID)
+            .set(TaskConfiguration.TASK, MasterTask.class)
+            .build(),
+        bgdControlParameters.getConfiguration());
+  }
+
+  /**
+   * @return Configuration for the SlaveTask
+   */
+  private Configuration getSlaveTaskConfiguration(final String taskId) {
+    final double pSuccess = bgdControlParameters.getProbOfSuccessfulIteration();
+    final int numberOfPartitions = dataLoadingService.getNumberOfPartitions();
+    final double pFailure = 1 - Math.pow(pSuccess, 1.0 / numberOfPartitions);
+    return Tang.Factory.getTang()
+        .newConfigurationBuilder(
+            TaskConfiguration.CONF
+                .set(TaskConfiguration.IDENTIFIER, taskId)
+                .set(TaskConfiguration.TASK, SlaveTask.class)
+                .build())
+        .bindNamedParameter(ModelDimensions.class, "" + bgdControlParameters.getDimensions())
+        .bindImplementation(LossFunction.class, bgdControlParameters.getLossFunction())
+        .bindNamedParameter(ProbabilityOfFailure.class, Double.toString(pFailure))
+        .build();
+  }
+
+  private Configuration getTaskPoisonConfiguration() {
+    return PoisonedConfiguration.TASK_CONF
+        .set(PoisonedConfiguration.CRASH_PROBABILITY, STARTUP_FAILURE_PROB)
+        .set(PoisonedConfiguration.CRASH_TIMEOUT, 1)
+        .build();
+  }
+
+  private String getContextId(final Configuration contextConf) {
+    try {
+      return TANG.newInjector(contextConf).getNamedInstance(ContextIdentifier.class);
+    } catch (final InjectionException e) {
+      throw new RuntimeException("Unable to inject context identifier from context conf", e);
+    }
+  }
+
+  private String getSlaveId(final ActiveContext activeContext) {
+    return "SlaveTask-" + slaveIds.getAndIncrement();
+  }
+
+  private boolean masterTaskSubmitted() {
+    return !masterSubmitted.compareAndSet(false, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDLocal.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDLocal.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDLocal.java
new file mode 100644
index 0000000..3a82314
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDLocal.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.group.bgd;
+
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.examples.group.utils.timer.Timer;
+import org.apache.reef.runtime.local.client.LocalRuntimeConfiguration;
+import org.apache.reef.tang.Configuration;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Runs BGD on the local runtime.
+ */
+public class BGDLocal {
+
+  private static final Logger LOG = Logger.getLogger(BGDLocal.class.getName());
+
+  private static final int NUM_LOCAL_THREADS = 20;
+  private static final int TIMEOUT = 10 * Timer.MINUTES;
+
+  public static void main(final String[] args) throws Exception {
+
+    final BGDClient bgdClient = BGDClient.fromCommandLine(args);
+
+    final Configuration runtimeConfiguration = LocalRuntimeConfiguration.CONF
+        .set(LocalRuntimeConfiguration.NUMBER_OF_THREADS, "" + NUM_LOCAL_THREADS)
+        .build();
+
+    final String jobName = System.getProperty("user.name") + "-" + "ResourceAwareBGDLocal";
+
+    final LauncherStatus status = bgdClient.run(runtimeConfiguration, jobName, TIMEOUT);
+
+    LOG.log(Level.INFO, "OUT: Status = {0}", status);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDYarn.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDYarn.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDYarn.java
new file mode 100644
index 0000000..19d3b10
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/BGDYarn.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.reef.examples.group.bgd;
+
+import org.apache.reef.client.LauncherStatus;
+import org.apache.reef.examples.group.utils.timer.Timer;
+import org.apache.reef.runtime.yarn.client.YarnClientConfiguration;
+import org.apache.reef.tang.Configuration;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * Runs BGD on the YARN runtime.
+ */
+public class BGDYarn {
+
+  private static final Logger LOG = Logger.getLogger(BGDYarn.class.getName());
+
+  private static final int TIMEOUT = 4 * Timer.HOURS;
+
+  public static void main(final String[] args) throws Exception {
+
+    final BGDClient bgdClient = BGDClient.fromCommandLine(args);
+
+    final Configuration runtimeConfiguration = YarnClientConfiguration.CONF
+        .set(YarnClientConfiguration.JVM_HEAP_SLACK, "0.1")
+        .build();
+
+    final String jobName = System.getProperty("user.name") + "-" + "BR-ResourceAwareBGD-YARN";
+
+    final LauncherStatus status = bgdClient.run(runtimeConfiguration, jobName, TIMEOUT);
+
+    LOG.log(Level.INFO, "OUT: Status = {0}", status);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ControlMessages.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ControlMessages.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ControlMessages.java
new file mode 100644
index 0000000..aeea56b
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ControlMessages.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.reef.examples.group.bgd;
+
+import java.io.Serializable;
+
+public enum ControlMessages implements Serializable {
+  ComputeGradientWithModel,
+  ComputeGradientWithMinEta,
+  DoLineSearch,
+  DoLineSearchWithModel,
+  Synchronize,
+  Stop
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ExampleList.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ExampleList.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ExampleList.java
new file mode 100644
index 0000000..97477a9
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/ExampleList.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.reef.examples.group.bgd;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.reef.examples.group.bgd.data.Example;
+import org.apache.reef.examples.group.bgd.data.parser.Parser;
+import org.apache.reef.io.data.loading.api.DataSet;
+import org.apache.reef.io.network.util.Pair;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ *
+ */
+public class ExampleList {
+
+  private static final Logger LOG = Logger.getLogger(ExampleList.class.getName());
+
+  private final List<Example> examples = new ArrayList<>();
+  private final DataSet<LongWritable, Text> dataSet;
+  private final Parser<String> parser;
+
+  @Inject
+  public ExampleList(final DataSet<LongWritable, Text> dataSet, final Parser<String> parser) {
+    this.dataSet = dataSet;
+    this.parser = parser;
+  }
+
+  /**
+   * @return the examples
+   */
+  public List<Example> getExamples() {
+    if (examples.isEmpty()) {
+      loadData();
+    }
+    return examples;
+  }
+
+  private void loadData() {
+    LOG.info("Loading data");
+    int i = 0;
+    for (final Pair<LongWritable, Text> examplePair : dataSet) {
+      final Example example = parser.parse(examplePair.second.toString());
+      examples.add(example);
+      if (++i % 2000 == 0) {
+        LOG.log(Level.FINE, "Done parsing {0} lines", i);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LineSearchReduceFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LineSearchReduceFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LineSearchReduceFunction.java
new file mode 100644
index 0000000..9132583
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LineSearchReduceFunction.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.group.bgd;
+
+import org.apache.reef.examples.group.utils.math.DenseVector;
+import org.apache.reef.examples.group.utils.math.Vector;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.util.Pair;
+
+import javax.inject.Inject;
+
+public class LineSearchReduceFunction implements Reduce.ReduceFunction<Pair<Vector, Integer>> {
+
+  @Inject
+  public LineSearchReduceFunction() {
+  }
+
+  @Override
+  public Pair<Vector, Integer> apply(final Iterable<Pair<Vector, Integer>> evals) {
+
+    Vector combinedEvaluations = null;
+    int numEx = 0;
+
+    for (final Pair<Vector, Integer> eval : evals) {
+      if (combinedEvaluations == null) {
+        combinedEvaluations = new DenseVector(eval.first);
+      } else {
+        combinedEvaluations.add(eval.first);
+      }
+      numEx += eval.second;
+    }
+
+    return new Pair<>(combinedEvaluations, numEx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LossAndGradientReduceFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LossAndGradientReduceFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LossAndGradientReduceFunction.java
new file mode 100644
index 0000000..cf4d0be
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/LossAndGradientReduceFunction.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.group.bgd;
+
+import org.apache.reef.examples.group.utils.math.DenseVector;
+import org.apache.reef.examples.group.utils.math.Vector;
+import org.apache.reef.io.network.group.api.operators.Reduce.ReduceFunction;
+import org.apache.reef.io.network.util.Pair;
+
+import javax.inject.Inject;
+
+public class LossAndGradientReduceFunction
+    implements ReduceFunction<Pair<Pair<Double, Integer>, Vector>> {
+
+  @Inject
+  public LossAndGradientReduceFunction() {
+  }
+
+  @Override
+  public Pair<Pair<Double, Integer>, Vector> apply(
+      final Iterable<Pair<Pair<Double, Integer>, Vector>> lags) {
+
+    double lossSum = 0.0;
+    int numEx = 0;
+    Vector combinedGradient = null;
+
+    for (final Pair<Pair<Double, Integer>, Vector> lag : lags) {
+      if (combinedGradient == null) {
+        combinedGradient = new DenseVector(lag.second);
+      } else {
+        combinedGradient.add(lag.second);
+      }
+      lossSum += lag.first.first;
+      numEx += lag.first.second;
+    }
+
+    return new Pair<>(new Pair<>(lossSum, numEx), combinedGradient);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/MasterTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/MasterTask.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/MasterTask.java
new file mode 100644
index 0000000..06ed5fd
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/MasterTask.java
@@ -0,0 +1,246 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.group.bgd;
+
+import org.apache.reef.examples.group.bgd.operatornames.*;
+import org.apache.reef.examples.group.bgd.parameters.*;
+import org.apache.reef.examples.group.bgd.utils.StepSizes;
+import org.apache.reef.examples.group.utils.math.DenseVector;
+import org.apache.reef.examples.group.utils.math.Vector;
+import org.apache.reef.examples.group.utils.timer.Timer;
+import org.apache.reef.exception.evaluator.NetworkException;
+import org.apache.reef.io.Tuple;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.GroupChanges;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupClient;
+import org.apache.reef.io.network.group.api.task.GroupCommClient;
+import org.apache.reef.io.network.util.Pair;
+import org.apache.reef.io.serialization.Codec;
+import org.apache.reef.io.serialization.SerializableCodec;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.task.Task;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class MasterTask implements Task {
+
+  public static final String TASK_ID = "MasterTask";
+
+  private static final Logger LOG = Logger.getLogger(MasterTask.class.getName());
+
+  private final CommunicationGroupClient communicationGroupClient;
+  private final Broadcast.Sender<ControlMessages> controlMessageBroadcaster;
+  private final Broadcast.Sender<Vector> modelBroadcaster;
+  private final Reduce.Receiver<Pair<Pair<Double, Integer>, Vector>> lossAndGradientReducer;
+  private final Broadcast.Sender<Pair<Vector, Vector>> modelAndDescentDirectionBroadcaster;
+  private final Broadcast.Sender<Vector> descentDriectionBroadcaster;
+  private final Reduce.Receiver<Pair<Vector, Integer>> lineSearchEvaluationsReducer;
+  private final Broadcast.Sender<Double> minEtaBroadcaster;
+  private final boolean ignoreAndContinue;
+  private final StepSizes ts;
+  private final double lambda;
+  private final int maxIters;
+  final ArrayList<Double> losses = new ArrayList<>();
+  final Codec<ArrayList<Double>> lossCodec = new SerializableCodec<ArrayList<Double>>();
+  private final Vector model;
+
+  boolean sendModel = true;
+  double minEta = 0;
+
+  @Inject
+  public MasterTask(
+      final GroupCommClient groupCommClient,
+      @Parameter(ModelDimensions.class) final int dimensions,
+      @Parameter(Lambda.class) final double lambda,
+      @Parameter(Iterations.class) final int maxIters,
+      @Parameter(EnableRampup.class) final boolean rampup,
+      final StepSizes ts) {
+
+    this.lambda = lambda;
+    this.maxIters = maxIters;
+    this.ts = ts;
+    this.ignoreAndContinue = rampup;
+    this.model = new DenseVector(dimensions);
+    this.communicationGroupClient = groupCommClient.getCommunicationGroup(AllCommunicationGroup.class);
+    this.controlMessageBroadcaster = communicationGroupClient.getBroadcastSender(ControlMessageBroadcaster.class);
+    this.modelBroadcaster = communicationGroupClient.getBroadcastSender(ModelBroadcaster.class);
+    this.lossAndGradientReducer = communicationGroupClient.getReduceReceiver(LossAndGradientReducer.class);
+    this.modelAndDescentDirectionBroadcaster = communicationGroupClient.getBroadcastSender(ModelAndDescentDirectionBroadcaster.class);
+    this.descentDriectionBroadcaster = communicationGroupClient.getBroadcastSender(DescentDirectionBroadcaster.class);
+    this.lineSearchEvaluationsReducer = communicationGroupClient.getReduceReceiver(LineSearchEvaluationsReducer.class);
+    this.minEtaBroadcaster = communicationGroupClient.getBroadcastSender(MinEtaBroadcaster.class);
+  }
+
+  @Override
+  public byte[] call(final byte[] memento) throws Exception {
+
+    double gradientNorm = Double.MAX_VALUE;
+    for (int iteration = 1; !converged(iteration, gradientNorm); ++iteration) {
+      try (final Timer t = new Timer("Current Iteration(" + (iteration) + ")")) {
+        final Pair<Double, Vector> lossAndGradient = computeLossAndGradient();
+        losses.add(lossAndGradient.first);
+        final Vector descentDirection = getDescentDirection(lossAndGradient.second);
+
+        updateModel(descentDirection);
+
+        gradientNorm = descentDirection.norm2();
+      }
+    }
+    LOG.log(Level.INFO, "OUT: Stop");
+    controlMessageBroadcaster.send(ControlMessages.Stop);
+
+    for (final Double loss : losses) {
+      LOG.log(Level.INFO, "OUT: LOSS = {0}", loss);
+    }
+    return lossCodec.encode(losses);
+  }
+
+  private void updateModel(final Vector descentDirection) throws NetworkException, InterruptedException {
+    try (final Timer t = new Timer("GetDescentDirection + FindMinEta + UpdateModel")) {
+      final Vector lineSearchEvals = lineSearch(descentDirection);
+      minEta = findMinEta(model, descentDirection, lineSearchEvals);
+      model.multAdd(minEta, descentDirection);
+    }
+
+    LOG.log(Level.INFO, "OUT: New Model = {0}", model);
+  }
+
+  private Vector lineSearch(final Vector descentDirection) throws NetworkException, InterruptedException {
+    Vector lineSearchResults = null;
+    boolean allDead = false;
+    do {
+      try (final Timer t = new Timer("LineSearch - Broadcast("
+          + (sendModel ? "ModelAndDescentDirection" : "DescentDirection") + ") + Reduce(LossEvalsInLineSearch)")) {
+        if (sendModel) {
+          LOG.log(Level.INFO, "OUT: DoLineSearchWithModel");
+          controlMessageBroadcaster.send(ControlMessages.DoLineSearchWithModel);
+          modelAndDescentDirectionBroadcaster.send(new Pair<>(model, descentDirection));
+        } else {
+          LOG.log(Level.INFO, "OUT: DoLineSearch");
+          controlMessageBroadcaster.send(ControlMessages.DoLineSearch);
+          descentDriectionBroadcaster.send(descentDirection);
+        }
+        final Pair<Vector, Integer> lineSearchEvals = lineSearchEvaluationsReducer.reduce();
+        if (lineSearchEvals != null) {
+          final int numExamples = lineSearchEvals.second;
+          lineSearchResults = lineSearchEvals.first;
+          lineSearchResults.scale(1.0 / numExamples);
+          LOG.log(Level.INFO, "OUT: #Examples: {0}", numExamples);
+          LOG.log(Level.INFO, "OUT: LineSearchEvals: {0}", lineSearchResults);
+          allDead = false;
+        } else {
+          allDead = true;
+        }
+      }
+
+      sendModel = chkAndUpdate();
+    } while (allDead || (!ignoreAndContinue && sendModel));
+    return lineSearchResults;
+  }
+
+  private Pair<Double, Vector> computeLossAndGradient() throws NetworkException, InterruptedException {
+    Pair<Double, Vector> returnValue = null;
+    boolean allDead = false;
+    do {
+      try (final Timer t = new Timer("Broadcast(" + (sendModel ? "Model" : "MinEta") + ") + Reduce(LossAndGradient)")) {
+        if (sendModel) {
+          LOG.log(Level.INFO, "OUT: ComputeGradientWithModel");
+          controlMessageBroadcaster.send(ControlMessages.ComputeGradientWithModel);
+          modelBroadcaster.send(model);
+        } else {
+          LOG.log(Level.INFO, "OUT: ComputeGradientWithMinEta");
+          controlMessageBroadcaster.send(ControlMessages.ComputeGradientWithMinEta);
+          minEtaBroadcaster.send(minEta);
+        }
+        final Pair<Pair<Double, Integer>, Vector> lossAndGradient = lossAndGradientReducer.reduce();
+
+        if (lossAndGradient != null) {
+          final int numExamples = lossAndGradient.first.second;
+          LOG.log(Level.INFO, "OUT: #Examples: {0}", numExamples);
+          final double lossPerExample = lossAndGradient.first.first / numExamples;
+          LOG.log(Level.INFO, "OUT: Loss: {0}", lossPerExample);
+          final double objFunc = ((lambda / 2) * model.norm2Sqr()) + lossPerExample;
+          LOG.log(Level.INFO, "OUT: Objective Func Value: {0}", objFunc);
+          final Vector gradient = lossAndGradient.second;
+          gradient.scale(1.0 / numExamples);
+          LOG.log(Level.INFO, "OUT: Gradient: {0}", gradient);
+          returnValue = new Pair<>(objFunc, gradient);
+          allDead = false;
+        } else {
+          allDead = true;
+        }
+      }
+      sendModel = chkAndUpdate();
+    } while (allDead || (!ignoreAndContinue && sendModel));
+    return returnValue;
+  }
+
+  private boolean chkAndUpdate() {
+    long t1 = System.currentTimeMillis();
+    final GroupChanges changes = communicationGroupClient.getTopologyChanges();
+    long t2 = System.currentTimeMillis();
+    LOG.log(Level.INFO, "OUT: Time to get TopologyChanges = " + (t2 - t1) / 1000.0 + " sec");
+    if (changes.exist()) {
+      LOG.log(Level.INFO, "OUT: There exist topology changes. Asking to update Topology");
+      t1 = System.currentTimeMillis();
+      communicationGroupClient.updateTopology();
+      t2 = System.currentTimeMillis();
+      LOG.log(Level.INFO, "OUT: Time to get TopologyChanges = " + (t2 - t1) / 1000.0 + " sec");
+      return true;
+    } else {
+      LOG.log(Level.INFO, "OUT: No changes in topology exist. So not updating topology");
+      return false;
+    }
+  }
+
+  private boolean converged(final int iters, final double gradNorm) {
+    return iters >= maxIters || Math.abs(gradNorm) <= 1e-3;
+  }
+
+  private double findMinEta(final Vector model, final Vector descentDir, final Vector lineSearchEvals) {
+    final double wNormSqr = model.norm2Sqr();
+    final double dNormSqr = descentDir.norm2Sqr();
+    final double wDotd = model.dot(descentDir);
+    final double[] t = ts.getT();
+    int i = 0;
+    for (final double eta : t) {
+      final double modelNormSqr = wNormSqr + (eta * eta) * dNormSqr + 2 * eta * wDotd;
+      final double loss = lineSearchEvals.get(i) + ((lambda / 2) * modelNormSqr);
+      lineSearchEvals.set(i, loss);
+      ++i;
+    }
+    LOG.log(Level.INFO, "OUT: Regularized LineSearchEvals: {0}", lineSearchEvals);
+    final Tuple<Integer, Double> minTup = lineSearchEvals.min();
+    LOG.log(Level.INFO, "OUT: MinTup: {0}", minTup);
+    final double minT = t[minTup.getKey()];
+    LOG.log(Level.INFO, "OUT: MinT: {0}", minT);
+    return minT;
+  }
+
+  private Vector getDescentDirection(final Vector gradient) {
+    gradient.multAdd(lambda, model);
+    gradient.scale(-1);
+    LOG.log(Level.INFO, "OUT: DescentDirection: {0}", gradient);
+    return gradient;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/SlaveTask.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/SlaveTask.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/SlaveTask.java
new file mode 100644
index 0000000..fadc16e
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/SlaveTask.java
@@ -0,0 +1,204 @@
+/**
+ * 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.reef.examples.group.bgd;
+
+import org.apache.reef.examples.group.bgd.data.Example;
+import org.apache.reef.examples.group.bgd.loss.LossFunction;
+import org.apache.reef.examples.group.bgd.operatornames.*;
+import org.apache.reef.examples.group.bgd.parameters.AllCommunicationGroup;
+import org.apache.reef.examples.group.bgd.parameters.ProbabilityOfFailure;
+import org.apache.reef.examples.group.bgd.utils.StepSizes;
+import org.apache.reef.examples.group.utils.math.DenseVector;
+import org.apache.reef.examples.group.utils.math.Vector;
+import org.apache.reef.io.network.group.api.operators.Broadcast;
+import org.apache.reef.io.network.group.api.operators.Reduce;
+import org.apache.reef.io.network.group.api.task.CommunicationGroupClient;
+import org.apache.reef.io.network.group.api.task.GroupCommClient;
+import org.apache.reef.io.network.util.Pair;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.task.Task;
+
+import javax.inject.Inject;
+import java.util.List;
+import java.util.logging.Logger;
+
+public class SlaveTask implements Task {
+
+  private static final Logger LOG = Logger.getLogger(SlaveTask.class.getName());
+
+  private final double FAILURE_PROB;
+
+  private final CommunicationGroupClient communicationGroup;
+  private final Broadcast.Receiver<ControlMessages> controlMessageBroadcaster;
+  private final Broadcast.Receiver<Vector> modelBroadcaster;
+  private final Reduce.Sender<Pair<Pair<Double, Integer>, Vector>> lossAndGradientReducer;
+  private final Broadcast.Receiver<Pair<Vector, Vector>> modelAndDescentDirectionBroadcaster;
+  private final Broadcast.Receiver<Vector> descentDirectionBroadcaster;
+  private final Reduce.Sender<Pair<Vector, Integer>> lineSearchEvaluationsReducer;
+  private final Broadcast.Receiver<Double> minEtaBroadcaster;
+  private List<Example> examples = null;
+  private final ExampleList dataSet;
+  private final LossFunction lossFunction;
+  private final StepSizes ts;
+
+  private Vector model = null;
+  private Vector descentDirection = null;
+
+  @Inject
+  public SlaveTask(
+      final GroupCommClient groupCommClient,
+      final ExampleList dataSet,
+      final LossFunction lossFunction,
+      @Parameter(ProbabilityOfFailure.class) final double pFailure,
+      final StepSizes ts) {
+
+    this.dataSet = dataSet;
+    this.lossFunction = lossFunction;
+    this.FAILURE_PROB = pFailure;
+    LOG.info("Using pFailure=" + this.FAILURE_PROB);
+    this.ts = ts;
+
+    this.communicationGroup = groupCommClient.getCommunicationGroup(AllCommunicationGroup.class);
+    this.controlMessageBroadcaster = communicationGroup.getBroadcastReceiver(ControlMessageBroadcaster.class);
+    this.modelBroadcaster = communicationGroup.getBroadcastReceiver(ModelBroadcaster.class);
+    this.lossAndGradientReducer = communicationGroup.getReduceSender(LossAndGradientReducer.class);
+    this.modelAndDescentDirectionBroadcaster = communicationGroup.getBroadcastReceiver(ModelAndDescentDirectionBroadcaster.class);
+    this.descentDirectionBroadcaster = communicationGroup.getBroadcastReceiver(DescentDirectionBroadcaster.class);
+    this.lineSearchEvaluationsReducer = communicationGroup.getReduceSender(LineSearchEvaluationsReducer.class);
+    this.minEtaBroadcaster = communicationGroup.getBroadcastReceiver(MinEtaBroadcaster.class);
+  }
+
+  @Override
+  public byte[] call(final byte[] memento) throws Exception {
+    /*
+     * In the case where there will be evaluator failure and data is not in
+     * memory we want to load the data while waiting to join the communication
+     * group
+     */
+    loadData();
+
+    for (boolean repeat = true; repeat; ) {
+
+      final ControlMessages controlMessage = controlMessageBroadcaster.receive();
+      switch (controlMessage) {
+
+        case Stop:
+          repeat = false;
+          break;
+
+        case ComputeGradientWithModel:
+          failPerhaps();
+          this.model = modelBroadcaster.receive();
+          lossAndGradientReducer.send(computeLossAndGradient());
+          break;
+
+        case ComputeGradientWithMinEta:
+          failPerhaps();
+          final double minEta = minEtaBroadcaster.receive();
+          assert (descentDirection != null);
+          this.descentDirection.scale(minEta);
+          assert (model != null);
+          this.model.add(descentDirection);
+          lossAndGradientReducer.send(computeLossAndGradient());
+          break;
+
+        case DoLineSearch:
+          failPerhaps();
+          this.descentDirection = descentDirectionBroadcaster.receive();
+          lineSearchEvaluationsReducer.send(lineSearchEvals());
+          break;
+
+        case DoLineSearchWithModel:
+          failPerhaps();
+          final Pair<Vector, Vector> modelAndDescentDir = modelAndDescentDirectionBroadcaster.receive();
+          this.model = modelAndDescentDir.first;
+          this.descentDirection = modelAndDescentDir.second;
+          lineSearchEvaluationsReducer.send(lineSearchEvals());
+          break;
+
+        default:
+          break;
+      }
+    }
+
+    return null;
+  }
+
+  private void failPerhaps() {
+    if (Math.random() < FAILURE_PROB) {
+      throw new RuntimeException("Simulated Failure");
+    }
+  }
+
+  private Pair<Vector, Integer> lineSearchEvals() {
+
+    if (examples == null) {
+      loadData();
+    }
+
+    final Vector zed = new DenseVector(examples.size());
+    final Vector ee = new DenseVector(examples.size());
+
+    for (int i = 0; i < examples.size(); i++) {
+      final Example example = examples.get(i);
+      double f = example.predict(model);
+      zed.set(i, f);
+      f = example.predict(descentDirection);
+      ee.set(i, f);
+    }
+
+    final double[] t = ts.getT();
+    final Vector evaluations = new DenseVector(t.length);
+    int i = 0;
+    for (final double d : t) {
+      double loss = 0;
+      for (int j = 0; j < examples.size(); j++) {
+        final Example example = examples.get(j);
+        final double val = zed.get(j) + d * ee.get(j);
+        loss += this.lossFunction.computeLoss(example.getLabel(), val);
+      }
+      evaluations.set(i++, loss);
+    }
+
+    return new Pair<>(evaluations, examples.size());
+  }
+
+  private Pair<Pair<Double, Integer>, Vector> computeLossAndGradient() {
+
+    if (examples == null) {
+      loadData();
+    }
+
+    final Vector gradient = new DenseVector(model.size());
+    double loss = 0.0;
+    for (final Example example : examples) {
+      final double f = example.predict(model);
+      final double g = this.lossFunction.computeGradient(example.getLabel(), f);
+      example.addGradient(gradient, g);
+      loss += this.lossFunction.computeLoss(example.getLabel(), f);
+    }
+
+    return new Pair<>(new Pair<>(loss, examples.size()), gradient);
+  }
+
+  private void loadData() {
+    LOG.info("Loading data");
+    examples = dataSet.getExamples();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/Example.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/Example.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/Example.java
new file mode 100644
index 0000000..2ec7146
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/Example.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.reef.examples.group.bgd.data;
+
+import org.apache.reef.examples.group.utils.math.Vector;
+
+import java.io.Serializable;
+
+/**
+ * Base interface for Examples for linear models.
+ */
+public interface Example extends Serializable {
+
+  /**
+   * Access to the label.
+   *
+   * @return the label
+   */
+  double getLabel();
+
+  /**
+   * Computes the prediction for this Example, given the model w.
+   * <p/>
+   * w.dot(this.getFeatures())
+   *
+   * @param w the model
+   * @return the prediction for this Example, given the model w.
+   */
+  double predict(Vector w);
+
+  /**
+   * Adds the current example's gradient to the gradientVector, assuming that
+   * the gradient with respect to the prediction is gradient.
+   */
+  void addGradient(Vector gradientVector, double gradient);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/SparseExample.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/SparseExample.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/SparseExample.java
new file mode 100644
index 0000000..094f1d8
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/SparseExample.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.reef.examples.group.bgd.data;
+
+import org.apache.reef.examples.group.utils.math.Vector;
+
+/**
+ * Example implementation on a index and value array.
+ */
+public final class SparseExample implements Example {
+
+  private static final long serialVersionUID = -2127500625316875426L;
+
+  private final float[] values;
+  private final int[] indices;
+  private final double label;
+
+  public SparseExample(final double label, final float[] values, final int[] indices) {
+    this.label = label;
+    this.values = values;
+    this.indices = indices;
+  }
+
+  public int getFeatureLength() {
+    return this.values.length;
+  }
+
+  @Override
+  public double getLabel() {
+    return this.label;
+  }
+
+  @Override
+  public double predict(final Vector w) {
+    double result = 0.0;
+    for (int i = 0; i < this.indices.length; ++i) {
+      result += w.get(this.indices[i]) * this.values[i];
+    }
+    return result;
+  }
+
+  @Override
+  public void addGradient(final Vector gradientVector, final double gradient) {
+    for (int i = 0; i < this.indices.length; ++i) {
+      final int index = this.indices[i];
+      final double contribution = gradient * this.values[i];
+      final double oldValue = gradientVector.get(index);
+      final double newValue = oldValue + contribution;
+      gradientVector.set(index, newValue);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/Parser.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/Parser.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/Parser.java
new file mode 100644
index 0000000..f4d8d09
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/Parser.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.reef.examples.group.bgd.data.parser;
+
+import org.apache.reef.examples.group.bgd.data.Example;
+
+/**
+ * Parses inputs into Examples.
+ *
+ * @param <T>
+ */
+public interface Parser<T> {
+
+  public Example parse(final T input);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/SVMLightParser.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/SVMLightParser.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/SVMLightParser.java
new file mode 100644
index 0000000..5f64606
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/data/parser/SVMLightParser.java
@@ -0,0 +1,98 @@
+/**
+ * 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.reef.examples.group.bgd.data.parser;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.reef.examples.group.bgd.data.Example;
+import org.apache.reef.examples.group.bgd.data.SparseExample;
+
+import javax.inject.Inject;
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * A Parser for SVMLight records
+ */
+public class SVMLightParser implements Parser<String> {
+
+  private static final Logger LOG = Logger.getLogger(SVMLightParser.class.getName());
+
+  @Inject
+  public SVMLightParser() {
+  }
+
+  @Override
+  public Example parse(final String line) {
+
+    final int entriesCount = StringUtils.countMatches(line, ":");
+    final int[] indices = new int[entriesCount];
+    final float[] values = new float[entriesCount];
+
+    final String[] entries = StringUtils.split(line, ' ');
+    String labelStr = entries[0];
+
+    final boolean pipeExists = labelStr.indexOf('|') != -1;
+    if (pipeExists) {
+      labelStr = labelStr.substring(0, labelStr.indexOf('|'));
+    }
+    double label = Double.parseDouble(labelStr);
+
+    if (label != 1) {
+      label = -1;
+    }
+
+    for (int j = 1; j < entries.length; ++j) {
+      final String x = entries[j];
+      final String[] entity = StringUtils.split(x, ':');
+      final int offset = pipeExists ? 0 : 1;
+      indices[j - 1] = Integer.parseInt(entity[0]) - offset;
+      values[j - 1] = Float.parseFloat(entity[1]);
+    }
+    return new SparseExample(label, values, indices);
+  }
+
+  public static void main(final String[] args) {
+    final Parser<String> parser = new SVMLightParser();
+    for (int i = 0; i < 10; i++) {
+      final List<SparseExample> examples = new ArrayList<>();
+      float avgFtLen = 0;
+      try (final BufferedReader br = new BufferedReader(new FileReader(
+          "C:\\Users\\shravan\\data\\splice\\hdi\\hdi_uncomp\\part-r-0000" + i))) {
+        String line = null;
+        while ((line = br.readLine()) != null) {
+          final SparseExample spEx = (SparseExample) parser.parse(line);
+          avgFtLen += spEx.getFeatureLength();
+          examples.add(spEx);
+        }
+      } catch (final IOException e) {
+        throw new RuntimeException("Exception", e);
+      }
+
+      LOG.log(Level.INFO, "OUT: {0} {1} {2}",
+          new Object[] { examples.size(), avgFtLen, avgFtLen / examples.size() });
+
+      examples.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LogisticLossFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LogisticLossFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LogisticLossFunction.java
new file mode 100644
index 0000000..78eb16f
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LogisticLossFunction.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.reef.examples.group.bgd.loss;
+
+import javax.inject.Inject;
+
+public final class LogisticLossFunction implements LossFunction {
+
+  /**
+   * Trivial constructor.
+   */
+  @Inject
+  public LogisticLossFunction() {
+  }
+
+  @Override
+  public double computeLoss(final double y, final double f) {
+    final double predictedTimesLabel = y * f;
+    return Math.log(1 + Math.exp(-predictedTimesLabel));
+  }
+
+  @Override
+  public double computeGradient(final double y, final double f) {
+    final double predictedTimesLabel = y * f;
+    return -y / (1 + Math.exp(predictedTimesLabel));
+  }
+
+  @Override
+  public String toString() {
+    return "LogisticLossFunction{}";
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LossFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LossFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LossFunction.java
new file mode 100644
index 0000000..e762add
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/LossFunction.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.group.bgd.loss;
+
+import org.apache.reef.tang.annotations.DefaultImplementation;
+
+/**
+ * Interface for Loss Functions.
+ */
+@DefaultImplementation(SquaredErrorLossFunction.class)
+public interface LossFunction {
+
+  /**
+   * Computes the loss incurred by predicting f, if y is the true label.
+   *
+   * @param y the label
+   * @param f the prediction
+   * @return the loss incurred by predicting f, if y is the true label.
+   */
+  double computeLoss(final double y, final double f);
+
+  /**
+   * Computes the gradient with respect to f, if y is the true label.
+   *
+   * @param y the label
+   * @param f the prediction
+   * @return the gradient with respect to f
+   */
+  double computeGradient(final double y, final double f);
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6c6ad336/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/SquaredErrorLossFunction.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/SquaredErrorLossFunction.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/SquaredErrorLossFunction.java
new file mode 100644
index 0000000..327f566
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/group/bgd/loss/SquaredErrorLossFunction.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.reef.examples.group.bgd.loss;
+
+import javax.inject.Inject;
+
+/**
+ * The Squared Error {@link LossFunction}.
+ */
+public class SquaredErrorLossFunction implements LossFunction {
+
+  /**
+   * Trivial constructor.
+   */
+  @Inject
+  public SquaredErrorLossFunction() {
+  }
+
+  @Override
+  public double computeLoss(double y, double f) {
+    return Math.pow(y - f, 2.0);
+  }
+
+  @Override
+  public double computeGradient(double y, double f) {
+    return (f - y) * 0.5;
+  }
+
+  @Override
+  public String toString() {
+    return "SquaredErrorLossFunction{}";
+  }
+}