Merge branch 'master' of ssh://gnunet.org/gnunet
[oweals/gnunet.git] / src / consensus / gnunet-service-consensus.c
index 4b69c5327e5799744b9b97d346ac9fc004d5a573..b934f468fbddad7cb6ea1a635e9f6175c5b147dd 100644 (file)
@@ -1,10 +1,10 @@
 /*
       This file is part of GNUnet
-      (C) 2012 Christian Grothoff (and other contributing authors)
+      Copyright (C) 2012, 2013, 2017 GNUnet e.V.
 
       GNUnet is free software; you can redistribute it and/or modify
       it under the terms of the GNU General Public License as published
-      by the Free Software Foundation; either version 2, or (at your
+      by the Free Software Foundation; either version 3, or (at your
       option) any later version.
 
       GNUnet is distributed in the hope that it will be useful, but
@@ -14,8 +14,8 @@
 
       You should have received a copy of the GNU General Public License
       along with GNUnet; see the file COPYING.  If not, write to the
-      Free Software Foundation, Inc., 59 Temple Place - Suite 330,
-      Boston, MA 02111-1307, USA.
+      Free Software Foundation, Inc., 51 Franklin Street, Fifth Floor,
+      Boston, MA 02110-1301, USA.
 */
 
 /**
  */
 
 #include "platform.h"
-#include "gnunet_common.h"
+#include "gnunet_util_lib.h"
+#include "gnunet_block_lib.h"
 #include "gnunet_protocols.h"
 #include "gnunet_applications.h"
-#include "gnunet_util_lib.h"
+#include "gnunet_set_service.h"
+#include "gnunet_statistics_service.h"
 #include "gnunet_consensus_service.h"
-#include "gnunet_core_service.h"
-#include "gnunet_stream_lib.h"
 #include "consensus_protocol.h"
-#include "ibf.h"
 #include "consensus.h"
 
 
-/**
- * Number of IBFs in a strata estimator.
- */
-#define STRATA_COUNT 32
-/**
- * Number of buckets per IBF.
- */
-#define STRATA_IBF_BUCKETS 80
-/**
- * hash num parameter for the difference digests and strata estimators
- */
-#define STRATA_HASH_NUM 3
+enum ReferendumVote
+{
+  /**
+   * Vote that nothing should change.
+   * This option is never voted explicitly.
+   */
+  VOTE_STAY = 0,
+  /**
+   * Vote that an element should be added.
+   */
+  VOTE_ADD = 1,
+  /**
+   * Vote that an element should be removed.
+   */
+  VOTE_REMOVE = 2,
+};
 
-/**
- * Number of buckets that can be transmitted in one message.
- */
-#define BUCKETS_PER_MESSAGE ((1<<15) / IBF_BUCKET_SIZE)
 
-/**
- * The maximum size of an ibf we use is 2^(MAX_IBF_ORDER).
- * Choose this value so that computing the IBF is still cheaper
- * than transmitting all values.
- */
-#define MAX_IBF_ORDER (16)
+enum EarlyStoppingPhase
+{
+  EARLY_STOPPING_NONE = 0,
+  EARLY_STOPPING_ONE_MORE = 1,
+  EARLY_STOPPING_DONE = 2,
+};
+
+
+GNUNET_NETWORK_STRUCT_BEGIN
 
 /**
- * Number exp-rounds.
+ * Tuple of integers that together
+ * identify a task uniquely.
  */
-#define NUM_EXP_ROUNDS (4)
+struct TaskKey {
+  /**
+   * A value from 'enum PhaseKind'.
+   */
+  uint16_t kind GNUNET_PACKED;
+
+  /**
+   * Number of the first peer
+   * in canonical order.
+   */
+  int16_t peer1 GNUNET_PACKED;
+
+  /**
+   * Number of the second peer in canonical order.
+   */
+  int16_t peer2 GNUNET_PACKED;
 
+  /**
+   * Repetition of the gradecast phase.
+   */
+  int16_t repetition GNUNET_PACKED;
 
-/* forward declarations */
+  /**
+   * Leader in the gradecast phase.
+   *
+   * Can be different from both peer1 and peer2.
+   */
+  int16_t leader GNUNET_PACKED;
+};
 
-struct ConsensusSession;
-struct IncomingSocket;
-struct ConsensusPeerInformation;
 
-static void
-client_send_next (struct ConsensusSession *session);
 
-static int
-get_peer_idx (const struct GNUNET_PeerIdentity *peer, const struct ConsensusSession *session);
+struct SetKey
+{
+  int set_kind GNUNET_PACKED;
+  int k1 GNUNET_PACKED;
+  int k2 GNUNET_PACKED;
+};
 
-static void 
-round_over (void *cls, const struct GNUNET_SCHEDULER_TaskContext *tc);
 
-static void
-send_ibf (struct ConsensusPeerInformation *cpi);
+struct SetEntry
+{
+  struct SetKey key;
+  struct GNUNET_SET_Handle *h;
+  /**
+   * GNUNET_YES if the set resulted
+   * from applying a referendum with contested
+   * elements.
+   */
+  int is_contested;
+};
 
-static void
-send_strata_estimator (struct ConsensusPeerInformation *cpi);
 
-static void
-decode (struct ConsensusPeerInformation *cpi);
+struct DiffKey
+{
+  int diff_kind GNUNET_PACKED;
+  int k1 GNUNET_PACKED;
+  int k2 GNUNET_PACKED;
+};
 
-static void 
-write_queued (void *cls, enum GNUNET_STREAM_Status status, size_t size);
+struct RfnKey
+{
+  int rfn_kind GNUNET_PACKED;
+  int k1 GNUNET_PACKED;
+  int k2 GNUNET_PACKED;
+};
 
-static void
-subround_over (void *cls, const struct GNUNET_SCHEDULER_TaskContext *tc);
 
+GNUNET_NETWORK_STRUCT_END
 
-/**
- * An element that is waiting to be transmitted to the client.
- */
-struct PendingElement
+enum PhaseKind
 {
+  PHASE_KIND_ALL_TO_ALL,
+  PHASE_KIND_ALL_TO_ALL_2,
+  PHASE_KIND_GRADECAST_LEADER,
+  PHASE_KIND_GRADECAST_ECHO,
+  PHASE_KIND_GRADECAST_ECHO_GRADE,
+  PHASE_KIND_GRADECAST_CONFIRM,
+  PHASE_KIND_GRADECAST_CONFIRM_GRADE,
   /**
-   * Pending elements are kept in a DLL.
+   * Apply a repetition of the all-to-all
+   * gradecast to the current set.
    */
-  struct PendingElement *next;
+  PHASE_KIND_APPLY_REP,
+  PHASE_KIND_FINISH,
+};
 
-  /**
-   * Pending elements are kept in a DLL.
-   */
-  struct PendingElement *prev;
 
+enum SetKind
+{
+  SET_KIND_NONE = 0,
+  SET_KIND_CURRENT,
   /**
-   * The actual element
+   * Last result set from a gradecast
    */
-  struct GNUNET_CONSENSUS_Element *element;
+  SET_KIND_LAST_GRADECAST,
+  SET_KIND_LEADER_PROPOSAL,
+  SET_KIND_ECHO_RESULT,
+};
 
-  /* peer this element is coming from */
-  struct ConsensusPeerInformation *cpi;
+enum DiffKind
+{
+  DIFF_KIND_NONE = 0,
+  DIFF_KIND_LEADER_PROPOSAL,
+  DIFF_KIND_LEADER_CONSENSUS,
+  DIFF_KIND_GRADECAST_RESULT,
+};
+
+enum RfnKind
+{
+  RFN_KIND_NONE = 0,
+  RFN_KIND_ECHO,
+  RFN_KIND_CONFIRM,
+  RFN_KIND_GRADECAST_RESULT
 };
 
 
-struct ElementList
+struct SetOpCls
 {
-  struct ElementList *next;
-  struct GNUNET_CONSENSUS_Element *element;
-  struct GNUNET_HashCode *element_hash;
+  struct SetKey input_set;
+
+  struct SetKey output_set;
+  struct RfnKey output_rfn;
+  struct DiffKey output_diff;
+
+  int do_not_remove;
+
+  int transceive_contested;
+
+  struct GNUNET_SET_OperationHandle *op;
 };
 
 
+struct FinishCls
+{
+  struct SetKey input_set;
+};
+
 /**
- * Describes the current round a consensus session is in.
+ * Closure for both @a start_task
+ * and @a cancel_task.
  */
-enum ConsensusRound
+union TaskFuncCls
 {
-  /**
-   * Not started the protocol yet.
-   */
-  CONSENSUS_ROUND_BEGIN=0,
-  /**
-   * Distribution of elements with the exponential scheme.
-   */
-  CONSENSUS_ROUND_EXCHANGE,
-  /**
-   * Exchange which elements each peer has, but not the elements.
-   */
-  CONSENSUS_ROUND_INVENTORY,
-  /**
-   * Collect and distribute missing values.
-   */
-  CONSENSUS_ROUND_STOCK,
-  /**
-   * Consensus concluded.
-   */
-  CONSENSUS_ROUND_FINISH
+  struct SetOpCls setop;
+  struct FinishCls finish;
 };
 
+struct TaskEntry;
 
-/**
- * Information about a peer that is in a consensus session.
+typedef void (*TaskFunc) (struct TaskEntry *task);
+
+/*
+ * Node in the consensus task graph.
  */
-struct ConsensusPeerInformation
+struct TaskEntry
 {
-  struct GNUNET_PeerIdentity peer_id;
+  struct TaskKey key;
 
-  /**
-   * Socket for communicating with the peer, either created by the local peer,
-   * or the remote peer.
-   */
-  struct GNUNET_STREAM_Socket *socket;
+  struct Step *step;
 
-  /**
-   * Message tokenizer, for the data received from this peer via the stream socket.
-   */
-  struct GNUNET_SERVER_MessageStreamTokenizer *mst;
+  int is_started;
 
-  /**
-   * Do we connect to the peer, or does the peer connect to us?
-   * Only valid for all-to-all phases
-   */
-  int is_outgoing;
+  int is_finished;
 
-  /**
-   * Did we receive/send a consensus hello?
-   */
-  int hello;
+  TaskFunc start;
+  TaskFunc cancel;
+
+  union TaskFuncCls cls;
+};
 
-  /**
-   * Handle for currently active read
-   */
-  struct GNUNET_STREAM_ReadHandle *rh;
 
+struct Step
+{
   /**
-   * Handle for currently active read
+   * All steps of one session are in a
+   * linked list for easier deallocation.
    */
-  struct GNUNET_STREAM_WriteHandle *wh;
-
-  enum {
-    /* beginning of round */
-    IBF_STATE_NONE=0,
-    /* we currently receive an ibf */
-    IBF_STATE_RECEIVING,
-    /* we currently transmit an ibf */
-    IBF_STATE_TRANSMITTING,
-    /* we decode a received ibf */
-    IBF_STATE_DECODING,
-    /* wait for elements and element requests */
-    IBF_STATE_ANTICIPATE_DIFF
-  } ibf_state ;
+  struct Step *prev;
 
   /**
-   * What is the order (=log2 size) of the ibf
-   * we're currently dealing with?
-   * Interpretation depends on ibf_state.
+   * All steps of one session are in a
+   * linked list for easier deallocation.
    */
-  int ibf_order;
+  struct Step *next;
+
+  struct ConsensusSession *session;
 
   /**
-   * The current IBF for this peer,
-   * purpose dependent on ibf_state
+   * Tasks that this step is composed of.
    */
-  struct InvertibleBloomFilter *ibf;
+  struct TaskEntry **tasks;
+  unsigned int tasks_len;
+  unsigned int tasks_cap;
 
-  /**
-   * How many buckets have we transmitted/received? 
-   * Interpretatin depends on ibf_state
+  unsigned int finished_tasks;
+
+  /*
+   * Tasks that have this task as dependency.
+   *
+   * We store pointers to subordinates rather
+   * than to prerequisites since it makes
+   * tracking the readiness of a task easier.
    */
-  int ibf_bucket_counter;
+  struct Step **subordinates;
+  unsigned int subordinates_len;
+  unsigned int subordinates_cap;
 
   /**
-   * Strata estimator of the peer, NULL if our peer
-   * initiated the reconciliation.
+   * Counter for the prerequisites of
+   * this step.
    */
-  struct StrataEstimator *se;
+  size_t pending_prereq;
 
-  /**
-   * Element keys that this peer misses, but we have them.
+  /*
+   * Task that will run this step despite
+   * any pending prerequisites.
    */
-  struct GNUNET_CONTAINER_MultiHashMap *requested_keys;
+  struct GNUNET_SCHEDULER_Task *timeout_task;
 
-  /**
-   * Element keys that this peer has, but we miss.
+  unsigned int is_running;
+
+  unsigned int is_finished;
+
+  /*
+   * Synchrony round of the task.
+   * Determines the deadline for the task.
    */
-  struct GNUNET_CONTAINER_MultiHashMap *reported_keys;
+  unsigned int round;
 
   /**
-   * Back-reference to the consensus session,
-   * to that ConsensusPeerInformation can be used as a closure
+   * Human-readable name for
+   * the task, used for debugging.
    */
-  struct ConsensusSession *session;
+  char *debug_name;
 
   /**
-   * Messages queued for the current round.
+   * When we're doing an early finish, how should this step be
+   * treated?
+   * If GNUNET_YES, the step will be marked as finished
+   * without actually running its tasks.
+   * Otherwise, the step will still be run even after
+   * an early finish.
+   *
+   * Note that a task may never be finished early if
+   * it is already running.
    */
-  struct QueuedMessage *messages_head;
+  int early_finishable;
+};
 
-  /**
-   * Messages queued for the current round.
+
+struct RfnElementInfo
+{
+  const struct GNUNET_SET_Element *element;
+
+  /*
+   * GNUNET_YES if the peer votes for the proposal.
    */
-  struct QueuedMessage *messages_tail;
+  int *votes;
 
   /**
-   * True if we are actually replaying the strata message,
-   * e.g. currently handling the premature_strata_message.
+   * Proposal for this element,
+   * can only be VOTE_ADD or VOTE_REMOVE.
    */
-  int replaying_strata_message;
+  enum ReferendumVote proposal;
+};
 
-  /**
-   * A strata message that is not actually for the current round,
-   * used in the exp-scheme.
+
+struct ReferendumEntry
+{
+  struct RfnKey key;
+
+  /*
+   * Elements where there is at least one proposed change.
+   *
+   * Maps the hash of the GNUNET_SET_Element
+   * to 'struct RfnElementInfo'.
    */
-  struct StrataMessage *premature_strata_message;
+  struct GNUNET_CONTAINER_MultiHashMap *rfn_elements;
+
+  unsigned int num_peers;
 
   /**
-   * We have finishes the exp-subround with the peer.
+   * Stores, for every peer in the session,
+   * whether the peer finished the whole referendum.
+   *
+   * Votes from peers are only counted if they're
+   * marked as commited (#GNUNET_YES) in the referendum.
+   *
+   * Otherwise (#GNUNET_NO), the requested changes are
+   * not counted for majority votes or thresholds.
    */
-  int exp_subround_finished;
+  int *peer_commited;
 
-  int inventory_synced;
 
   /**
-   * Round this peer seems to be in, according to the last SE we got.
-   * Necessary to store this, as we sometimes need to respond to a request from an
-   * older round, while we are already in the next round.
+   * Contestation state of the peer.  If a peer is contested, the values it
+   * contributed are still counted for applying changes, but the grading is
+   * affected.
    */
-  enum ConsensusRound apparent_round;
-
+  int *peer_contested;
 };
 
-typedef void (*QueuedMessageCallback) (void *msg);
 
-/**
- * A doubly linked list of messages.
- */
-struct QueuedMessage
+struct DiffElementInfo
 {
-  struct GNUNET_MessageHeader *msg;
-
-  /**
-   * Queued messages are stored in a doubly linked list.
-   */
-  struct QueuedMessage *next;
+  const struct GNUNET_SET_Element *element;
 
   /**
-   * Queued messages are stored in a doubly linked list.
+   * Positive weight for 'add', negative
+   * weights for 'remove'.
    */
-  struct QueuedMessage *prev;
-
-  QueuedMessageCallback cb;
-  
-  void *cls;
+  int weight;
 };
 
 
-struct StrataEstimator
+/**
+ * Weighted diff.
+ */
+struct DiffEntry
 {
-  struct InvertibleBloomFilter **strata;
+  struct DiffKey key;
+  struct GNUNET_CONTAINER_MultiHashMap *changes;
 };
 
 
+
 /**
  * A consensus session consists of one local client and the remote authorities.
  */
@@ -337,182 +410,104 @@ struct ConsensusSession
    */
   struct ConsensusSession *prev;
 
-  /**
-   * Join message. Used to initialize the session later,
-   * if the identity of the local peer is not yet known.
-   * NULL if the session has been fully initialized.
-   */
-  struct GNUNET_CONSENSUS_JoinMessage *join_msg;
+  unsigned int num_client_insert_pending;
 
-  /**
-  * Global consensus identification, computed
-  * from the session id and participating authorities.
-  */
-  struct GNUNET_HashCode global_id;
+  struct GNUNET_CONTAINER_MultiHashMap *setmap;
+  struct GNUNET_CONTAINER_MultiHashMap *rfnmap;
+  struct GNUNET_CONTAINER_MultiHashMap *diffmap;
 
   /**
-   * Local client in this consensus session.
-   * There is only one client per consensus session.
+   * Array of peers with length 'num_peers'.
    */
-  struct GNUNET_SERVER_Client *client;
+  int *peers_blacklisted;
 
-  /**
-   * Elements in the consensus set of this session,
-   * all of them either have been sent by or approved by the client.
-   * Contains ElementList.
-   * Used as a unique-key hashmap.
+  /*
+   * Mapping from (hashed) TaskKey to TaskEntry.
+   *
+   * We map the application_id for a round to the task that should be
+   * executed, so we don't have to go through all task whenever we get
+   * an incoming set op request.
    */
-  struct GNUNET_CONTAINER_MultiHashMap *values;
+  struct GNUNET_CONTAINER_MultiHashMap *taskmap;
 
-  /**
-   * Elements that have not been approved (or rejected) by the client yet.
-   */
-  struct PendingElement *client_approval_head;
+  struct Step *steps_head;
+  struct Step *steps_tail;
+
+  int conclude_started;
+
+  int conclude_done;
 
   /**
-   * Elements that have not been approved (or rejected) by the client yet.
-   */
-  struct PendingElement *client_approval_tail;
+  * Global consensus identification, computed
+  * from the session id and participating authorities.
+  */
+  struct GNUNET_HashCode global_id;
 
   /**
-   * Messages to be sent to the local client that owns this session
+   * Client that inhabits the session
    */
-  struct QueuedMessage *client_messages_head;
+  struct GNUNET_SERVICE_Client *client;
 
   /**
-   * Messages to be sent to the local client that owns this session
+   * Queued messages to the client.
    */
-  struct QueuedMessage *client_messages_tail;
+  struct GNUNET_MQ_Handle *client_mq;
 
   /**
-   * Currently active transmit handle for sending to the client
+   * Time when the conclusion of the consensus should begin.
    */
-  struct GNUNET_SERVER_TransmitHandle *client_th;
+  struct GNUNET_TIME_Absolute conclude_start;
 
   /**
    * Timeout for all rounds together, single rounds will schedule a timeout task
    * with a fraction of the conclude timeout.
+   * Only valid once the current round is not CONSENSUS_ROUND_BEGIN.
    */
-  struct GNUNET_TIME_Relative conclude_timeout;
-  
-  /**
-   * Timeout task identifier for the current round
-   */
-  GNUNET_SCHEDULER_TaskIdentifier round_timeout_tid;
+  struct GNUNET_TIME_Absolute conclude_deadline;
 
-  /**
-   * Number of other peers in the consensus
-   */
-  unsigned int num_peers;
+  struct GNUNET_PeerIdentity *peers;
 
   /**
-   * Information about the other peers,
-   * their state, etc.
+   * Number of other peers in the consensus.
    */
-  struct ConsensusPeerInformation *info;
+  unsigned int num_peers;
 
   /**
    * Index of the local peer in the peers array
    */
-  int local_peer_idx;
-
-  /**
-   * Strata estimator, computed online
-   */
-  struct StrataEstimator *se;
-
-  /**
-   * Pre-computed IBFs
-   */
-  struct InvertibleBloomFilter **ibfs;
-
-  /**
-   * Current round
-   */
-  enum ConsensusRound current_round;
-
-  int exp_round;
-
-  int exp_subround;
-
-  /**
-   * Permutation of peers for the current round,
-   * maps logical index (for current round) to physical index (location in info array)
-   */
-  int *shuffle;
-
-  /**
-   * The partner for the current exp-round
-   */
-  struct ConsensusPeerInformation* partner_outgoing;
-
-  /**
-   * The partner for the current exp-round
-   */
-  struct ConsensusPeerInformation* partner_incoming;
-};
-
-
-/**
- * Sockets from other peers who want to communicate with us.
- * It may not be known yet which consensus session they belong to.
- * Also, the session might not exist yet locally.
- */
-struct IncomingSocket
-{
-  /**
-   * Incoming sockets are kept in a double linked list.
-   */
-  struct IncomingSocket *next;
-
-  /**
-   * Incoming sockets are kept in a double linked list.
-   */
-  struct IncomingSocket *prev;
-
-  /**
-   * The actual socket.
-   */
-  struct GNUNET_STREAM_Socket *socket;
+  unsigned int local_peer_idx;
 
   /**
-   * Handle for currently active read
+   * Listener for requests from other peers.
+   * Uses the session's global id as app id.
    */
-  struct GNUNET_STREAM_ReadHandle *rh;
+  struct GNUNET_SET_ListenHandle *set_listener;
 
   /**
-   * Peer that connected to us with the socket.
+   * State of our early stopping scheme.
    */
-  struct GNUNET_PeerIdentity peer_id;
+  int early_stopping;
 
   /**
-   * Message stream tokenizer for this socket.
+   * Our set size from the first round.
    */
-  struct GNUNET_SERVER_MessageStreamTokenizer *mst;
+  uint64_t first_size;
 
-  /**
-   * Peer-in-session this socket belongs to, once known, otherwise NULL.
-   */
-  struct ConsensusPeerInformation *cpi;
+  uint64_t *first_sizes_received;
 
   /**
-   * Set to the global session id, if the peer sent us a hello-message,
-   * but the session does not exist yet.
+   * Bounded Eppstein lower bound.
    */
-  struct GNUNET_HashCode *requested_gid;
+  uint64_t lower_bound;
 };
 
-
-static struct IncomingSocket *incoming_sockets_head;
-static struct IncomingSocket *incoming_sockets_tail;
-
 /**
- * Linked list of sesstions this peer participates in.
+ * Linked list of sessions this peer participates in.
  */
 static struct ConsensusSession *sessions_head;
 
 /**
- * Linked list of sesstions this peer participates in.
+ * Linked list of sessions this peer participates in.
  */
 static struct ConsensusSession *sessions_tail;
 
@@ -521,1283 +516,1845 @@ static struct ConsensusSession *sessions_tail;
  */
 static const struct GNUNET_CONFIGURATION_Handle *cfg;
 
-/**
- * Handle to the server for this service.
- */
-static struct GNUNET_SERVER_Handle *srv;
-
 /**
  * Peer that runs this service.
  */
-static struct GNUNET_PeerIdentity *my_peer;
-
-/**
- * Handle to the core service. Only used during service startup, will be NULL after that.
- */
-static struct GNUNET_CORE_Handle *core;
+static struct GNUNET_PeerIdentity my_peer;
 
 /**
- * Listener for sockets from peers that want to reconcile with us.
+ * Statistics handle.
  */
-static struct GNUNET_STREAM_ListenSocket *listener;
+struct GNUNET_STATISTICS_Handle *statistics;
 
 
-/**
- * Queue a message to be sent to the inhabiting client of a session.
- *
- * @param session session
- * @param msg message we want to queue
- */
-static void
-queue_client_message (struct ConsensusSession *session, struct GNUNET_MessageHeader *msg)
-{
-  struct QueuedMessage *qm;
-  qm = GNUNET_malloc (sizeof *qm);
-  qm->msg = msg;
-  GNUNET_CONTAINER_DLL_insert_tail (session->client_messages_head, session->client_messages_tail, qm);
-}
-
-/**
- * Queue a message to be sent to another peer
- *
- * @param cpi peer
- * @param msg message we want to queue
- * @param cb callback, called when the message is given to strem
- * @param cls closure for cb
- */
 static void
-queue_peer_message_with_cls (struct ConsensusPeerInformation *cpi, struct GNUNET_MessageHeader *msg, QueuedMessageCallback cb, void *cls)
-{
-  struct QueuedMessage *qm;
-  qm = GNUNET_malloc (sizeof *qm);
-  qm->msg = msg;
-  qm->cls = cls;
-  qm->cb = cb;
-  GNUNET_CONTAINER_DLL_insert_tail (cpi->messages_head, cpi->messages_tail, qm);
-  if (cpi->wh == NULL)
-    write_queued (cpi, GNUNET_STREAM_OK, 0);
-}
+finish_task (struct TaskEntry *task);
 
 
-/**
- * Queue a message to be sent to another peer
- *
- * @param cpi peer
- * @param msg message we want to queue
- */
 static void
-queue_peer_message (struct ConsensusPeerInformation *cpi, struct GNUNET_MessageHeader *msg)
-{
-  queue_peer_message_with_cls (cpi, msg, NULL, NULL);
-}
+run_ready_steps (struct ConsensusSession *session);
 
 
-/*
-static void
-clear_peer_messages (struct ConsensusPeerInformation *cpi)
+static const char *
+phasename (uint16_t phase)
 {
-  cpi->messages_head = NULL;
-  cpi->messages_tail = NULL;
+  switch (phase)
+  {
+    case PHASE_KIND_ALL_TO_ALL: return "ALL_TO_ALL";
+    case PHASE_KIND_ALL_TO_ALL_2: return "ALL_TO_ALL_2";
+    case PHASE_KIND_FINISH: return "FINISH";
+    case PHASE_KIND_GRADECAST_LEADER: return "GRADECAST_LEADER";
+    case PHASE_KIND_GRADECAST_ECHO: return "GRADECAST_ECHO";
+    case PHASE_KIND_GRADECAST_ECHO_GRADE: return "GRADECAST_ECHO_GRADE";
+    case PHASE_KIND_GRADECAST_CONFIRM: return "GRADECAST_CONFIRM";
+    case PHASE_KIND_GRADECAST_CONFIRM_GRADE: return "GRADECAST_CONFIRM_GRADE";
+    case PHASE_KIND_APPLY_REP: return "APPLY_REP";
+    default: return "(unknown)";
+  }
 }
-*/
 
 
-/**
- * Estimate set difference with two strata estimators,
- * i.e. arrays of IBFs.
- * Does not not modify its arguments.
- *
- * @param se1 first strata estimator
- * @param se2 second strata estimator
- * @return the estimated difference
- */
-static int
-estimate_difference (const struct StrataEstimator *se1,
-                     const struct StrataEstimator *se2)
+static const char *
+setname (uint16_t kind)
 {
-  int i;
-  int count;
-  count = 0;
-  for (i = STRATA_COUNT - 1; i >= 0; i--)
-  {
-    struct InvertibleBloomFilter *diff;
-    /* number of keys decoded from the ibf */
-    int ibf_count;
-    int more;
-    ibf_count = 0;
-    /* FIXME: implement this without always allocating new IBFs */
-    diff = ibf_dup (se1->strata[i]);
-    ibf_subtract (diff, se2->strata[i]);
-    for (;;)
-    {
-      more = ibf_decode (diff, NULL, NULL);
-      if (GNUNET_NO == more)
-      {
-        count += ibf_count;
-        break;
-      }
-      if (GNUNET_SYSERR == more)
-      {
-        ibf_destroy (diff);
-        return count * (1 << (i + 1));
-      }
-      ibf_count++;
-    }
-    ibf_destroy (diff);
+  switch (kind)
+  {
+    case SET_KIND_CURRENT: return "CURRENT";
+    case SET_KIND_LEADER_PROPOSAL: return "LEADER_PROPOSAL";
+    case SET_KIND_NONE: return "NONE";
+    default: return "(unknown)";
   }
-  return count;
 }
 
-
-/**
- * Called when receiving data from a peer that is member of
- * an inhabited consensus session.
- *
- * @param cls the closure from GNUNET_STREAM_read
- * @param status the status of the stream at the time this function is called
- * @param data traffic from the other side
- * @param size the number of bytes available in data read; will be 0 on timeout 
- * @return number of bytes of processed from 'data' (any data remaining should be
- *         given to the next time the read processor is called).
- */
-static size_t
-session_stream_data_processor (void *cls,
-                       enum GNUNET_STREAM_Status status,
-                       const void *data,
-                       size_t size)
-{
-  struct ConsensusPeerInformation *cpi;
-  int ret;
-
-  GNUNET_assert (GNUNET_STREAM_OK == status);
-  cpi = cls;
-  GNUNET_assert (NULL != cpi->mst);
-  ret = GNUNET_SERVER_mst_receive (cpi->mst, cpi, data, size, GNUNET_NO, GNUNET_YES);
-  if (GNUNET_SYSERR == ret)
-  {
-    /* FIXME: handle this correctly */
-    GNUNET_assert (0);
+static const char *
+rfnname (uint16_t kind)
+{
+  switch (kind)
+  {
+    case RFN_KIND_NONE: return "NONE";
+    case RFN_KIND_ECHO: return "ECHO";
+    case RFN_KIND_CONFIRM: return "CONFIRM";
+    default: return "(unknown)";
   }
-  /* read again */
-  cpi->rh = GNUNET_STREAM_read (cpi->socket, GNUNET_TIME_UNIT_FOREVER_REL,
-                                &session_stream_data_processor, cpi);
-  /* we always read all data */
-  return size;
 }
 
-/**
- * Called when we receive data from a peer that is not member of
- * a session yet, or the session is not yet inhabited.
- *
- * @param cls the closure from GNUNET_STREAM_read
- * @param status the status of the stream at the time this function is called
- * @param data traffic from the other side
- * @param size the number of bytes available in data read; will be 0 on timeout 
- * @return number of bytes of processed from 'data' (any data remaining should be
- *         given to the next time the read processor is called).
- */
-static size_t
-incoming_stream_data_processor (void *cls,
-                       enum GNUNET_STREAM_Status status,
-                       const void *data,
-                       size_t size)
+static const char *
+diffname (uint16_t kind)
 {
-  struct IncomingSocket *incoming;
-  int ret;
-
-  GNUNET_assert (GNUNET_STREAM_OK == status);
-  incoming = cls;
-  ret = GNUNET_SERVER_mst_receive (incoming->mst, incoming, data, size, GNUNET_NO, GNUNET_YES);
-  if (GNUNET_SYSERR == ret)
+  switch (kind)
   {
-    /* FIXME: handle this correctly */
-    GNUNET_assert (0);
+    case DIFF_KIND_NONE: return "NONE";
+    case DIFF_KIND_LEADER_CONSENSUS: return "LEADER_CONSENSUS";
+    case DIFF_KIND_GRADECAST_RESULT: return "GRADECAST_RESULT";
+    case DIFF_KIND_LEADER_PROPOSAL: return "LEADER_PROPOSAL";
+    default: return "(unknown)";
   }
-  /* read again */
-  incoming->rh = GNUNET_STREAM_read (incoming->socket, GNUNET_TIME_UNIT_FOREVER_REL,
-                                     &incoming_stream_data_processor, incoming);
-  /* we always read all data */
-  return size;
 }
 
+#ifdef GNUNET_EXTRA_LOGGING
+
 
-static void
-send_elements (struct ConsensusPeerInformation *cpi, struct ElementList *head)
+static const char *
+debug_str_element (const struct GNUNET_SET_Element *el)
 {
-  struct GNUNET_CONSENSUS_Element *element;
-  struct GNUNET_MessageHeader *element_msg;
-  size_t msize;
+  struct GNUNET_HashCode hash;
 
-  while (NULL != head)
-  {
-    element = head->element;
-    msize = sizeof (struct GNUNET_MessageHeader) + element->size;
-    element_msg = GNUNET_malloc (msize);
-    element_msg->size = htons (msize);
-    switch (cpi->apparent_round)
-    {
-      case CONSENSUS_ROUND_STOCK:
-      case CONSENSUS_ROUND_EXCHANGE:
-        element_msg->type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS);
-        break;
-      case CONSENSUS_ROUND_INVENTORY:
-        element_msg->type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS_REPORT);
-        break;
-      default:
-        GNUNET_break (0);
-    }
-    GNUNET_assert (NULL != element->data);
-    memcpy (&element_msg[1], element->data, element->size);
-    queue_peer_message (cpi, element_msg);
-    head = head->next;
-  }
+  GNUNET_SET_element_hash (el, &hash);
+
+  return GNUNET_h2s (&hash);
 }
 
-/**
- * Iterator to insert values into an ibf.
- *
- * @param cls closure
- * @param key current key code
- * @param value value in the hash map
- * @return GNUNET_YES if we should continue to
- *         iterate,
- *         GNUNET_NO if not.
- */
-static int
-ibf_values_iterator (void *cls,
-                     const struct GNUNET_HashCode *key,
-                     void *value)
-{
-  struct ConsensusPeerInformation *cpi;
-  struct ElementList *head;
-  struct IBF_Key ibf_key;
-  cpi = cls;
-  head = value;
-  ibf_key = ibf_key_from_hashcode (head->element_hash);
-  GNUNET_assert (ibf_key.key_val == ibf_key_from_hashcode (key).key_val);
-  ibf_insert (cpi->session->ibfs[cpi->ibf_order], ibf_key);
-  return GNUNET_YES;
+static const char *
+debug_str_task_key (struct TaskKey *tk)
+{
+  static char buf[256];
+
+  snprintf (buf, sizeof (buf),
+            "TaskKey kind=%s, p1=%d, p2=%d, l=%d, rep=%d",
+            phasename (tk->kind), tk->peer1, tk->peer2,
+            tk->leader, tk->repetition);
+
+  return buf;
 }
 
-/**
- * Create and populate an IBF for the specified peer,
- * if it does not already exist.
- *
- * @param cpi peer to create the ibf for
- */
-static void
-prepare_ibf (struct ConsensusPeerInformation *cpi)
+static const char *
+debug_str_diff_key (struct DiffKey *dk)
 {
-  if (NULL == cpi->session->ibfs[cpi->ibf_order])
-  {
-    cpi->session->ibfs[cpi->ibf_order] = ibf_create (1 << cpi->ibf_order, STRATA_HASH_NUM, 0);
-    GNUNET_CONTAINER_multihashmap_iterate (cpi->session->values, ibf_values_iterator, cpi);
-  }
+  static char buf[256];
+
+  snprintf (buf, sizeof (buf),
+            "DiffKey kind=%s, k1=%d, k2=%d",
+            diffname (dk->diff_kind), dk->k1, dk->k2);
+
+  return buf;
 }
 
+static const char *
+debug_str_set_key (const struct SetKey *sk)
+{
+  static char buf[256];
+
+  snprintf (buf, sizeof (buf),
+            "SetKey kind=%s, k1=%d, k2=%d",
+            setname (sk->set_kind), sk->k1, sk->k2);
+
+  return buf;
+}
+
+
+static const char *
+debug_str_rfn_key (const struct RfnKey *rk)
+{
+  static char buf[256];
+
+  snprintf (buf, sizeof (buf),
+            "RfnKey kind=%s, k1=%d, k2=%d",
+            rfnname (rk->rfn_kind), rk->k1, rk->k2);
+
+  return buf;
+}
+
+#endif /* GNUNET_EXTRA_LOGGING */
+
 
 /**
- * Called when a remote peer wants to inform the local peer
- * that the remote peer misses elements.
- * Elements are not reconciled.
+ * Send the final result set of the consensus to the client, element by
+ * element.
  *
- * @param cpi session
- * @param msg message
+ * @param cls closure
+ * @param element the current element, NULL if all elements have been
+ *        iterated over
+ * @return #GNUNET_YES to continue iterating, #GNUNET_NO to stop.
  */
 static int
-handle_p2p_element_report (struct ConsensusPeerInformation *cpi, const struct GNUNET_MessageHeader *msg)
+send_to_client_iter (void *cls,
+                     const struct GNUNET_SET_Element *element)
 {
-  GNUNET_assert (0);
+  struct TaskEntry *task = (struct TaskEntry *) cls;
+  struct ConsensusSession *session = task->step->session;
+  struct GNUNET_MQ_Envelope *ev;
+
+  if (NULL != element)
+  {
+    struct GNUNET_CONSENSUS_ElementMessage *m;
+    const struct ConsensusElement *ce;
+
+    GNUNET_assert (GNUNET_BLOCK_TYPE_CONSENSUS_ELEMENT == element->element_type);
+    ce = element->data;
+
+    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "marker is %u\n", (unsigned) ce->marker);
+
+    if (0 != ce->marker)
+      return GNUNET_YES;
+
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "P%d: sending element %s to client\n",
+                session->local_peer_idx,
+                debug_str_element (element));
+
+    ev = GNUNET_MQ_msg_extra (m, element->size - sizeof (struct ConsensusElement),
+                              GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_RECEIVED_ELEMENT);
+    m->element_type = ce->payload_type;
+    GNUNET_memcpy (&m[1], &ce[1], element->size - sizeof (struct ConsensusElement));
+    GNUNET_MQ_send (session->client_mq, ev);
+  }
+  else
+  {
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "P%d: finished iterating elements for client\n",
+                session->local_peer_idx);
+    ev = GNUNET_MQ_msg_header (GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_CONCLUDE_DONE);
+    GNUNET_MQ_send (session->client_mq, ev);
+  }
+  return GNUNET_YES;
 }
 
 
-static int
-exp_subround_finished (const struct ConsensusSession *session)
+static struct SetEntry *
+lookup_set (struct ConsensusSession *session, struct SetKey *key)
 {
-  int not_finished;
-  not_finished = 0;
-  if ((session->partner_outgoing != NULL) && (session->partner_outgoing->exp_subround_finished == GNUNET_NO))
-      not_finished++;
-  if ((session->partner_incoming != NULL) && (session->partner_incoming->exp_subround_finished == GNUNET_NO))
-      not_finished++;
-  if (0 == not_finished)
-    return GNUNET_YES;
-  return GNUNET_NO;
+  struct GNUNET_HashCode hash;
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "P%u: looking up set {%s}\n",
+              session->local_peer_idx,
+              debug_str_set_key (key));
+
+  GNUNET_assert (SET_KIND_NONE != key->set_kind);
+  GNUNET_CRYPTO_hash (key, sizeof (struct SetKey), &hash);
+  return GNUNET_CONTAINER_multihashmap_get (session->setmap, &hash);
 }
 
-static int
-inventory_round_finished (struct ConsensusSession *session)
+
+static struct DiffEntry *
+lookup_diff (struct ConsensusSession *session, struct DiffKey *key)
 {
-  int i;
-  int finished;
-  finished = 0;
-  for (i = 0; i < session->num_peers; i++)
-    if (GNUNET_YES == session->info[i].inventory_synced)
-      finished++;
-  if (finished >= (session->num_peers / 2))
-    return GNUNET_YES;
-  return GNUNET_NO;
+  struct GNUNET_HashCode hash;
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "P%u: looking up diff {%s}\n",
+              session->local_peer_idx,
+              debug_str_diff_key (key));
+
+  GNUNET_assert (DIFF_KIND_NONE != key->diff_kind);
+  GNUNET_CRYPTO_hash (key, sizeof (struct DiffKey), &hash);
+  return GNUNET_CONTAINER_multihashmap_get (session->diffmap, &hash);
 }
 
 
+static struct ReferendumEntry *
+lookup_rfn (struct ConsensusSession *session, struct RfnKey *key)
+{
+  struct GNUNET_HashCode hash;
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "P%u: looking up rfn {%s}\n",
+              session->local_peer_idx,
+              debug_str_rfn_key (key));
+
+  GNUNET_assert (RFN_KIND_NONE != key->rfn_kind);
+  GNUNET_CRYPTO_hash (key, sizeof (struct RfnKey), &hash);
+  return GNUNET_CONTAINER_multihashmap_get (session->rfnmap, &hash);
+}
+
 
 static void
-fin_sent_cb (void *cls)
+diff_insert (struct DiffEntry *diff,
+             int weight,
+             const struct GNUNET_SET_Element *element)
 {
-  struct ConsensusPeerInformation *cpi;
-  cpi = cls;
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: sent FIN\n", cpi->session->local_peer_idx);
-  switch (cpi->session->current_round)
+  struct DiffElementInfo *di;
+  struct GNUNET_HashCode hash;
+
+  GNUNET_assert ( (1 == weight) || (-1 == weight));
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "diff_insert with element size %u\n",
+              element->size);
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "hashing element\n");
+
+  GNUNET_SET_element_hash (element, &hash);
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "hashed element\n");
+
+  di = GNUNET_CONTAINER_multihashmap_get (diff->changes, &hash);
+
+  if (NULL == di)
   {
-    case CONSENSUS_ROUND_EXCHANGE:
-    case CONSENSUS_ROUND_STOCK:
-      if (cpi->session->current_round != cpi->apparent_round)
-      {
-        GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: FIN to SYNC from the past\n", cpi->session->local_peer_idx);
-        break;
-      }
-      cpi->exp_subround_finished = GNUNET_YES;
-      /* the subround is only really over if *both* partners are done */
-      if (GNUNET_YES == exp_subround_finished (cpi->session))
-        subround_over (cpi->session, NULL);
-      else
-        GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: still waiting for more after FIN sent\n", cpi->session->local_peer_idx);
-      break;
-    case CONSENSUS_ROUND_INVENTORY:
-      cpi->inventory_synced = GNUNET_YES;
-      if (inventory_round_finished (cpi->session) && cpi->session->current_round == cpi->apparent_round)
-        round_over (cpi->session, NULL);
-      /* FIXME: maybe go to next round */
-      break;
-    default:
-      GNUNET_break (0);
+    di = GNUNET_new (struct DiffElementInfo);
+    di->element = GNUNET_SET_element_dup (element);
+    GNUNET_assert (GNUNET_OK ==
+                   GNUNET_CONTAINER_multihashmap_put (diff->changes,
+                                                      &hash, di,
+                                                      GNUNET_CONTAINER_MULTIHASHMAPOPTION_UNIQUE_FAST));
   }
+
+  di->weight = weight;
 }
 
 
-/**
- * Gets called when the other peer wants us to inform that
- * it has decoded our ibf and sent us all elements / requests
- */
-static int
-handle_p2p_synced (struct ConsensusPeerInformation *cpi, const struct GNUNET_MessageHeader *msg)
-{
-  struct ConsensusRoundMessage *fin_msg;
-
-  switch (cpi->session->current_round)
-  {
-    case CONSENSUS_ROUND_INVENTORY:
-      cpi->inventory_synced = GNUNET_YES;
-    case CONSENSUS_ROUND_STOCK:
-    case CONSENSUS_ROUND_EXCHANGE:
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got SYNC from P%d\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      fin_msg = GNUNET_malloc (sizeof *fin_msg);
-      fin_msg->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_FIN);
-      fin_msg->header.size = htons (sizeof *fin_msg);
-      fin_msg->round = cpi->apparent_round;
-      /* the subround os over once we kicked off sending the fin msg */
-      /* FIXME: assert we are talking to the right peer! */
-      queue_peer_message_with_cls (cpi, (struct GNUNET_MessageHeader *) fin_msg, fin_sent_cb, cpi);
-      /* FIXME: mark peer as synced */
-      break;
-    default:
-      GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "unexpected SYNCED message the current round\n");
-      break;
-  }
-  return GNUNET_YES;
+static void
+rfn_commit (struct ReferendumEntry *rfn,
+            uint16_t commit_peer)
+{
+  GNUNET_assert (commit_peer < rfn->num_peers);
+
+  rfn->peer_commited[commit_peer] = GNUNET_YES;
 }
 
 
-/**
- * The other peer wants us to inform that he sent us all the elements we requested.
- */
-static int
-handle_p2p_fin (struct ConsensusPeerInformation *cpi, const struct GNUNET_MessageHeader *msg)
+static void
+rfn_contest (struct ReferendumEntry *rfn,
+             uint16_t contested_peer)
 {
-  struct ConsensusRoundMessage *round_msg;
-  round_msg = (struct ConsensusRoundMessage *) msg;
-  /* FIXME: only call subround_over if round is the current one! */
-  switch (cpi->session->current_round)
-  {
-    case CONSENSUS_ROUND_EXCHANGE:
-    case CONSENSUS_ROUND_STOCK:
-      if (cpi->session->current_round != round_msg->round)
-      {
-        GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got FIN from P%d (past round)\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-        cpi->ibf_state = IBF_STATE_NONE;
-        cpi->ibf_bucket_counter = 0;
-        break;
-      }
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got FIN from P%d (exp)\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      cpi->exp_subround_finished = GNUNET_YES;
-      if (GNUNET_YES == exp_subround_finished (cpi->session))
-        subround_over (cpi->session, NULL);
-      else
-        GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: still waiting for more after got FIN\n", cpi->session->local_peer_idx);
-    break;
-    case CONSENSUS_ROUND_INVENTORY:
-      cpi->inventory_synced = GNUNET_YES;
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got FIN from P%d (a2a)\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      if (inventory_round_finished (cpi->session))
-        round_over (cpi->session, NULL);
-      break;
-    default:
-      GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "unexpected FIN message the current round\n");
-      break;
-  }
-  return GNUNET_YES;
+  GNUNET_assert (contested_peer < rfn->num_peers);
+
+  rfn->peer_contested[contested_peer] = GNUNET_YES;
 }
 
 
-static struct StrataEstimator *
-strata_estimator_create ()
+static uint16_t
+rfn_noncontested (struct ReferendumEntry *rfn)
 {
-  struct StrataEstimator *se;
-  int i;
-
-  /* fixme: allocate everything in one chunk */
+  uint16_t i;
+  uint16_t ret;
 
-  se = GNUNET_malloc (sizeof (struct StrataEstimator));
-  se->strata = GNUNET_malloc (sizeof (struct InvertibleBloomFilter) * STRATA_COUNT);
-  for (i = 0; i < STRATA_COUNT; i++)
-    se->strata[i] = ibf_create (STRATA_IBF_BUCKETS, STRATA_HASH_NUM, 0);
+  ret = 0;
+  for (i = 0; i < rfn->num_peers; i++)
+    if ( (GNUNET_YES == rfn->peer_commited[i]) && (GNUNET_NO == rfn->peer_contested[i]) )
+      ret++;
 
-  return se;
+  return ret;
 }
 
+
 static void
-strata_estimator_destroy (struct StrataEstimator *se)
+rfn_vote (struct ReferendumEntry *rfn,
+          uint16_t voting_peer,
+          enum ReferendumVote vote,
+          const struct GNUNET_SET_Element *element)
 {
-  int i;
-  for (i = 0; i < STRATA_COUNT; i++)
-    ibf_destroy (se->strata[i]);
-  GNUNET_free (se->strata);
-  GNUNET_free (se);
+  struct RfnElementInfo *ri;
+  struct GNUNET_HashCode hash;
+
+  GNUNET_assert (voting_peer < rfn->num_peers);
+
+  /* Explicit voting only makes sense with VOTE_ADD or VOTE_REMOTE,
+     since VOTE_KEEP is implicit in not voting. */
+  GNUNET_assert ( (VOTE_ADD == vote) || (VOTE_REMOVE == vote) );
+
+  GNUNET_SET_element_hash (element, &hash);
+  ri = GNUNET_CONTAINER_multihashmap_get (rfn->rfn_elements, &hash);
+
+  if (NULL == ri)
+  {
+    ri = GNUNET_new (struct RfnElementInfo);
+    ri->element = GNUNET_SET_element_dup (element);
+    ri->votes = GNUNET_new_array (rfn->num_peers, int);
+    GNUNET_assert (GNUNET_OK ==
+                   GNUNET_CONTAINER_multihashmap_put (rfn->rfn_elements,
+                                                      &hash, ri,
+                                                      GNUNET_CONTAINER_MULTIHASHMAPOPTION_UNIQUE_FAST));
+  }
+
+  ri->votes[voting_peer] = GNUNET_YES;
+  ri->proposal = vote;
+}
+
+
+static uint16_t
+task_other_peer (struct TaskEntry *task)
+{
+  uint16_t me = task->step->session->local_peer_idx;
+  if (task->key.peer1 == me)
+    return task->key.peer2;
+  return task->key.peer1;
 }
 
 
 static int
-is_premature_strata_message (const struct ConsensusSession *session, const struct StrataMessage *strata_msg)
+cmp_uint64_t (const void *pa, const void *pb)
 {
-  switch (strata_msg->round)
-  {
-    case CONSENSUS_ROUND_STOCK:
-    case CONSENSUS_ROUND_EXCHANGE:
-      /* here, we also have to compare subrounds */
-      if ( (strata_msg->round != session->current_round) ||
-           (strata_msg->exp_round != session->exp_round) ||
-           (strata_msg->exp_subround != session->exp_subround))
-        return GNUNET_YES;
-      break;
-    default:
-      if (session->current_round != strata_msg->round)
-        return GNUNET_YES;
-    break;
-  }
-  return GNUNET_NO;
+  uint64_t a = *(uint64_t *) pa;
+  uint64_t b = *(uint64_t *) pb;
+
+  if (a == b)
+    return 0;
+  if (a < b)
+    return -1;
+  return 1;
 }
 
 
 /**
- * Called when a peer sends us its strata estimator.
- * In response, we sent out IBF of appropriate size back.
+ * Callback for set operation results. Called for each element
+ * in the result set.
  *
- * @param cpi session
- * @param strata_msg message
+ * @param cls closure
+ * @param element a result element, only valid if status is GNUNET_SET_STATUS_OK
+ * @param current_size current set size
+ * @param status see enum GNUNET_SET_Status
  */
-static int
-handle_p2p_strata (struct ConsensusPeerInformation *cpi, const struct StrataMessage *strata_msg)
+static void
+set_result_cb (void *cls,
+               const struct GNUNET_SET_Element *element,
+               uint64_t current_size,
+               enum GNUNET_SET_Status status)
 {
-  int i;
-  unsigned int diff;
-  void *buf;
-  size_t size;
+  struct TaskEntry *task = cls;
+  struct ConsensusSession *session = task->step->session;
+  struct SetEntry *output_set = NULL;
+  struct DiffEntry *output_diff = NULL;
+  struct ReferendumEntry *output_rfn = NULL;
+  unsigned int other_idx;
+  struct SetOpCls *setop;
+  const struct ConsensusElement *consensus_element = NULL;
 
-  if ((cpi->session->current_round == CONSENSUS_ROUND_STOCK) && (strata_msg->round == CONSENSUS_ROUND_INVENTORY))
+  if (NULL != element)
   {
-    /* we still have to handle this request appropriately */
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got inventory SE from P%d, we are already further alog\n",
-                cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "P%u: got element of type %u, status %u\n",
+                session->local_peer_idx,
+                (unsigned) element->element_type,
+                (unsigned) status);
+    GNUNET_assert (GNUNET_BLOCK_TYPE_CONSENSUS_ELEMENT == element->element_type);
+    consensus_element = element->data;
   }
-  else if (is_premature_strata_message (cpi->session, strata_msg))
+
+  setop = &task->cls.setop;
+
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "P%u: got set result for {%s}, status %u\n",
+              session->local_peer_idx,
+              debug_str_task_key (&task->key),
+              status);
+
+  if (GNUNET_NO == task->is_started)
   {
-    if (GNUNET_NO == cpi->replaying_strata_message)
-    {
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got probably premature SE from P%d, (%d,%d)\n",
-                  cpi->session->local_peer_idx, (int) (cpi - cpi->session->info), strata_msg->exp_round, strata_msg->exp_subround);
-      cpi->premature_strata_message = (struct StrataMessage *) GNUNET_copy_message ((struct GNUNET_MessageHeader *) strata_msg);
-    }
-    return GNUNET_YES;
+    GNUNET_break_op (0);
+    return;
   }
 
-  if (NULL == cpi->se)
-    cpi->se = strata_estimator_create ();
+  if (GNUNET_YES == task->is_finished)
+  {
+    GNUNET_break_op (0);
+    return;
+  }
 
-  cpi->apparent_round = strata_msg->round;
+  other_idx = task_other_peer (task);
 
-  size = ntohs (strata_msg->header.size);
-  buf = (void *) &strata_msg[1];
-  for (i = 0; i < STRATA_COUNT; i++)
+  if (SET_KIND_NONE != setop->output_set.set_kind)
   {
-    int res;
-    res = ibf_read (&buf, &size, cpi->se->strata[i]);
-    GNUNET_assert (GNUNET_OK == res);
+    output_set = lookup_set (session, &setop->output_set);
+    GNUNET_assert (NULL != output_set);
   }
 
-  diff = estimate_difference (cpi->session->se, cpi->se);
+  if (DIFF_KIND_NONE != setop->output_diff.diff_kind)
+  {
+    output_diff = lookup_diff (session, &setop->output_diff);
+    GNUNET_assert (NULL != output_diff);
+  }
 
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got SE from P%d, diff=%d\n",
-              cpi->session->local_peer_idx, (int) (cpi - cpi->session->info), diff);
+  if (RFN_KIND_NONE != setop->output_rfn.rfn_kind)
+  {
+    output_rfn = lookup_rfn (session, &setop->output_rfn);
+    GNUNET_assert (NULL != output_rfn);
+  }
 
-  switch (cpi->session->current_round)
+  if (GNUNET_YES == session->peers_blacklisted[other_idx])
   {
-    case CONSENSUS_ROUND_EXCHANGE:
-    case CONSENSUS_ROUND_INVENTORY:
-    case CONSENSUS_ROUND_STOCK:
-      /* send IBF of the right size */
-      cpi->ibf_order = 0;
-      while (((1 << cpi->ibf_order) < diff) || STRATA_HASH_NUM > (1 << cpi->ibf_order) )
-        cpi->ibf_order++;
-      if (cpi->ibf_order > MAX_IBF_ORDER)
-        cpi->ibf_order = MAX_IBF_ORDER;
-      cpi->ibf_order += 1;
-      /* create ibf if not already pre-computed */
-      prepare_ibf (cpi);
-      if (NULL != cpi->ibf)
-        ibf_destroy (cpi->ibf);
-      cpi->ibf = ibf_dup (cpi->session->ibfs[cpi->ibf_order]);
-      cpi->ibf_state = IBF_STATE_TRANSMITTING;
-      cpi->ibf_bucket_counter = 0;
-      send_ibf (cpi);
-      break;
-    default:
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: got unexpected SE from P%d\n",
-                  cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      break;
+    /* Peer might have been blacklisted
+       by a gradecast running in parallel, ignore elements from now */
+    if (GNUNET_SET_STATUS_ADD_LOCAL == status)
+      return;
+    if (GNUNET_SET_STATUS_ADD_REMOTE == status)
+      return;
   }
-  return GNUNET_YES;
-}
 
+  if ( (NULL != consensus_element) && (0 != consensus_element->marker) )
+  {
+    GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                "P%u: got some marker\n",
+                  session->local_peer_idx);
+    if ( (GNUNET_YES == setop->transceive_contested) &&
+         (CONSENSUS_MARKER_CONTESTED == consensus_element->marker) )
+    {
+      GNUNET_assert (NULL != output_rfn);
+      rfn_contest (output_rfn, task_other_peer (task));
+      return;
+    }
 
-static int
-handle_p2p_ibf (struct ConsensusPeerInformation *cpi, const struct DifferenceDigest *digest)
-{
-  int num_buckets;
-  void *buf;
+    if (CONSENSUS_MARKER_SIZE == consensus_element->marker)
+    {
 
-  /* FIXME: find out if we're still expecting the same ibf! */
+      GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                  "P%u: got size marker\n",
+                  session->local_peer_idx);
 
-  cpi->apparent_round = cpi->session->current_round;
 
-  num_buckets = (ntohs (digest->header.size) - (sizeof *digest)) / IBF_BUCKET_SIZE;
-  switch (cpi->ibf_state)
+      struct ConsensusSizeElement *cse = (void *) consensus_element;
+
+      if (cse->sender_index == other_idx)
+      {
+        if (NULL == session->first_sizes_received)
+          session->first_sizes_received = GNUNET_new_array (session->num_peers, uint64_t);
+        session->first_sizes_received[other_idx] = GNUNET_ntohll (cse->size);
+
+        uint64_t *copy = GNUNET_memdup (session->first_sizes_received, sizeof (uint64_t) * session->num_peers);
+        qsort (copy, session->num_peers, sizeof (uint64_t), cmp_uint64_t);
+        session->lower_bound = copy[session->num_peers / 3 + 1];
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: lower bound %llu\n",
+                    session->local_peer_idx,
+                    (long long) session->lower_bound);
+      }
+      return;
+    }
+
+    return;
+  }
+
+  switch (status)
   {
-    case IBF_STATE_NONE:
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: receiving IBF from P%d\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      cpi->ibf_state = IBF_STATE_RECEIVING;
-      cpi->ibf_order = digest->order;
-      cpi->ibf_bucket_counter = 0;
-      if (NULL != cpi->ibf)
+    case GNUNET_SET_STATUS_ADD_LOCAL:
+      GNUNET_assert (NULL != consensus_element);
+      GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                  "Adding element in Task {%s}\n",
+                  debug_str_task_key (&task->key));
+      if (NULL != output_set)
+      {
+        // FIXME: record pending adds, use callback
+        GNUNET_SET_add_element (output_set->h,
+                                element,
+                                NULL,
+                                NULL);
+#ifdef GNUNET_EXTRA_LOGGING
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: adding element %s into set {%s} of task {%s}\n",
+                    session->local_peer_idx,
+                    debug_str_element (element),
+                    debug_str_set_key (&setop->output_set),
+                    debug_str_task_key (&task->key));
+#endif
+      }
+      if (NULL != output_diff)
+      {
+        diff_insert (output_diff, 1, element);
+#ifdef GNUNET_EXTRA_LOGGING
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: adding element %s into diff {%s} of task {%s}\n",
+                    session->local_peer_idx,
+                    debug_str_element (element),
+                    debug_str_diff_key (&setop->output_diff),
+                    debug_str_task_key (&task->key));
+#endif
+      }
+      if (NULL != output_rfn)
       {
-        ibf_destroy (cpi->ibf);
-        cpi->ibf = NULL;
+        rfn_vote (output_rfn, task_other_peer (task), VOTE_ADD, element);
+#ifdef GNUNET_EXTRA_LOGGING
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: adding element %s into rfn {%s} of task {%s}\n",
+                    session->local_peer_idx,
+                    debug_str_element (element),
+                    debug_str_rfn_key (&setop->output_rfn),
+                    debug_str_task_key (&task->key));
+#endif
       }
+      // XXX: add result to structures in task
       break;
-    case IBF_STATE_ANTICIPATE_DIFF:
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: receiving IBF from P%d (probably out IBF did not decode)\n",
-                  cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      cpi->ibf_state = IBF_STATE_RECEIVING;
-      cpi->ibf_order = digest->order;
-      cpi->ibf_bucket_counter = 0;
-      if (NULL != cpi->ibf)
+    case GNUNET_SET_STATUS_ADD_REMOTE:
+      GNUNET_assert (NULL != consensus_element);
+      if (GNUNET_YES == setop->do_not_remove)
+        break;
+      if (CONSENSUS_MARKER_CONTESTED == consensus_element->marker)
+        break;
+      GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                  "Removing element in Task {%s}\n",
+                  debug_str_task_key (&task->key));
+      if (NULL != output_set)
       {
-        ibf_destroy (cpi->ibf);
-        cpi->ibf = NULL;
+        // FIXME: record pending adds, use callback
+        GNUNET_SET_remove_element (output_set->h,
+                                   element,
+                                   NULL,
+                                   NULL);
+#ifdef GNUNET_EXTRA_LOGGING
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: removing element %s from set {%s} of task {%s}\n",
+                    session->local_peer_idx,
+                    debug_str_element (element),
+                    debug_str_set_key (&setop->output_set),
+                    debug_str_task_key (&task->key));
+#endif
+      }
+      if (NULL != output_diff)
+      {
+        diff_insert (output_diff, -1, element);
+#ifdef GNUNET_EXTRA_LOGGING
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: removing element %s from diff {%s} of task {%s}\n",
+                    session->local_peer_idx,
+                    debug_str_element (element),
+                    debug_str_diff_key (&setop->output_diff),
+                    debug_str_task_key (&task->key));
+#endif
+      }
+      if (NULL != output_rfn)
+      {
+        rfn_vote (output_rfn, task_other_peer (task), VOTE_REMOVE, element);
+#ifdef GNUNET_EXTRA_LOGGING
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: removing element %s from rfn {%s} of task {%s}\n",
+                    session->local_peer_idx,
+                    debug_str_element (element),
+                    debug_str_rfn_key (&setop->output_rfn),
+                    debug_str_task_key (&task->key));
+#endif
       }
       break;
-    case IBF_STATE_RECEIVING:
+    case GNUNET_SET_STATUS_DONE:
+      // XXX: check first if any changes to the underlying
+      // set are still pending
+      GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                  "Finishing setop in Task {%s}\n",
+                  debug_str_task_key (&task->key));
+      if (NULL != output_rfn)
+      {
+        rfn_commit (output_rfn, task_other_peer (task));
+      }
+      if (PHASE_KIND_ALL_TO_ALL == task->key.kind)
+      {
+        session->first_size = current_size;
+      }
+      finish_task (task);
       break;
+    case GNUNET_SET_STATUS_FAILURE:
+      // XXX: cleanup
+      GNUNET_break_op (0);
+      finish_task (task);
+      return;
     default:
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: unexpected IBF from P%d\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-      return GNUNET_YES;
+      /* not reached */
+      GNUNET_assert (0);
   }
+}
+
+#ifdef EVIL
+
+enum EvilnessType
+{
+  EVILNESS_NONE,
+  EVILNESS_CRAM_ALL,
+  EVILNESS_CRAM_LEAD,
+  EVILNESS_CRAM_ECHO,
+  EVILNESS_SLACK,
+  EVILNESS_SLACK_A2A,
+};
+
+enum EvilnessSubType
+{
+  EVILNESS_SUB_NONE,
+  EVILNESS_SUB_REPLACEMENT,
+  EVILNESS_SUB_NO_REPLACEMENT,
+};
+
+struct Evilness
+{
+  enum EvilnessType type;
+  enum EvilnessSubType subtype;
+  unsigned int num;
+};
 
-  if (cpi->ibf_bucket_counter + num_buckets > (1 << cpi->ibf_order))
+
+static int
+parse_evilness_cram_subtype (const char *evil_subtype_str, struct Evilness *evil)
+{
+  if (0 == strcmp ("replace", evil_subtype_str))
+  {
+    evil->subtype = EVILNESS_SUB_REPLACEMENT;
+  }
+  else if (0 == strcmp ("noreplace", evil_subtype_str))
+  {
+    evil->subtype = EVILNESS_SUB_NO_REPLACEMENT;
+  }
+  else
   {
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: overfull IBF from P%d\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
-    return GNUNET_YES;
+    GNUNET_log (GNUNET_ERROR_TYPE_ERROR,
+                "Malformed field '%s' in EVIL_SPEC (unknown subtype), behaving like a good peer.\n",
+                evil_subtype_str);
+    return GNUNET_SYSERR;
   }
+  return GNUNET_OK;
+}
 
 
-  if (NULL == cpi->ibf)
-    cpi->ibf = ibf_create (1 << cpi->ibf_order, STRATA_HASH_NUM, 0);
+static void
+get_evilness (struct ConsensusSession *session, struct Evilness *evil)
+{
+  char *evil_spec;
+  char *field;
+  char *evil_type_str = NULL;
+  char *evil_subtype_str = NULL;
 
-  buf = (void *) &digest[1];
-  ibf_read_slice (&buf, NULL, cpi->ibf_bucket_counter, num_buckets, cpi->ibf);
+  GNUNET_assert (NULL != evil);
 
-  cpi->ibf_bucket_counter += num_buckets;
+  if (GNUNET_OK != GNUNET_CONFIGURATION_get_value_string (cfg, "consensus", "EVIL_SPEC", &evil_spec))
+  {
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "P%u: no evilness\n",
+                session->local_peer_idx);
+    evil->type = EVILNESS_NONE;
+    return;
+  }
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "P%u: got evilness spec\n",
+              session->local_peer_idx);
 
-  if (cpi->ibf_bucket_counter == (1 << cpi->ibf_order))
+  for (field = strtok (evil_spec, "/");
+       NULL != field;
+       field = strtok (NULL, "/"))
   {
-    cpi->ibf_state = IBF_STATE_DECODING;
-    cpi->ibf_bucket_counter = 0;
-    prepare_ibf (cpi);
-    ibf_subtract (cpi->ibf, cpi->session->ibfs[cpi->ibf_order]);
-    decode (cpi);
+    unsigned int peer_num;
+    unsigned int evil_num;
+    int ret;
+
+    evil_type_str = NULL;
+    evil_subtype_str = NULL;
+
+    ret = sscanf (field, "%u;%m[a-z-];%m[a-z-];%u", &peer_num, &evil_type_str, &evil_subtype_str, &evil_num);
+
+    if (ret != 4)
+    {
+      GNUNET_log (GNUNET_ERROR_TYPE_ERROR,
+                  "Malformed field '%s' in EVIL_SPEC (expected 4 components got %d), behaving like a good peer.\n",
+                  field,
+                  ret);
+      goto not_evil;
+    }
+
+    GNUNET_assert (NULL != evil_type_str);
+    GNUNET_assert (NULL != evil_subtype_str);
+
+    if (peer_num == session->local_peer_idx)
+    {
+      if (0 == strcmp ("slack", evil_type_str))
+      {
+        evil->type = EVILNESS_SLACK;
+      }
+      if (0 == strcmp ("slack-a2a", evil_type_str))
+      {
+        evil->type = EVILNESS_SLACK_A2A;
+      }
+      else if (0 == strcmp ("cram-all", evil_type_str))
+      {
+        evil->type = EVILNESS_CRAM_ALL;
+        evil->num = evil_num;
+        if (GNUNET_OK != parse_evilness_cram_subtype (evil_subtype_str, evil))
+          goto not_evil;
+      }
+      else if (0 == strcmp ("cram-lead", evil_type_str))
+      {
+        evil->type = EVILNESS_CRAM_LEAD;
+        evil->num = evil_num;
+        if (GNUNET_OK != parse_evilness_cram_subtype (evil_subtype_str, evil))
+          goto not_evil;
+      }
+      else if (0 == strcmp ("cram-echo", evil_type_str))
+      {
+        evil->type = EVILNESS_CRAM_ECHO;
+        evil->num = evil_num;
+        if (GNUNET_OK != parse_evilness_cram_subtype (evil_subtype_str, evil))
+          goto not_evil;
+      }
+      else
+      {
+        GNUNET_log (GNUNET_ERROR_TYPE_ERROR,
+                    "Malformed field '%s' in EVIL_SPEC (unknown type), behaving like a good peer.\n",
+                    evil_type_str);
+        goto not_evil;
+      }
+      goto cleanup;
+    }
+    /* No GNUNET_free since memory was allocated by libc */
+    free (evil_type_str);
+    evil_type_str = NULL;
+    evil_subtype_str = NULL;
   }
-  return GNUNET_YES;
+not_evil:
+  evil->type = EVILNESS_NONE;
+cleanup:
+  GNUNET_free (evil_spec);
+  /* no GNUNET_free_non_null since it wasn't
+   * allocated with GNUNET_malloc */
+  if (NULL != evil_type_str)
+    free (evil_type_str);
+  if (NULL != evil_subtype_str)
+    free (evil_subtype_str);
 }
 
+#endif
+
 
 /**
- * Handle an element that another peer sent us
+ * Commit the appropriate set for a
+ * task.
  */
-static int
-handle_p2p_element (struct ConsensusPeerInformation *cpi, const struct GNUNET_MessageHeader *element_msg)
+static void
+commit_set (struct ConsensusSession *session,
+            struct TaskEntry *task)
 {
-  struct PendingElement *pending_element;
-  struct GNUNET_CONSENSUS_Element *element;
-  struct GNUNET_CONSENSUS_ElementMessage *client_element_msg;
-  size_t size;
+  struct SetEntry *set;
+  struct SetOpCls *setop = &task->cls.setop;
 
-  switch (cpi->session->current_round)
+  GNUNET_assert (NULL != setop->op);
+  set = lookup_set (session, &setop->input_set);
+  GNUNET_assert (NULL != set);
+
+  if ( (GNUNET_YES == setop->transceive_contested) && (GNUNET_YES == set->is_contested) )
   {
-    case CONSENSUS_ROUND_STOCK:
-      /* FIXME: check if we really expect the element */
-    case CONSENSUS_ROUND_EXCHANGE:
-      break;
-    default:
-      GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "got unexpected element, ignoring\n");
-      return GNUNET_YES;
+    struct GNUNET_SET_Element element;
+    struct ConsensusElement ce = { 0 };
+    ce.marker = CONSENSUS_MARKER_CONTESTED;
+    element.data = &ce;
+    element.size = sizeof (struct ConsensusElement);
+    element.element_type = GNUNET_BLOCK_TYPE_CONSENSUS_ELEMENT;
+    GNUNET_SET_add_element (set->h, &element, NULL, NULL);
   }
 
-  size = ntohs (element_msg->size) - sizeof *element_msg;
+  if (PHASE_KIND_ALL_TO_ALL_2 == task->key.kind)
+  {
+    struct GNUNET_SET_Element element;
+    struct ConsensusSizeElement cse = { 0 };
+    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "inserting size marker\n");
+    cse.ce.marker = CONSENSUS_MARKER_SIZE;
+    cse.size = GNUNET_htonll (session->first_size);
+    cse.sender_index = session->local_peer_idx;
+    element.data = &cse;
+    element.size = sizeof (struct ConsensusSizeElement);
+    element.element_type = GNUNET_BLOCK_TYPE_CONSENSUS_ELEMENT;
+    GNUNET_SET_add_element (set->h, &element, NULL, NULL);
+  }
 
-  element = GNUNET_malloc (size + sizeof *element);
-  element->size = size;
-  memcpy (&element[1], &element_msg[1], size);
-  element->data = &element[1];
+#ifdef EVIL
+  {
+    unsigned int i;
+    struct Evilness evil;
 
-  pending_element = GNUNET_malloc (sizeof *pending_element);
-  pending_element->element = element;
-  GNUNET_CONTAINER_DLL_insert_tail (cpi->session->client_approval_head, cpi->session->client_approval_tail, pending_element);
+    get_evilness (session, &evil);
+    if (EVILNESS_NONE != evil.type)
+    {
+      /* Useful for evaluation */
+      GNUNET_STATISTICS_set (statistics,
+                             "is evil",
+                             1,
+                             GNUNET_NO);
+    }
+    switch (evil.type)
+    {
+      case EVILNESS_CRAM_ALL:
+      case EVILNESS_CRAM_LEAD:
+      case EVILNESS_CRAM_ECHO:
+        /* We're not cramming elements in the
+           all-to-all round, since that would just
+           add more elements to the result set, but
+           wouldn't test robustness. */
+        if (PHASE_KIND_ALL_TO_ALL == task->key.kind)
+        {
+          GNUNET_SET_commit (setop->op, set->h);
+          break;
+        }
+        if ((EVILNESS_CRAM_LEAD == evil.type) &&
+            ((PHASE_KIND_GRADECAST_LEADER != task->key.kind) || SET_KIND_CURRENT != set->key.set_kind))
+        {
+          GNUNET_SET_commit (setop->op, set->h);
+          break;
+        }
+        if (EVILNESS_CRAM_ECHO == evil.type && (PHASE_KIND_GRADECAST_ECHO != task->key.kind))
+        {
+          GNUNET_SET_commit (setop->op, set->h);
+          break;
+        }
+        for (i = 0; i < evil.num; i++)
+        {
+          struct GNUNET_SET_Element element;
+          struct ConsensusStuffedElement se = { 0 };
+          element.data = &se;
+          element.size = sizeof (struct ConsensusStuffedElement);
+          element.element_type = GNUNET_BLOCK_TYPE_CONSENSUS_ELEMENT;
+
+          if (EVILNESS_SUB_REPLACEMENT == evil.subtype)
+          {
+            /* Always generate a new element. */
+            GNUNET_CRYPTO_hash_create_random (GNUNET_CRYPTO_QUALITY_WEAK, &se.rand);
+          }
+          else if (EVILNESS_SUB_NO_REPLACEMENT == evil.subtype)
+          {
+            /* Always cram the same elements, derived from counter. */
+            GNUNET_CRYPTO_hash (&i, sizeof (i), &se.rand);
+          }
+          else
+          {
+            GNUNET_assert (0);
+          }
+          GNUNET_SET_add_element (set->h, &element, NULL, NULL);
+#ifdef GNUNET_EXTRA_LOGGING
+          GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                      "P%u: evil peer: cramming element %s into set {%s} of task {%s}\n",
+                      session->local_peer_idx,
+                      debug_str_element (&element),
+                      debug_str_set_key (&setop->input_set),
+                      debug_str_task_key (&task->key));
+#endif
+        }
+        GNUNET_STATISTICS_update (statistics,
+                                  "# stuffed elements",
+                                  evil.num,
+                                  GNUNET_NO);
+        GNUNET_SET_commit (setop->op, set->h);
+        break;
+      case EVILNESS_SLACK:
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: evil peer: slacking\n",
+                    (unsigned int) session->local_peer_idx);
+        /* Do nothing. */
+      case EVILNESS_SLACK_A2A:
+        if ( (PHASE_KIND_ALL_TO_ALL_2 == task->key.kind ) ||
+             (PHASE_KIND_ALL_TO_ALL == task->key.kind) )
+        {
+          struct GNUNET_SET_Handle *empty_set;
+          empty_set = GNUNET_SET_create (cfg, GNUNET_SET_OPERATION_UNION);
+          GNUNET_SET_commit (setop->op, empty_set);
+          GNUNET_SET_destroy (empty_set);
+        }
+        else
+        {
+          GNUNET_SET_commit (setop->op, set->h);
+        }
+        break;
+      case EVILNESS_NONE:
+        GNUNET_SET_commit (setop->op, set->h);
+        break;
+    }
+  }
+#else
+  if (GNUNET_NO == session->peers_blacklisted[task_other_peer (task)])
+  {
+    GNUNET_SET_commit (setop->op, set->h);
+  }
+  else
+  {
+    /* For our testcases, we don't want the blacklisted
+       peers to wait. */
+    GNUNET_SET_operation_cancel (setop->op);
+    setop->op = NULL;
+  }
+#endif
+}
 
-  client_element_msg = GNUNET_malloc (size + sizeof *client_element_msg);
-  client_element_msg->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_RECEIVED_ELEMENT);
-  client_element_msg->header.size = htons (size + sizeof *client_element_msg);
-  memcpy (&client_element_msg[1], &element[1], size);
 
-  queue_client_message (cpi->session, (struct GNUNET_MessageHeader *) client_element_msg);
+static void
+put_diff (struct ConsensusSession *session,
+         struct DiffEntry *diff)
+{
+  struct GNUNET_HashCode hash;
 
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "received element, size=%d\n", size);
+  GNUNET_assert (NULL != diff);
 
-  client_send_next (cpi->session);
-  
-  return GNUNET_YES;
+  GNUNET_CRYPTO_hash (&diff->key, sizeof (struct DiffKey), &hash);
+  GNUNET_assert (GNUNET_OK ==
+                 GNUNET_CONTAINER_multihashmap_put (session->diffmap, &hash, diff,
+                                                    GNUNET_CONTAINER_MULTIHASHMAPOPTION_UNIQUE_ONLY));
 }
 
+static void
+put_set (struct ConsensusSession *session,
+         struct SetEntry *set)
+{
+  struct GNUNET_HashCode hash;
 
-/**
- * Handle a request for elements.
- * 
- * @param cpi peer that is requesting the element
- * @param msg the element request message
- */
-static int
-handle_p2p_element_request (struct ConsensusPeerInformation *cpi, const struct ElementRequest *msg)
+  GNUNET_assert (NULL != set->h);
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Putting set %s\n",
+              debug_str_set_key (&set->key));
+
+  GNUNET_CRYPTO_hash (&set->key, sizeof (struct SetKey), &hash);
+  GNUNET_assert (GNUNET_SYSERR !=
+                 GNUNET_CONTAINER_multihashmap_put (session->setmap, &hash, set,
+                                                    GNUNET_CONTAINER_MULTIHASHMAPOPTION_REPLACE));
+}
+
+
+static void
+put_rfn (struct ConsensusSession *session,
+         struct ReferendumEntry *rfn)
 {
-  struct GNUNET_HashCode hashcode;
-  struct IBF_Key *ibf_key;
-  unsigned int num;
+  struct GNUNET_HashCode hash;
 
-  /* element requests are allowed in every round */
+  GNUNET_CRYPTO_hash (&rfn->key, sizeof (struct RfnKey), &hash);
+  GNUNET_assert (GNUNET_OK ==
+                 GNUNET_CONTAINER_multihashmap_put (session->rfnmap, &hash, rfn,
+                                                    GNUNET_CONTAINER_MULTIHASHMAPOPTION_UNIQUE_ONLY));
+}
 
-  num = ntohs (msg->header.size) / sizeof (struct IBF_Key);
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "handling element request for %u elements\n", num);
-  
-  ibf_key = (struct IBF_Key *) &msg[1];
-  while (num--)
-  {
-    struct ElementList *head;
-    ibf_hashcode_from_key (*ibf_key, &hashcode);
-    head = GNUNET_CONTAINER_multihashmap_get (cpi->session->values, &hashcode);
-    send_elements (cpi, head);
-    ibf_key++;
-  }
-  return GNUNET_YES;
+
+
+static void
+task_cancel_reconcile (struct TaskEntry *task)
+{
+  /* not implemented yet */
+  GNUNET_assert (0);
 }
 
-/**
- * If necessary, send a message to the peer, depending on the current
- * round.
- */
+
 static void
-embrace_peer (struct ConsensusPeerInformation *cpi)
+apply_diff_to_rfn (struct DiffEntry *diff,
+                   struct ReferendumEntry *rfn,
+                   uint16_t voting_peer,
+                   uint16_t num_peers)
 {
-  GNUNET_assert (GNUNET_YES == cpi->hello);
-  switch (cpi->session->current_round)
+  struct GNUNET_CONTAINER_MultiHashMapIterator *iter;
+  struct DiffElementInfo *di;
+
+  iter = GNUNET_CONTAINER_multihashmap_iterator_create (diff->changes);
+
+  while (GNUNET_YES ==
+         GNUNET_CONTAINER_multihashmap_iterator_next (iter,
+                                                      NULL,
+                                                      (const void **) &di))
   {
-    case CONSENSUS_ROUND_EXCHANGE:
-      if (cpi->session->partner_outgoing != cpi)
-        break;
-      /* fallthrough */
-    case CONSENSUS_ROUND_INVENTORY:
-      /* fallthrough */
-    case CONSENSUS_ROUND_STOCK:
-      if (cpi == cpi->session->partner_outgoing)
-        send_strata_estimator (cpi);
-    default:
-      break;
+    if (di->weight > 0)
+    {
+      rfn_vote (rfn, voting_peer, VOTE_ADD, di->element);
+    }
+    if (di->weight < 0)
+    {
+      rfn_vote (rfn, voting_peer, VOTE_REMOVE, di->element);
+    }
   }
+
+  GNUNET_CONTAINER_multihashmap_iterator_destroy (iter);
 }
 
 
-/**
- * Handle a HELLO-message, send when another peer wants to join a session where
- * our peer is a member. The session may or may not be inhabited yet.
- */
-static int
-handle_p2p_hello (struct IncomingSocket *inc, const struct ConsensusHello *hello)
+struct DiffEntry *
+diff_create ()
 {
-  /* FIXME: session might not exist yet. create an uninhabited session and wait for a client */
-  struct ConsensusSession *session;
+  struct DiffEntry *d = GNUNET_new (struct DiffEntry);
+
+  d->changes = GNUNET_CONTAINER_multihashmap_create (8, GNUNET_NO);
+
+  return d;
+}
+
 
-  session = sessions_head;
-  while (NULL != session)
+struct DiffEntry *
+diff_compose (struct DiffEntry *diff_1,
+              struct DiffEntry *diff_2)
+{
+  struct DiffEntry *diff_new;
+  struct GNUNET_CONTAINER_MultiHashMapIterator *iter;
+  struct DiffElementInfo *di;
+
+  diff_new = diff_create ();
+
+  iter = GNUNET_CONTAINER_multihashmap_iterator_create (diff_1->changes);
+  while (GNUNET_YES == GNUNET_CONTAINER_multihashmap_iterator_next (iter, NULL, (const void **) &di))
   {
-    if (0 == GNUNET_CRYPTO_hash_cmp (&session->global_id, &hello->global_id))
-    {
-      int idx;
-      idx = get_peer_idx (&inc->peer_id, session);
-      GNUNET_assert (-1 != idx);
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "peer %d hello'ed session %d\n", idx);
-      inc->cpi = &session->info[idx];
-      inc->cpi->mst = inc->mst;
-      inc->cpi->hello = GNUNET_YES;
-      inc->cpi->socket = inc->socket;
-      embrace_peer (inc->cpi);
-      return GNUNET_YES;
-    }
-    session = session->next;
+    diff_insert (diff_new, di->weight, di->element);
   }
-  GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "peer tried to HELLO uninhabited session\n");
-  return GNUNET_NO;
+  GNUNET_CONTAINER_multihashmap_iterator_destroy (iter);
+
+  iter = GNUNET_CONTAINER_multihashmap_iterator_create (diff_2->changes);
+  while (GNUNET_YES == GNUNET_CONTAINER_multihashmap_iterator_next (iter, NULL, (const void **) &di))
+  {
+    diff_insert (diff_new, di->weight, di->element);
+  }
+  GNUNET_CONTAINER_multihashmap_iterator_destroy (iter);
+
+  return diff_new;
+}
+
+
+struct ReferendumEntry *
+rfn_create (uint16_t size)
+{
+  struct ReferendumEntry *rfn;
+
+  rfn = GNUNET_new (struct ReferendumEntry);
+  rfn->rfn_elements = GNUNET_CONTAINER_multihashmap_create (8, GNUNET_NO);
+  rfn->peer_commited = GNUNET_new_array (size, int);
+  rfn->peer_contested = GNUNET_new_array (size, int);
+  rfn->num_peers = size;
+
+  return rfn;
+}
+
+
+#if UNUSED
+static void
+diff_destroy (struct DiffEntry *diff)
+{
+  GNUNET_CONTAINER_multihashmap_destroy (diff->changes);
+  GNUNET_free (diff);
 }
+#endif
 
 
 /**
- * Send a strata estimator.
- *
- * @param cpi the peer
+ * For a given majority, count what the outcome
+ * is (add/remove/keep), and give the number
+ * of peers that voted for this outcome.
  */
 static void
-send_strata_estimator (struct ConsensusPeerInformation *cpi)
+rfn_majority (const struct ReferendumEntry *rfn,
+              const struct RfnElementInfo *ri,
+              uint16_t *ret_majority,
+              enum ReferendumVote *ret_vote)
 {
-  struct StrataMessage *strata_msg;
-  void *buf;
-  size_t msize;
-  int i;
+  uint16_t votes_yes = 0;
+  uint16_t num_commited = 0;
+  uint16_t i;
 
-  cpi->apparent_round = cpi->session->current_round;
-  cpi->ibf_state = IBF_STATE_NONE;
-  cpi->ibf_bucket_counter = 0;
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Computing rfn majority for element %s of rfn {%s}\n",
+              debug_str_element (ri->element),
+              debug_str_rfn_key (&rfn->key));
 
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: sending SE(%d) to P%d\n",
-              cpi->session->local_peer_idx, cpi->session->current_round, (int) (cpi - cpi->session->info));
+  for (i = 0; i < rfn->num_peers; i++)
+  {
+    if (GNUNET_NO == rfn->peer_commited[i])
+      continue;
+    num_commited++;
 
-  msize = (sizeof *strata_msg) + (STRATA_COUNT * IBF_BUCKET_SIZE * STRATA_IBF_BUCKETS);
+    if (GNUNET_YES == ri->votes[i])
+      votes_yes++;
+  }
 
-  strata_msg = GNUNET_malloc (msize);
-  strata_msg->header.size = htons (msize);
-  strata_msg->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_DELTA_ESTIMATE);
-  strata_msg->round = cpi->session->current_round;
-  strata_msg->exp_round = cpi->session->exp_round;
-  strata_msg->exp_subround = cpi->session->exp_subround;
-  
-  buf = &strata_msg[1];
-  for (i = 0; i < STRATA_COUNT; i++)
+  if (votes_yes > (num_commited) / 2)
+  {
+    *ret_vote = ri->proposal;
+    *ret_majority = votes_yes;
+  }
+  else
   {
-    ibf_write (cpi->session->se->strata[i], &buf, NULL);
+    *ret_vote = VOTE_STAY;
+    *ret_majority = num_commited - votes_yes;
   }
+}
+
+
+struct SetCopyCls
+{
+  struct TaskEntry *task;
+  struct SetKey dst_set_key;
+};
+
+
+static void
+set_copy_cb (void *cls, struct GNUNET_SET_Handle *copy)
+{
+  struct SetCopyCls *scc = cls;
+  struct TaskEntry *task = scc->task;
+  struct SetKey dst_set_key = scc->dst_set_key;
+  struct SetEntry *set;
 
-  queue_peer_message (cpi, (struct GNUNET_MessageHeader *) strata_msg);
+  GNUNET_free (scc);
+  set = GNUNET_new (struct SetEntry);
+  set->h = copy;
+  set->key = dst_set_key;
+  put_set (task->step->session, set);
+
+  task->start (task);
 }
 
 
 /**
- * Send an IBF of the order specified in cpi.
- *
- * @param cpi the peer
+ * Call the start function of the given
+ * task again after we created a copy of the given set.
  */
 static void
-send_ibf (struct ConsensusPeerInformation *cpi)
+create_set_copy_for_task (struct TaskEntry *task,
+                          struct SetKey *src_set_key,
+                          struct SetKey *dst_set_key)
 {
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: sending IBF to P%d\n",
-              cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
+  struct SetEntry *src_set;
+  struct SetCopyCls *scc = GNUNET_new (struct SetCopyCls);
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Copying set {%s} to {%s} for task {%s}\n",
+              debug_str_set_key (src_set_key),
+              debug_str_set_key (dst_set_key),
+              debug_str_task_key (&task->key));
+
+  scc->task = task;
+  scc->dst_set_key = *dst_set_key;
+  src_set = lookup_set (task->step->session, src_set_key);
+  GNUNET_assert (NULL != src_set);
+  GNUNET_SET_copy_lazy (src_set->h,
+                        set_copy_cb,
+                        scc);
+}
+
 
-  cpi->ibf_bucket_counter = 0;
-  while (cpi->ibf_bucket_counter < (1 << cpi->ibf_order))
+struct SetMutationProgressCls
+{
+  int num_pending;
+  /**
+   * Task to finish once all changes are through.
+   */
+  struct TaskEntry *task;
+};
+
+
+static void
+set_mutation_done (void *cls)
+{
+  struct SetMutationProgressCls *pc = cls;
+
+  GNUNET_assert (pc->num_pending > 0);
+
+  pc->num_pending--;
+
+  if (0 == pc->num_pending)
   {
-    int num_buckets;
-    void *buf;
-    struct DifferenceDigest *digest;
-    int msize;
+    struct TaskEntry *task = pc->task;
+    GNUNET_free (pc);
+    finish_task (task);
+  }
+}
 
-    num_buckets = (1 << cpi->ibf_order) - cpi->ibf_bucket_counter;
-    /* limit to maximum */
-    if (num_buckets > BUCKETS_PER_MESSAGE)
-      num_buckets = BUCKETS_PER_MESSAGE;
 
-    msize = (sizeof *digest) + (num_buckets * IBF_BUCKET_SIZE);
+static void
+try_finish_step_early (struct Step *step)
+{
+  unsigned int i;
 
-    digest = GNUNET_malloc (msize);
-    digest->header.size = htons (msize);
-    digest->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_DIFFERENCE_DIGEST);
-    digest->order = cpi->ibf_order;
-    digest->round = cpi->apparent_round;
+  if (GNUNET_YES == step->is_running)
+    return;
+  if (GNUNET_YES == step->is_finished)
+    return;
+  if (GNUNET_NO == step->early_finishable)
+    return;
+
+  step->is_finished = GNUNET_YES;
+
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Finishing step `%s' early.\n",
+              step->debug_name);
+#endif
 
-    buf = &digest[1];
-    ibf_write_slice (cpi->ibf, cpi->ibf_bucket_counter, num_buckets, &buf, NULL);
+  for (i = 0; i < step->subordinates_len; i++)
+  {
+    GNUNET_assert (step->subordinates[i]->pending_prereq > 0);
+    step->subordinates[i]->pending_prereq--;
+#ifdef GNUNET_EXTRA_LOGGING
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "Decreased pending_prereq to %u for step `%s'.\n",
+                (unsigned int) step->subordinates[i]->pending_prereq,
+                step->subordinates[i]->debug_name);
+
+#endif
+    try_finish_step_early (step->subordinates[i]);
+  }
+
+  // XXX: maybe schedule as task to avoid recursion?
+  run_ready_steps (step->session);
+}
+
+
+static void
+finish_step (struct Step *step)
+{
+  unsigned int i;
+
+  GNUNET_assert (step->finished_tasks == step->tasks_len);
+  GNUNET_assert (GNUNET_YES == step->is_running);
+  GNUNET_assert (GNUNET_NO == step->is_finished);
+
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "All tasks of step `%s' with %u subordinates finished.\n",
+              step->debug_name,
+              step->subordinates_len);
+#endif
 
-    queue_peer_message (cpi, (struct GNUNET_MessageHeader *) digest);
+  for (i = 0; i < step->subordinates_len; i++)
+  {
+    GNUNET_assert (step->subordinates[i]->pending_prereq > 0);
+    step->subordinates[i]->pending_prereq--;
+#ifdef GNUNET_EXTRA_LOGGING
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "Decreased pending_prereq to %u for step `%s'.\n",
+                (unsigned int) step->subordinates[i]->pending_prereq,
+                step->subordinates[i]->debug_name);
 
-    cpi->ibf_bucket_counter += num_buckets;
+#endif
   }
-  cpi->ibf_bucket_counter = 0;
-  cpi->ibf_state = IBF_STATE_ANTICIPATE_DIFF;
+
+  step->is_finished = GNUNET_YES;
+
+  // XXX: maybe schedule as task to avoid recursion?
+  run_ready_steps (step->session);
 }
 
 
+
 /**
- * Decode the current diff ibf, and send elements/requests/reports/
+ * Apply the result from one round of gradecasts (i.e. every peer
+ * should have gradecasted) to the peer's current set.
  *
- * @param cpi partner peer
+ * @param task the task with context information
  */
 static void
-decode (struct ConsensusPeerInformation *cpi)
-{
-  struct IBF_Key key;
-  struct GNUNET_HashCode hashcode;
-  int side;
+task_start_apply_round (struct TaskEntry *task)
+{
+  struct ConsensusSession *session = task->step->session;
+  struct SetKey sk_in;
+  struct SetKey sk_out;
+  struct RfnKey rk_in;
+  struct SetEntry *set_out;
+  struct ReferendumEntry *rfn_in;
+  struct GNUNET_CONTAINER_MultiHashMapIterator *iter;
+  struct RfnElementInfo *ri;
+  struct SetMutationProgressCls *progress_cls;
+  uint16_t worst_majority = UINT16_MAX;
+
+  sk_in = (struct SetKey) { SET_KIND_CURRENT, task->key.repetition };
+  rk_in = (struct RfnKey) { RFN_KIND_GRADECAST_RESULT, task->key.repetition };
+  sk_out = (struct SetKey) { SET_KIND_CURRENT, task->key.repetition + 1 };
+
+  set_out = lookup_set (session, &sk_out);
+  if (NULL == set_out)
+  {
+    create_set_copy_for_task (task, &sk_in, &sk_out);
+    return;
+  }
 
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: decoding ibf from P%d\n", cpi->session->local_peer_idx, (int) (cpi - cpi->session->info));
+  rfn_in = lookup_rfn (session, &rk_in);
+  GNUNET_assert (NULL != rfn_in);
 
-  for (;;)
+  progress_cls = GNUNET_new (struct SetMutationProgressCls);
+  progress_cls->task = task;
+
+  iter = GNUNET_CONTAINER_multihashmap_iterator_create (rfn_in->rfn_elements);
+
+  while (GNUNET_YES ==
+         GNUNET_CONTAINER_multihashmap_iterator_next (iter,
+                                                      NULL,
+                                                      (const void **) &ri))
   {
-    int res;
+    uint16_t majority_num;
+    enum ReferendumVote majority_vote;
 
-    res = ibf_decode (cpi->ibf, &side, &key);
-    if (GNUNET_SYSERR == res)
-    {
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "decoding failed, transmitting larger IBF\n");
-      /* decoding failed, we tell the other peer by sending our ibf with a larger order */
-      cpi->ibf_order++;
-      prepare_ibf (cpi);
-      cpi->ibf = ibf_dup (cpi->session->ibfs[cpi->ibf_order]);
-      cpi->ibf_state = IBF_STATE_TRANSMITTING;
-      cpi->ibf_bucket_counter = 0;
-      send_ibf (cpi);
-      return;
-    }
-    if (GNUNET_NO == res)
-    {
-      struct ConsensusRoundMessage *msg;
-      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: transmitted all values, sending SYNC\n", cpi->session->local_peer_idx);
-      msg = GNUNET_malloc (sizeof *msg);
-      msg->header.size = htons (sizeof *msg);
-      msg->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_SYNCED);
-      msg->round = cpi->apparent_round;
-      queue_peer_message (cpi, (struct GNUNET_MessageHeader *) msg);
-      return;
-    }
-    if (-1 == side)
+    rfn_majority (rfn_in, ri, &majority_num, &majority_vote);
+
+    if (worst_majority > majority_num)
+      worst_majority = majority_num;
+
+    switch (majority_vote)
     {
-      struct ElementList *head;
-      /* we have the element(s), send it to the other peer */
-      ibf_hashcode_from_key (key, &hashcode);
-      head = GNUNET_CONTAINER_multihashmap_get (cpi->session->values, &hashcode);
-      send_elements (cpi, head);
+      case VOTE_ADD:
+        progress_cls->num_pending++;
+        GNUNET_assert (GNUNET_OK ==
+                       GNUNET_SET_add_element (set_out->h,
+                                               ri->element,
+                                               &set_mutation_done,
+                                               progress_cls));
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: apply round: adding element %s with %u-majority.\n",
+                    session->local_peer_idx,
+                    debug_str_element (ri->element), majority_num);
+        break;
+      case VOTE_REMOVE:
+        progress_cls->num_pending++;
+        GNUNET_assert (GNUNET_OK ==
+                       GNUNET_SET_remove_element (set_out->h,
+                                                  ri->element,
+                                                  &set_mutation_done,
+                                                  progress_cls));
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: apply round: deleting element %s with %u-majority.\n",
+                    session->local_peer_idx,
+                    debug_str_element (ri->element), majority_num);
+        break;
+      case VOTE_STAY:
+        GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                    "P%u: apply round: keeping element %s with %u-majority.\n",
+                    session->local_peer_idx,
+                    debug_str_element (ri->element), majority_num);
+        // do nothing
+        break;
+      default:
+        GNUNET_assert (0);
+        break;
     }
-    else
-    {
-      struct ElementRequest *msg;
-      size_t msize;
-      struct IBF_Key *p;
+  }
 
-      msize = (sizeof *msg) + sizeof (struct IBF_Key);
-      msg = GNUNET_malloc (msize);
-      switch (cpi->apparent_round)
+  if (0 == progress_cls->num_pending)
+  {
+    // call closure right now, no pending ops
+    GNUNET_free (progress_cls);
+    finish_task (task);
+  }
+
+  {
+    uint16_t thresh = (session->num_peers / 3) * 2;
+
+    if (worst_majority >= thresh)
+    {
+      switch (session->early_stopping)
       {
-        case CONSENSUS_ROUND_STOCK:
-          /* FIXME: check if we really want to request the element */
-        case CONSENSUS_ROUND_EXCHANGE:
-          msg->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS_REQUEST);
+        case EARLY_STOPPING_NONE:
+          session->early_stopping = EARLY_STOPPING_ONE_MORE;
+          GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+                      "P%u: Stopping early (after one more superround)\n",
+                      session->local_peer_idx);
           break;
-        case CONSENSUS_ROUND_INVENTORY:
-          msg->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS_REPORT);
+        case EARLY_STOPPING_ONE_MORE:
+          GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%u: finishing steps due to early finish\n",
+                      session->local_peer_idx);
+          session->early_stopping = EARLY_STOPPING_DONE;
+          {
+            struct Step *step;
+            for (step = session->steps_head; NULL != step; step = step->next)
+              try_finish_step_early (step);
+          }
+          break;
+        case EARLY_STOPPING_DONE:
+          /* We shouldn't be here anymore after early stopping */
+          GNUNET_break (0);
           break;
         default:
           GNUNET_assert (0);
+          break;
       }
-      msg->header.size = htons (msize);
-      p = (struct IBF_Key *) &msg[1];
-      *p = key;
-      queue_peer_message (cpi, (struct GNUNET_MessageHeader *) msg);
+    }
+    else if (EARLY_STOPPING_NONE != session->early_stopping)
+    {
+      // Our assumption about the number of bad peers
+      // has been broken.
+      GNUNET_break_op (0);
+    }
+    else
+    {
+      GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%u: NOT finishing early (majority not good enough)\n",
+                  session->local_peer_idx);
     }
   }
+  GNUNET_CONTAINER_multihashmap_iterator_destroy (iter);
 }
-
-
-/**
- * Functions with this signature are called whenever a
- * complete message is received by the tokenizer.
- *
- * Do not call GNUNET_SERVER_mst_destroy in callback
- *
- * @param cls closure
- * @param client identification of the client
- * @param message the actual message
- * @return GNUNET_OK on success, GNUNET_SYSERR to stop further processing
- */
-static int
-mst_session_callback (void *cls, void *client, const struct GNUNET_MessageHeader *message)
-{
-  struct ConsensusPeerInformation *cpi;
-  cpi =  cls;
-  switch (ntohs (message->type))
-  {
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_DELTA_ESTIMATE:
-      return handle_p2p_strata (cpi, (struct StrataMessage *) message);
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_DIFFERENCE_DIGEST:
-      return handle_p2p_ibf (cpi, (struct DifferenceDigest *) message);
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS:
-      return handle_p2p_element (cpi, message);
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS_REPORT:
-      return handle_p2p_element_report (cpi, message);
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ELEMENTS_REQUEST:
-      return handle_p2p_element_request (cpi, (struct ElementRequest *) message);
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_SYNCED:
-      return handle_p2p_synced (cpi, message);
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_FIN:
-      return handle_p2p_fin (cpi, message);
-    default:
-      GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "ignoring unexpected message type (%u) from peer: %s\n",
-                  ntohs (message->type), GNUNET_h2s (&cpi->peer_id.hashPubKey));
-  }
-  return GNUNET_OK;
-}
-
-
-/**
- * Handle tokenized messages from stream sockets.
- * Delegate them if the socket belongs to a session,
- * handle hello messages otherwise.
- *
- * Do not call GNUNET_SERVER_mst_destroy in callback
- *
- * @param cls closure, unused
- * @param client incoming socket this message comes from
- * @param message the actual message
- *
- * @return GNUNET_OK on success, GNUNET_SYSERR to stop further processing
- */
-static int
-mst_incoming_callback (void *cls, void *client, const struct GNUNET_MessageHeader *message)
+
+
+static void
+task_start_grade (struct TaskEntry *task)
 {
-  struct IncomingSocket *inc;
-  inc = (struct IncomingSocket *) client;
-  switch (ntohs( message->type))
+  struct ConsensusSession *session = task->step->session;
+  struct ReferendumEntry *output_rfn;
+  struct ReferendumEntry *input_rfn;
+  struct DiffEntry *input_diff;
+  struct RfnKey rfn_key;
+  struct DiffKey diff_key;
+  struct GNUNET_CONTAINER_MultiHashMapIterator *iter;
+  struct RfnElementInfo *ri;
+  unsigned int gradecast_confidence = 2;
+
+  rfn_key = (struct RfnKey) { RFN_KIND_GRADECAST_RESULT, task->key.repetition };
+  output_rfn = lookup_rfn (session, &rfn_key);
+  if (NULL == output_rfn)
   {
-    case GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_HELLO:
-      return handle_p2p_hello (inc, (struct ConsensusHello *) message);
-    default:
-      if (NULL != inc->cpi)
-        return mst_session_callback (inc->cpi, client, message);
-      GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "ignoring unexpected message type (%u) from peer: %s (not in session)\n",
-                  ntohs (message->type), GNUNET_h2s (&inc->peer_id.hashPubKey));
+    output_rfn = rfn_create (session->num_peers);
+    output_rfn->key = rfn_key;
+    put_rfn (session, output_rfn);
   }
-  return GNUNET_OK;
-}
 
+  diff_key = (struct DiffKey) { DIFF_KIND_LEADER_PROPOSAL, task->key.repetition, task->key.leader };
+  input_diff = lookup_diff (session, &diff_key);
+  GNUNET_assert (NULL != input_diff);
 
-/**
- * Functions of this type are called upon new stream connection from other peers
- * or upon binding error which happen when the app_port given in
- * GNUNET_STREAM_listen() is already taken.
- *
- * @param cls the closure from GNUNET_STREAM_listen
- * @param socket the socket representing the stream; NULL on binding error
- * @param initiator the identity of the peer who wants to establish a stream
- *            with us; NULL on binding error
- * @return GNUNET_OK to keep the socket open, GNUNET_SYSERR to close the
- *             stream (the socket will be invalid after the call)
- */
-static int
-listen_cb (void *cls,
-           struct GNUNET_STREAM_Socket *socket,
-           const struct GNUNET_PeerIdentity *initiator)
-{
-  struct IncomingSocket *incoming;
-  GNUNET_assert (NULL != socket);
-  incoming = GNUNET_malloc (sizeof *incoming);
-  incoming->socket = socket;
-  incoming->peer_id = *initiator;
-  incoming->rh = GNUNET_STREAM_read (socket, GNUNET_TIME_UNIT_FOREVER_REL,
-                                     &incoming_stream_data_processor, incoming);
-  incoming->mst = GNUNET_SERVER_mst_create (mst_incoming_callback, incoming);
-  GNUNET_CONTAINER_DLL_insert_tail (incoming_sockets_head, incoming_sockets_tail, incoming);
-  return GNUNET_OK;
-}
+  rfn_key = (struct RfnKey) { RFN_KIND_ECHO, task->key.repetition, task->key.leader };
+  input_rfn = lookup_rfn (session, &rfn_key);
+  GNUNET_assert (NULL != input_rfn);
 
+  iter = GNUNET_CONTAINER_multihashmap_iterator_create (input_rfn->rfn_elements);
 
-/**
- * Iterator over hash map entries.
- *
- * @param cls closure
- * @param key current key code
- * @param value value in the hash map
- * @return GNUNET_YES if we should continue to
- *         iterate,
- *         GNUNET_NO if not.
- */
-static int
-destroy_element_list_iter (void *cls,
-                           const struct GNUNET_HashCode * key,
-                           void *value)
-{
-  struct ElementList *el;
-  el = value;
-  while (NULL != el)
+  apply_diff_to_rfn (input_diff, output_rfn, task->key.leader, session->num_peers);
+
+  while (GNUNET_YES ==
+         GNUNET_CONTAINER_multihashmap_iterator_next (iter,
+                                                      NULL,
+                                                      (const void **) &ri))
   {
-    struct ElementList *el_old;
-    el_old = el;
-    el = el->next;
-    GNUNET_free (el_old->element_hash);
-    GNUNET_free (el_old->element);
-    GNUNET_free (el_old);
+    uint16_t majority_num;
+    enum ReferendumVote majority_vote;
+
+    // XXX: we need contested votes and non-contested votes here
+    rfn_majority (input_rfn, ri, &majority_num, &majority_vote);
+
+    if (majority_num <= session->num_peers / 3)
+      majority_vote = VOTE_REMOVE;
+
+    switch (majority_vote)
+    {
+      case VOTE_STAY:
+        break;
+      case VOTE_ADD:
+        rfn_vote (output_rfn, task->key.leader, VOTE_ADD, ri->element);
+        break;
+      case VOTE_REMOVE:
+        rfn_vote (output_rfn, task->key.leader, VOTE_REMOVE, ri->element);
+        break;
+      default:
+        GNUNET_assert (0);
+        break;
+    }
   }
-  return GNUNET_YES;
+  GNUNET_CONTAINER_multihashmap_iterator_destroy (iter);
+
+  {
+    uint16_t noncontested;
+    noncontested = rfn_noncontested (input_rfn);
+    if (noncontested < (session->num_peers / 3) * 2)
+    {
+      gradecast_confidence = GNUNET_MIN(1, gradecast_confidence);
+    }
+    if (noncontested < (session->num_peers / 3) + 1)
+    {
+      gradecast_confidence = 0;
+    }
+  }
+
+  if (gradecast_confidence >= 1)
+    rfn_commit (output_rfn, task->key.leader);
+
+  if (gradecast_confidence <= 1)
+    session->peers_blacklisted[task->key.leader] = GNUNET_YES;
+
+  finish_task (task);
 }
 
 
-/**
- * Destroy a session, free all resources associated with it.
- * 
- * @param session the session to destroy
- */
 static void
-destroy_session (struct ConsensusSession *session)
+task_start_reconcile (struct TaskEntry *task)
 {
-  int i;
+  struct SetEntry *input;
+  struct SetOpCls *setop = &task->cls.setop;
+  struct ConsensusSession *session = task->step->session;
 
-  GNUNET_CONTAINER_DLL_remove (sessions_head, sessions_tail, session);
-  GNUNET_SERVER_client_drop (session->client);
-  session->client = NULL;
-  if (NULL != session->shuffle)
+  input = lookup_set (session, &setop->input_set);
+  GNUNET_assert (NULL != input);
+  GNUNET_assert (NULL != input->h);
+
+  /* We create the outputs for the operation here
+     (rather than in the set operation callback)
+     because we want something valid in there, even
+     if the other peer doesn't talk to us */
+
+  if (SET_KIND_NONE != setop->output_set.set_kind)
   {
-    GNUNET_free (session->shuffle);
-    session->shuffle = NULL;
+    /* If we don't have an existing output set,
+       we clone the input set. */
+    if (NULL == lookup_set (session, &setop->output_set))
+    {
+      create_set_copy_for_task (task, &setop->input_set, &setop->output_set);
+      return;
+    }
   }
-  if (NULL != session->se)
+
+  if (RFN_KIND_NONE != setop->output_rfn.rfn_kind)
   {
-    strata_estimator_destroy (session->se);
-    session->se = NULL;
+    if (NULL == lookup_rfn (session, &setop->output_rfn))
+    {
+      struct ReferendumEntry *rfn;
+
+      GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                  "P%u: output rfn <%s> missing, creating.\n",
+                  session->local_peer_idx,
+                  debug_str_rfn_key (&setop->output_rfn));
+
+      rfn = rfn_create (session->num_peers);
+      rfn->key = setop->output_rfn;
+      put_rfn (session, rfn);
+    }
   }
-  if (NULL != session->info)
+
+  if (DIFF_KIND_NONE != setop->output_diff.diff_kind)
   {
-    for (i = 0; i < session->num_peers; i++)
+    if (NULL == lookup_diff (session, &setop->output_diff))
     {
-      struct ConsensusPeerInformation *cpi;
-      cpi = &session->info[i];
-      if ((NULL != cpi) && (NULL != cpi->socket))
-      {
-        if (NULL != cpi->rh)
-        {
-          GNUNET_STREAM_read_cancel (cpi->rh);
-          cpi->rh = NULL;
-        } 
-        if (NULL != cpi->wh)
-        {
-          GNUNET_STREAM_write_cancel (cpi->wh);
-          cpi->wh = NULL;
-        } 
-        GNUNET_STREAM_close (cpi->socket);
-        cpi->socket = NULL;
-      }
-      if (NULL != cpi->se)
-      {
-        strata_estimator_destroy (cpi->se);
-        cpi->se = NULL;
-      }
-      if (NULL != cpi->ibf)
-      {
-        ibf_destroy (cpi->ibf);
-        cpi->ibf = NULL;
-      }
-      if (NULL != cpi->mst)
-      {
-        GNUNET_SERVER_mst_destroy (cpi->mst);
-        cpi->mst = NULL;
-      }
+      struct DiffEntry *diff;
+
+      diff = diff_create ();
+      diff->key = setop->output_diff;
+      put_diff (session, diff);
     }
-    GNUNET_free (session->info);
-    session->info = NULL;
   }
-  if (NULL != session->ibfs)
+
+  if ( (task->key.peer1 == session->local_peer_idx) && (task->key.peer2 == session->local_peer_idx) )
+  {
+    /* XXX: mark the corresponding rfn as commited if necessary */
+    finish_task (task);
+    return;
+  }
+
+  if (task->key.peer1 == session->local_peer_idx)
+  {
+    struct GNUNET_CONSENSUS_RoundContextMessage rcm = { 0 };
+
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "P%u: Looking up set {%s} to run remote union\n",
+                session->local_peer_idx,
+                debug_str_set_key (&setop->input_set));
+
+    rcm.header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ROUND_CONTEXT);
+    rcm.header.size = htons (sizeof (struct GNUNET_CONSENSUS_RoundContextMessage));
+
+    rcm.kind = htons (task->key.kind);
+    rcm.peer1 = htons (task->key.peer1);
+    rcm.peer2 = htons (task->key.peer2);
+    rcm.leader = htons (task->key.leader);
+    rcm.repetition = htons (task->key.repetition);
+
+    GNUNET_assert (NULL == setop->op);
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "P%u: initiating set op with P%u, our set is %s\n",
+                session->local_peer_idx, task->key.peer2, debug_str_set_key (&setop->input_set));
+
+    struct GNUNET_SET_Option opts[] = {
+      { GNUNET_SET_OPTION_BYZANTINE, { .num = session->lower_bound } },
+      { GNUNET_SET_OPTION_END },
+    };
+
+    // XXX: maybe this should be done while
+    // setting up tasks alreays?
+    setop->op = GNUNET_SET_prepare (&session->peers[task->key.peer2],
+                                    &session->global_id,
+                                    &rcm.header,
+                                    GNUNET_SET_RESULT_SYMMETRIC,
+                                    opts,
+                                    set_result_cb,
+                                    task);
+
+    commit_set (session, task);
+  }
+  else if (task->key.peer2 == session->local_peer_idx)
   {
-    for (i = 0; i <= MAX_IBF_ORDER; i++)
+    /* Wait for the other peer to contact us */
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "P%u: waiting set op with P%u\n",
+                session->local_peer_idx, task->key.peer1);
+
+    if (NULL != setop->op)
     {
-      if (NULL != session->ibfs[i])
-      {
-        ibf_destroy (session->ibfs[i]);
-        session->ibfs[i] = NULL;
-      }
+      commit_set (session, task);
     }
-    GNUNET_free (session->ibfs);
-    session->ibfs = NULL;
   }
-  if (NULL != session->values)
+  else
   {
-    GNUNET_CONTAINER_multihashmap_iterate (session->values, destroy_element_list_iter, NULL);
-    GNUNET_CONTAINER_multihashmap_destroy (session->values);
-    session->values = NULL;
+    /* We made an error while constructing the task graph. */
+    GNUNET_assert (0);
   }
-  GNUNET_free (session);
 }
 
 
-/**
- * Disconnect a client, and destroy all sessions associated with it.
- *
- * @param client the client to disconnect
- */
 static void
-disconnect_client (struct GNUNET_SERVER_Client *client)
-{
-  struct ConsensusSession *session;
-  GNUNET_SERVER_client_disconnect (client);
-  
-  /* if the client owns a session, remove it */
-  session = sessions_head;
-  while (NULL != session)
+task_start_eval_echo (struct TaskEntry *task)
+{
+  struct GNUNET_CONTAINER_MultiHashMapIterator *iter;
+  struct ReferendumEntry *input_rfn;
+  struct RfnElementInfo *ri;
+  struct SetEntry *output_set;
+  struct SetMutationProgressCls *progress_cls;
+  struct ConsensusSession *session = task->step->session;
+  struct SetKey sk_in;
+  struct SetKey sk_out;
+  struct RfnKey rk_in;
+
+  sk_in = (struct SetKey) { SET_KIND_LEADER_PROPOSAL, task->key.repetition, task->key.leader };
+  sk_out = (struct SetKey) { SET_KIND_ECHO_RESULT, task->key.repetition, task->key.leader };
+  output_set = lookup_set (session, &sk_out);
+  if (NULL == output_set)
+  {
+    create_set_copy_for_task (task, &sk_in, &sk_out);
+    return;
+  }
+
+
+  {
+    // FIXME: should be marked as a shallow copy, so
+    // we can destroy everything correctly
+    struct SetEntry *last_set = GNUNET_new (struct SetEntry);
+    last_set->h = output_set->h;
+    last_set->key = (struct SetKey) { SET_KIND_LAST_GRADECAST };
+    put_set (session, last_set);
+  }
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Evaluating referendum in Task {%s}\n",
+              debug_str_task_key (&task->key));
+
+  progress_cls = GNUNET_new (struct SetMutationProgressCls);
+  progress_cls->task = task;
+
+  rk_in = (struct RfnKey) { RFN_KIND_ECHO, task->key.repetition, task->key.leader };
+  input_rfn = lookup_rfn (session, &rk_in);
+
+  GNUNET_assert (NULL != input_rfn);
+
+  iter = GNUNET_CONTAINER_multihashmap_iterator_create (input_rfn->rfn_elements);
+  GNUNET_assert (NULL != iter);
+
+  while (GNUNET_YES ==
+         GNUNET_CONTAINER_multihashmap_iterator_next (iter,
+                                                      NULL,
+                                                      (const void **) &ri))
   {
-    if (client == session->client)
+    enum ReferendumVote majority_vote;
+    uint16_t majority_num;
+
+    rfn_majority (input_rfn, ri, &majority_num, &majority_vote);
+
+    if (majority_num < session->num_peers / 3)
     {
-      destroy_session (session);
-      break;
+      /* It is not the case that all nonfaulty peers
+         echoed the same value.  Since we're doing a set reconciliation, we
+         can't simply send "nothing" for the value.  Thus we mark our 'confirm'
+         reconciliation as contested.  Other peers might not know that the
+         leader is faulty, thus we still re-distribute in the confirmation
+         round. */
+      output_set->is_contested = GNUNET_YES;
     }
-    session = session->next;
-  }
-}
 
+    switch (majority_vote)
+    {
+      case VOTE_ADD:
+        progress_cls->num_pending++;
+        GNUNET_assert (GNUNET_OK ==
+                       GNUNET_SET_add_element (output_set->h,
+                                               ri->element,
+                                               set_mutation_done,
+                                               progress_cls));
+        break;
+      case VOTE_REMOVE:
+        progress_cls->num_pending++;
+        GNUNET_assert (GNUNET_OK ==
+                       GNUNET_SET_remove_element (output_set->h,
+                                                  ri->element,
+                                                  set_mutation_done,
+                                                  progress_cls));
+        break;
+      case VOTE_STAY:
+        /* Nothing to do. */
+        break;
+      default:
+        /* not reached */
+        GNUNET_assert (0);
+    }
+  }
 
-/**
- * Compute a global, (hopefully) unique consensus session id,
- * from the local id of the consensus session, and the identities of all participants.
- * Thus, if the local id of two consensus sessions coincide, but are not comprised of
- * exactly the same peers, the global id will be different.
- *
- * @param session session to generate the global id for
- * @param session_id local id of the consensus session
- */
-static void
-compute_global_id (struct ConsensusSession *session, const struct GNUNET_HashCode *session_id)
-{
-  int i;
-  struct GNUNET_HashCode tmp;
+  GNUNET_CONTAINER_multihashmap_iterator_destroy (iter);
 
-  session->global_id = *session_id;
-  for (i = 0; i < session->num_peers; ++i)
+  if (0 == progress_cls->num_pending)
   {
-    GNUNET_CRYPTO_hash_xor (&session->global_id, &session->info[i].peer_id.hashPubKey, &tmp);
-    session->global_id = tmp;
-    GNUNET_CRYPTO_hash (&session->global_id, sizeof (struct GNUNET_PeerIdentity), &tmp);
-    session->global_id = tmp;
+    // call closure right now, no pending ops
+    GNUNET_free (progress_cls);
+    finish_task (task);
   }
 }
 
 
-/**
- * Transmit a queued message to the session's client.
- *
- * @param cls consensus session
- * @param size number of bytes available in buf
- * @param buf where the callee should write the message
- * @return number of bytes written to buf
- */
-static size_t
-transmit_queued (void *cls, size_t size,
-                 void *buf)
+static void
+task_start_finish (struct TaskEntry *task)
 {
-  struct ConsensusSession *session;
-  struct QueuedMessage *qmsg;
-  size_t msg_size;
+  struct SetEntry *final_set;
+  struct ConsensusSession *session = task->step->session;
 
-  session = cls;
-  session->client_th = NULL;
+  final_set = lookup_set (session, &task->cls.finish.input_set);
 
-  qmsg = session->client_messages_head;
-  GNUNET_CONTAINER_DLL_remove (session->client_messages_head, session->client_messages_tail, qmsg);
-  GNUNET_assert (qmsg);
+  GNUNET_assert (NULL != final_set);
 
-  if (NULL == buf)
-  {
-    destroy_session (session);
-    return 0;
-  }
 
-  msg_size = ntohs (qmsg->msg->size);
+  GNUNET_SET_iterate (final_set->h,
+                      send_to_client_iter,
+                      task);
+}
 
-  GNUNET_assert (size >= msg_size);
+static void
+start_task (struct ConsensusSession *session, struct TaskEntry *task)
+{
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "P%u: starting task {%s}\n", session->local_peer_idx, debug_str_task_key (&task->key));
 
-  memcpy (buf, qmsg->msg, msg_size);
-  GNUNET_free (qmsg->msg);
-  GNUNET_free (qmsg);
+  GNUNET_assert (GNUNET_NO == task->is_started);
+  GNUNET_assert (GNUNET_NO == task->is_finished);
+  GNUNET_assert (NULL != task->start);
 
-  client_send_next (session);
+  task->start (task);
 
-  return msg_size;
+  task->is_started = GNUNET_YES;
 }
 
 
-/**
- * Schedule transmitting the next queued message (if any) to the inhabiting client of a session.
- *
- * @param session the consensus session
+
+
+/*
+ * Run all steps of the session that don't any
+ * more dependencies.
  */
 static void
-client_send_next (struct ConsensusSession *session)
+run_ready_steps (struct ConsensusSession *session)
 {
+  struct Step *step;
 
-  GNUNET_assert (NULL != session);
-
-  if (NULL != session->client_th)
-    return;
+  step = session->steps_head;
 
-  if (NULL != session->client_messages_head)
+  while (NULL != step)
   {
-    int msize;
-    msize = ntohs (session->client_messages_head->msg->size);
-    session->client_th = GNUNET_SERVER_notify_transmit_ready (session->client, msize, 
-                                                              GNUNET_TIME_UNIT_FOREVER_REL,
-                                                              &transmit_queued, session);
+    if ( (GNUNET_NO == step->is_running) && (0 == step->pending_prereq) && (GNUNET_NO == step->is_finished) )
+    {
+      size_t i;
+
+      GNUNET_assert (0 == step->finished_tasks);
+
+#ifdef GNUNET_EXTRA_LOGGING
+      GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "P%u: Running step `%s' of round %d with %d tasks and %d subordinates\n",
+                  session->local_peer_idx,
+                  step->debug_name,
+                  step->round, step->tasks_len, step->subordinates_len);
+#endif
+
+      step->is_running = GNUNET_YES;
+      for (i = 0; i < step->tasks_len; i++)
+        start_task (session, step->tasks[i]);
+
+      /* Sometimes there is no task to trigger finishing the step, so we have to do it here. */
+      if ( (step->finished_tasks == step->tasks_len) && (GNUNET_NO == step->is_finished))
+        finish_step (step);
+
+      /* Running the next ready steps will be triggered by task completion */
+      return;
+    }
+    step = step->next;
   }
+
+  return;
 }
 
 
-/**
- * Although GNUNET_CRYPTO_hash_cmp exisits, it does not have
- * the correct signature to be used with e.g. qsort.
- * We use this function instead.
- *
- * @param h1 some hash code
- * @param h2 some hash code
- * @return 1 if h1 > h2, -1 if h1 < h2 and 0 if h1 == h2.
- */
-static int
-hash_cmp (const void *h1, const void *h2)
+
+static void
+finish_task (struct TaskEntry *task)
 {
-  return GNUNET_CRYPTO_hash_cmp ((struct GNUNET_HashCode *) h1, (struct GNUNET_HashCode *) h2);
+  GNUNET_assert (GNUNET_NO == task->is_finished);
+  task->is_finished = GNUNET_YES;
+
+  task->step->finished_tasks++;
+
+  if (task->step->finished_tasks == task->step->tasks_len)
+    finish_step (task->step);
 }
 
 
@@ -1813,848 +2370,912 @@ get_peer_idx (const struct GNUNET_PeerIdentity *peer, const struct ConsensusSess
 {
   int i;
   for (i = 0; i < session->num_peers; i++)
-    if (0 == memcmp (peer, &session->info[i].peer_id, sizeof *peer))
+    if (0 == memcmp (peer, &session->peers[i], sizeof (struct GNUNET_PeerIdentity)))
       return i;
   return -1;
 }
 
 
 /**
- * Called when stream has finishes writing the hello message
+ * Compute a global, (hopefully) unique consensus session id,
+ * from the local id of the consensus session, and the identities of all participants.
+ * Thus, if the local id of two consensus sessions coincide, but are not comprised of
+ * exactly the same peers, the global id will be different.
+ *
+ * @param session session to generate the global id for
+ * @param local_session_id local id of the consensus session
  */
 static void
-hello_cont (void *cls, enum GNUNET_STREAM_Status status, size_t size)
+compute_global_id (struct ConsensusSession *session,
+                  const struct GNUNET_HashCode *local_session_id)
 {
-  struct ConsensusPeerInformation *cpi;
+  const char *salt = "gnunet-service-consensus/session_id";
 
-  cpi = cls;
-  cpi->wh = NULL;
-  cpi->hello = GNUNET_YES;
-  GNUNET_assert (GNUNET_STREAM_OK == status);
-  embrace_peer (cpi);
+  GNUNET_assert (GNUNET_YES ==
+                 GNUNET_CRYPTO_kdf (&session->global_id,
+                                    sizeof (struct GNUNET_HashCode),
+                                    salt,
+                                    strlen (salt),
+                                    session->peers,
+                                    session->num_peers * sizeof (struct GNUNET_PeerIdentity),
+                                    local_session_id,
+                                    sizeof (struct GNUNET_HashCode),
+                                    NULL));
 }
 
 
 /**
- * Called when we established a stream connection to another peer
+ * Compare two peer identities.
  *
- * @param cls cpi of the peer we just connected to
- * @param socket socket to use to communicate with the other side (read/write)
+ * @param h1 some peer identity
+ * @param h2 some peer identity
+ * @return 1 if h1 > h2, -1 if h1 < h2 and 0 if h1 == h2.
  */
-static void
-open_cb (void *cls, struct GNUNET_STREAM_Socket *socket)
+static int
+peer_id_cmp (const void *h1, const void *h2)
 {
-  struct ConsensusPeerInformation *cpi;
-  struct ConsensusHello *hello;
-
-  cpi = cls;
-  hello = GNUNET_malloc (sizeof *hello);
-  hello->header.size = htons (sizeof *hello);
-  hello->header.type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_HELLO);
-  memcpy (&hello->global_id, &cpi->session->global_id, sizeof (struct GNUNET_HashCode));
-  GNUNET_assert (NULL == cpi->mst);
-  cpi->mst = GNUNET_SERVER_mst_create (mst_session_callback, cpi);
-  cpi->wh =
-      GNUNET_STREAM_write (socket, hello, sizeof *hello, GNUNET_TIME_UNIT_FOREVER_REL, hello_cont, cpi);
-  GNUNET_free (hello);
-  cpi->rh = GNUNET_STREAM_read (socket, GNUNET_TIME_UNIT_FOREVER_REL,
-                                &session_stream_data_processor, cpi);
+  return memcmp (h1, h2, sizeof (struct GNUNET_PeerIdentity));
 }
 
 
 /**
  * Create the sorted list of peers for the session,
  * add the local peer if not in the join message.
+ *
+ * @param session session to initialize
+ * @param join_msg join message with the list of peers participating at the end
  */
 static void
-initialize_session_peer_list (struct ConsensusSession *session)
+initialize_session_peer_list (struct ConsensusSession *session,
+                              const struct GNUNET_CONSENSUS_JoinMessage *join_msg)
 {
-  unsigned int local_peer_in_list;
-  uint32_t listed_peers;
-  const struct GNUNET_PeerIdentity *msg_peers;
-  struct GNUNET_PeerIdentity *peers;
-  unsigned int i;
+  const struct GNUNET_PeerIdentity *msg_peers
+    = (const struct GNUNET_PeerIdentity *) &join_msg[1];
+  int local_peer_in_list;
 
-  GNUNET_assert (NULL != session->join_msg);
-
-  /* peers in the join message, may or may not include the local peer */
-  listed_peers = ntohl (session->join_msg->num_peers);
-  
-  session->num_peers = listed_peers;
-
-  msg_peers = (struct GNUNET_PeerIdentity *) &session->join_msg[1];
+  session->num_peers = ntohl (join_msg->num_peers);
 
+  /* Peers in the join message, may or may not include the local peer,
+     Add it if it is missing. */
   local_peer_in_list = GNUNET_NO;
-  for (i = 0; i < listed_peers; i++)
+  for (unsigned int i = 0; i < session->num_peers; i++)
   {
-    if (0 == memcmp (&msg_peers[i], my_peer, sizeof (struct GNUNET_PeerIdentity)))
+    if (0 == memcmp (&msg_peers[i],
+                     &my_peer,
+                     sizeof (struct GNUNET_PeerIdentity)))
     {
       local_peer_in_list = GNUNET_YES;
       break;
     }
   }
-
   if (GNUNET_NO == local_peer_in_list)
     session->num_peers++;
 
-  peers = GNUNET_malloc (session->num_peers * sizeof (struct GNUNET_PeerIdentity));
-
+  session->peers = GNUNET_new_array (session->num_peers,
+                                     struct GNUNET_PeerIdentity);
   if (GNUNET_NO == local_peer_in_list)
-    peers[session->num_peers - 1] = *my_peer;
-
-  memcpy (peers, msg_peers, listed_peers * sizeof (struct GNUNET_PeerIdentity));
-  qsort (peers, session->num_peers, sizeof (struct GNUNET_PeerIdentity), &hash_cmp);
-
-  session->info = GNUNET_malloc (session->num_peers * sizeof (struct ConsensusPeerInformation));
-
-  for (i = 0; i < session->num_peers; ++i)
-  {
-    /* initialize back-references, so consensus peer information can
-     * be used as closure */
-    session->info[i].session = session;
-    session->info[i].peer_id = peers[i];
-  }
+    session->peers[session->num_peers - 1] = my_peer;
 
-  free (peers);
+  GNUNET_memcpy (session->peers,
+                 msg_peers,
+                 ntohl (join_msg->num_peers) * sizeof (struct GNUNET_PeerIdentity));
+  qsort (session->peers,
+         session->num_peers,
+         sizeof (struct GNUNET_PeerIdentity),
+         &peer_id_cmp);
 }
 
 
-static void
-strata_estimator_insert (struct StrataEstimator *se, struct GNUNET_HashCode *key)
+static struct TaskEntry *
+lookup_task (struct ConsensusSession *session,
+             struct TaskKey *key)
 {
-  uint32_t v;
-  int i;
-  v = key->bits[0];
-  /* count trailing '1'-bits of v */
-  for (i = 0; v & 1; v>>=1, i++)
-    /* empty */;
-  ibf_insert (se->strata[i], ibf_key_from_hashcode (key));
+  struct GNUNET_HashCode hash;
+
+
+  GNUNET_CRYPTO_hash (key, sizeof (struct TaskKey), &hash);
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "Looking up task hash %s\n",
+              GNUNET_h2s (&hash));
+  return GNUNET_CONTAINER_multihashmap_get (session->taskmap, &hash);
 }
 
 
 /**
- * Add incoming peer connections to the session,
- * for peers who have connected to us before the local session has been established
+ * Called when another peer wants to do a set operation with the
+ * local peer.
  *
- * @param session ...
+ * @param cls closure
+ * @param other_peer the other peer
+ * @param context_msg message with application specific information from
+ *        the other peer
+ * @param request request from the other peer, use GNUNET_SET_accept
+ *        to accept it, otherwise the request will be refused
+ *        Note that we don't use a return value here, as it is also
+ *        necessary to specify the set we want to do the operation with,
+ *        whith sometimes can be derived from the context message.
+ *        Also necessary to specify the timeout.
  */
 static void
-add_incoming_peers (struct ConsensusSession *session)
+set_listen_cb (void *cls,
+               const struct GNUNET_PeerIdentity *other_peer,
+               const struct GNUNET_MessageHeader *context_msg,
+               struct GNUNET_SET_Request *request)
 {
-  struct IncomingSocket *inc;
-  inc = incoming_sockets_head;
+  struct ConsensusSession *session = cls;
+  struct TaskKey tk;
+  struct TaskEntry *task;
+  struct GNUNET_CONSENSUS_RoundContextMessage *cm;
 
-  while (NULL != inc)
+  if (NULL == context_msg)
   {
-    if (0 == GNUNET_CRYPTO_hash_cmp (&session->global_id, inc->requested_gid))
-    {
-      int i;
-      for (i = 0; i < session->num_peers; i++)
-      {
-        struct ConsensusPeerInformation *cpi;
-        cpi = &session->info[i];
-        if (0 == memcmp (&inc->peer_id, &cpi->peer_id, sizeof (struct GNUNET_PeerIdentity)))
-        {
-          cpi->socket = inc->socket;
-          inc->cpi = cpi;
-          inc->cpi->mst = inc->mst;
-          inc->cpi->hello = GNUNET_YES;
-          break;
-        }
-      }
-    }
-    inc = inc->next;
+    GNUNET_break_op (0);
+    return;
   }
-}
 
+  if (GNUNET_MESSAGE_TYPE_CONSENSUS_P2P_ROUND_CONTEXT != ntohs (context_msg->type))
+  {
+    GNUNET_break_op (0);
+    return;
+  }
 
-/**
- * Initialize the session, continue receiving messages from the owning client
- *
- * @param session the session to initialize
- */
-static void
-initialize_session (struct ConsensusSession *session)
-{
-  const struct ConsensusSession *other_session;
+  if (sizeof (struct GNUNET_CONSENSUS_RoundContextMessage) != ntohs (context_msg->size))
+  {
+    GNUNET_break_op (0);
+    return;
+  }
+
+  cm = (struct GNUNET_CONSENSUS_RoundContextMessage *) context_msg;
 
-  GNUNET_assert (NULL != session->join_msg);
-  initialize_session_peer_list (session);
-  session->current_round = CONSENSUS_ROUND_BEGIN;
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "session with %u peers\n", session->num_peers);
-  compute_global_id (session, &session->join_msg->session_id);
+  tk = ((struct TaskKey) {
+      .kind = ntohs (cm->kind),
+      .peer1 = ntohs (cm->peer1),
+      .peer2 = ntohs (cm->peer2),
+      .repetition = ntohs (cm->repetition),
+      .leader = ntohs (cm->leader),
+  });
 
-  /* Check if some local client already owns the session. */
-  other_session = sessions_head;
-  while (NULL != other_session)
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "P%u: got req for task %s\n",
+              session->local_peer_idx, debug_str_task_key (&tk));
+
+  task = lookup_task (session, &tk);
+
+  if (NULL == task)
   {
-    if ((other_session != session) && 
-        (0 == GNUNET_CRYPTO_hash_cmp (&session->global_id, &other_session->global_id)))
-    {
-      /* session already owned by another client */
-      GNUNET_break (0);
-      disconnect_client (session->client);
-      return;
-    }
-    other_session = other_session->next;
+    GNUNET_break_op (0);
+    return;
   }
 
-  session->values = GNUNET_CONTAINER_multihashmap_create (256, GNUNET_NO);
-  session->local_peer_idx = get_peer_idx (my_peer, session);
-  GNUNET_assert (-1 != session->local_peer_idx);
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "%d is the local peer\n", session->local_peer_idx);
-  session->se = strata_estimator_create ();
-  session->ibfs = GNUNET_malloc ((MAX_IBF_ORDER+1) * sizeof (struct InvertibleBloomFilter *));
-  GNUNET_free (session->join_msg);
-  session->join_msg = NULL;
-  add_incoming_peers (session);
-  GNUNET_SERVER_receive_done (session->client, GNUNET_OK);
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "session %s initialized\n", GNUNET_h2s (&session->global_id));
+  if (GNUNET_YES == task->is_finished)
+  {
+    GNUNET_break_op (0);
+    return;
+  }
+
+  if (task->key.peer2 != session->local_peer_idx)
+  {
+    /* We're being asked, so we must be thne 2nd peer. */
+    GNUNET_break_op (0);
+    return;
+  }
+
+  GNUNET_assert (! ((task->key.peer1 == session->local_peer_idx) &&
+                    (task->key.peer2 == session->local_peer_idx)));
+
+  struct GNUNET_SET_Option opts[] = {
+    { GNUNET_SET_OPTION_BYZANTINE, { .num = session->lower_bound } },
+    { GNUNET_SET_OPTION_END },
+  };
+
+  task->cls.setop.op = GNUNET_SET_accept (request,
+                                          GNUNET_SET_RESULT_SYMMETRIC,
+                                          opts,
+                                          set_result_cb,
+                                          task);
+
+  /* If the task hasn't been started yet,
+     we wait for that until we commit. */
+
+  if (GNUNET_YES == task->is_started)
+  {
+    commit_set (session, task);
+  }
 }
 
 
-/**
- * Called when a client wants to join a consensus session.
- *
- * @param cls unused
- * @param client client that sent the message
- * @param m message sent by the client
- */
+
 static void
-client_join (void *cls,
-             struct GNUNET_SERVER_Client *client,
-             const struct GNUNET_MessageHeader *m)
+put_task (struct GNUNET_CONTAINER_MultiHashMap *taskmap,
+          struct TaskEntry *t)
 {
-  struct ConsensusSession *session;
+  struct GNUNET_HashCode round_hash;
+  struct Step *s;
 
-  // make sure the client has not already joined a session
-  session = sessions_head;
-  while (NULL != session)
-  {
-    if (session->client == client)
-    {
-      GNUNET_break (0);
-      disconnect_client (client);
-      return;
-    }
-    session = session->next;
-  }
+  GNUNET_assert (NULL != t->step);
 
-  session = GNUNET_malloc (sizeof (struct ConsensusSession));
-  session->join_msg = (struct GNUNET_CONSENSUS_JoinMessage *) GNUNET_copy_message (m);
-  session->client = client;
-  GNUNET_SERVER_client_keep (client);
+  t = GNUNET_memdup (t, sizeof (struct TaskEntry));
 
-  GNUNET_CONTAINER_DLL_insert (sessions_head, sessions_tail, session);
+  s = t->step;
 
-  // Initialize session later if local peer identity is not known yet.
-  if (NULL == my_peer)
+  if (s->tasks_len == s->tasks_cap)
   {
-    GNUNET_SERVER_disable_receive_done_warning (client);
-    return;
+    unsigned int target_size = 3 * (s->tasks_cap + 1) / 2;
+    GNUNET_array_grow (s->tasks,
+                       s->tasks_cap,
+                       target_size);
   }
 
-  initialize_session (session);
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_assert (NULL != s->debug_name);
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG, "Putting task <%s> into step `%s'\n",
+              debug_str_task_key (&t->key),
+              s->debug_name);
+#endif
+
+  s->tasks[s->tasks_len] = t;
+  s->tasks_len++;
+
+  GNUNET_CRYPTO_hash (&t->key, sizeof (struct TaskKey), &round_hash);
+  GNUNET_assert (GNUNET_OK ==
+      GNUNET_CONTAINER_multihashmap_put (taskmap, &round_hash, t,
+                                         GNUNET_CONTAINER_MULTIHASHMAPOPTION_UNIQUE_ONLY));
 }
 
 
-/**
- * Hash a block of data, producing a replicated ibf hash.
- */
 static void
-hash_for_ibf (const void *block, size_t size, struct GNUNET_HashCode *ret)
+install_step_timeouts (struct ConsensusSession *session)
 {
-  struct IBF_Key ibf_key;
-  GNUNET_CRYPTO_hash (block, size, ret);
-  ibf_key = ibf_key_from_hashcode (ret);
-  ibf_hashcode_from_key (ibf_key, ret);
+  /* Given the fully constructed task graph
+     with rounds for tasks, we can give the tasks timeouts. */
+
+  // unsigned int max_round;
+
+  /* XXX: implement! */
 }
 
 
+
+/*
+ * Arrange two peers in some canonical order.
+ */
 static void
-insert_element (struct ConsensusSession *session, struct GNUNET_CONSENSUS_Element *element)
+arrange_peers (uint16_t *p1, uint16_t *p2, uint16_t n)
 {
-  struct GNUNET_HashCode hash;
-  struct ElementList *head;
+  uint16_t a;
+  uint16_t b;
 
-  hash_for_ibf (element->data, element->size, &hash);
+  GNUNET_assert (*p1 < n);
+  GNUNET_assert (*p2 < n);
 
-  head = GNUNET_CONTAINER_multihashmap_get (session->values, &hash);
-
-  if (NULL == head)
+  if (*p1 < *p2)
   {
-    int i;
-
-    head = GNUNET_malloc (sizeof *head);
-    head->element = element;
-    head->next = NULL;
-    head->element_hash = GNUNET_memdup (&hash, sizeof hash);
-    GNUNET_CONTAINER_multihashmap_put (session->values, &hash, head,
-                                       GNUNET_CONTAINER_MULTIHASHMAPOPTION_UNIQUE_ONLY);
-    strata_estimator_insert (session->se, &hash);
+    a = *p1;
+    b = *p2;
+  }
+  else
+  {
+    a = *p2;
+    b = *p1;
+  }
 
-    for (i = 0; i <= MAX_IBF_ORDER; i++)
-      if (NULL != session->ibfs[i])
-        ibf_insert (session->ibfs[i], ibf_key_from_hashcode (&hash));
+  /* For uniformly random *p1, *p2,
+     this condition is true with 50% chance */
+  if (((b - a) + n) % n <= n / 2)
+  {
+    *p1 = a;
+    *p2 = b;
   }
   else
   {
-    struct ElementList *el;
-    el = GNUNET_malloc (sizeof *el);
-    head->element = element;
-    head->next = NULL;
-    head->element_hash = GNUNET_memdup (&hash, sizeof hash);
-    while (NULL != head->next)
-      head = head->next;
-    head->next = el;
+    *p1 = b;
+    *p2 = a;
   }
 }
 
 
 /**
- * Called when a client performs an insert operation.
- *
- * @param cls (unused)
- * @param client client handle
- * @param m message sent by the client
+ * Record @a dep as a dependency of @a step.
  */
-void
-client_insert (void *cls,
-             struct GNUNET_SERVER_Client *client,
-             const struct GNUNET_MessageHeader *m)
+static void
+step_depend_on (struct Step *step, struct Step *dep)
 {
-  struct ConsensusSession *session;
-  struct GNUNET_CONSENSUS_ElementMessage *msg;
-  struct GNUNET_CONSENSUS_Element *element;
-  int element_size;
+  /* We're not checking for cyclic dependencies,
+     but this is a cheap sanity check. */
+  GNUNET_assert (step != dep);
+  GNUNET_assert (NULL != step);
+  GNUNET_assert (NULL != dep);
+  GNUNET_assert (dep->round <= step->round);
 
-  session = sessions_head;
-  while (NULL != session)
-  {
-    if (session->client == client)
-      break;
-  }
+#ifdef GNUNET_EXTRA_LOGGING
+  /* Make sure we have complete debugging information.
+     Also checks that we don't screw up too badly
+     constructing the task graph. */
+  GNUNET_assert (NULL != step->debug_name);
+  GNUNET_assert (NULL != dep->debug_name);
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Making step `%s' depend on `%s'\n",
+              step->debug_name,
+              dep->debug_name);
+#endif
 
-  if (NULL == session)
+  if (dep->subordinates_cap == dep->subordinates_len)
   {
-    GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "client tried to insert, but client is not in any session\n");
-    GNUNET_SERVER_client_disconnect (client);
-    return;
+    unsigned int target_size = 3 * (dep->subordinates_cap + 1) / 2;
+    GNUNET_array_grow (dep->subordinates,
+                       dep->subordinates_cap,
+                       target_size);
   }
 
-  msg = (struct GNUNET_CONSENSUS_ElementMessage *) m;
-  element_size = ntohs (msg->header.size )- sizeof (struct GNUNET_CONSENSUS_ElementMessage);
-
-  element = GNUNET_malloc (sizeof (struct GNUNET_CONSENSUS_Element) + element_size);
+  GNUNET_assert (dep->subordinates_len <= dep->subordinates_cap);
 
-  element->type = msg->element_type;
-  element->size = element_size;
-  memcpy (&element[1], &msg[1], element_size);
-  element->data = &element[1];
+  dep->subordinates[dep->subordinates_len] = step;
+  dep->subordinates_len++;
 
-  GNUNET_assert (NULL != element->data);
-
-  insert_element (session, element);
+  step->pending_prereq++;
+}
 
-  GNUNET_SERVER_receive_done (client, GNUNET_OK);
 
-  client_send_next (session);
+static struct Step *
+create_step (struct ConsensusSession *session, int round, int early_finishable)
+{
+  struct Step *step;
+  step = GNUNET_new (struct Step);
+  step->session = session;
+  step->round = round;
+  step->early_finishable = early_finishable;
+  GNUNET_CONTAINER_DLL_insert_tail (session->steps_head,
+                                    session->steps_tail,
+                                    step);
+  return step;
 }
 
 
-
 /**
- * Functions of this signature are called whenever writing operations
- * on a stream are executed
- *
- * @param cls the closure from GNUNET_STREAM_write
- * @param status the status of the stream at the time this function is called;
- *          GNUNET_STREAM_OK if writing to stream was completed successfully;
- *          GNUNET_STREAM_TIMEOUT if the given data is not sent successfully
- *          (this doesn't mean that the data is never sent, the receiver may
- *          have read the data but its ACKs may have been lost);
- *          GNUNET_STREAM_SHUTDOWN if the stream is shutdown for writing in the
- *          mean time; GNUNET_STREAM_SYSERR if the stream is broken and cannot
- *          be processed.
- * @param size the number of bytes written
+ * Construct the task graph for a single
+ * gradecast.
  */
-static void 
-write_queued (void *cls, enum GNUNET_STREAM_Status status, size_t size)
+static void
+construct_task_graph_gradecast (struct ConsensusSession *session,
+                                uint16_t rep,
+                                uint16_t lead,
+                                struct Step *step_before,
+                                struct Step *step_after)
 {
-  struct ConsensusPeerInformation *cpi;
+  uint16_t n = session->num_peers;
+  uint16_t me = session->local_peer_idx;
 
-  GNUNET_assert (GNUNET_STREAM_OK == status);
-  cpi = cls;
-  cpi->wh = NULL;
-  if (NULL != cpi->messages_head)
-  {
-    struct QueuedMessage *qm;
-    qm = cpi->messages_head;
-    GNUNET_CONTAINER_DLL_remove (cpi->messages_head, cpi->messages_tail, qm);
-    cpi->wh = GNUNET_STREAM_write (cpi->socket, qm->msg, ntohs (qm->msg->size),
-                                   GNUNET_TIME_UNIT_FOREVER_REL,
-                                   write_queued, cpi);
-    if (NULL != qm->cb)
-      qm->cb (qm->cls);
-    GNUNET_free (qm->msg);
-    GNUNET_free (qm);
-    GNUNET_assert (NULL != cpi->wh);
-  }
-}
+  uint16_t p1;
+  uint16_t p2;
 
+  /* The task we're currently setting up. */
+  struct TaskEntry task;
 
-static void
-shuffle (struct ConsensusSession *session)
-{
-  /* FIXME: implement */
-}
+  struct Step *step;
+  struct Step *prev_step;
 
+  uint16_t round;
 
-/**
- * Find and set the partner_incoming and partner_outgoing of our peer,
- * one of them may not exist in most cases.
- *
- * @param session the consensus session
- */
-static void
-find_partners (struct ConsensusSession *session)
-{
-  int mark[session->num_peers];
-  int i;
-  memset (mark, 0, session->num_peers * sizeof (int));
-  session->partner_incoming = session->partner_outgoing = NULL;
-  for (i = 0; i < session->num_peers; i++)
+  unsigned int k;
+
+  round = step_before->round + 1;
+
+  /* gcast step 1: leader disseminates */
+
+  step = create_step (session, round, GNUNET_YES);
+
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_asprintf (&step->debug_name, "disseminate leader %u rep %u", lead, rep);
+#endif
+  step_depend_on (step, step_before);
+
+  if (lead == me)
   {
-    int arc;
-    if (0 != mark[i])
-      continue;
-    arc = (i + (1 << session->exp_subround)) % session->num_peers;
-    mark[i] = mark[arc] = 1;
-    GNUNET_assert (i != arc);
-    if (i == session->local_peer_idx)
-    {
-      GNUNET_assert (NULL == session->partner_outgoing);
-      session->partner_outgoing = &session->info[session->shuffle[arc]];
-      session->partner_outgoing->exp_subround_finished = GNUNET_NO;
-    }
-    if (arc == session->local_peer_idx)
+    for (k = 0; k < n; k++)
     {
-      GNUNET_assert (NULL == session->partner_incoming);
-      session->partner_incoming = &session->info[session->shuffle[i]];
-      session->partner_incoming->exp_subround_finished = GNUNET_NO;
+      if (k == me)
+        continue;
+      p1 = me;
+      p2 = k;
+      arrange_peers (&p1, &p2, n);
+      task = ((struct TaskEntry) {
+        .step = step,
+        .start = task_start_reconcile,
+        .cancel = task_cancel_reconcile,
+        .key = (struct TaskKey) { PHASE_KIND_GRADECAST_LEADER, p1, p2, rep, me },
+      });
+      task.cls.setop.input_set = (struct SetKey) { SET_KIND_CURRENT, rep };
+      put_task (session->taskmap, &task);
     }
+    /* We run this task to make sure that the leader
+       has the stored the SET_KIND_LEADER set of himself,
+       so he can participate in the rest of the gradecast
+       without the code having to handle any special cases. */
+    task = ((struct TaskEntry) {
+      .step = step,
+      .key = (struct TaskKey) { PHASE_KIND_GRADECAST_LEADER, me, me, rep, me },
+      .start = task_start_reconcile,
+      .cancel = task_cancel_reconcile,
+    });
+    task.cls.setop.input_set = (struct SetKey) { SET_KIND_CURRENT, rep };
+    task.cls.setop.output_set = (struct SetKey) { SET_KIND_LEADER_PROPOSAL, rep, me };
+    task.cls.setop.output_diff = (struct DiffKey) { DIFF_KIND_LEADER_PROPOSAL, rep, me };
+    put_task (session->taskmap, &task);
   }
+  else
+  {
+    p1 = me;
+    p2 = lead;
+    arrange_peers (&p1, &p2, n);
+    task = ((struct TaskEntry) {
+      .step = step,
+      .key = (struct TaskKey) { PHASE_KIND_GRADECAST_LEADER, p1, p2, rep, lead },
+      .start = task_start_reconcile,
+      .cancel = task_cancel_reconcile,
+    });
+    task.cls.setop.input_set = (struct SetKey) { SET_KIND_CURRENT, rep };
+    task.cls.setop.output_set = (struct SetKey) { SET_KIND_LEADER_PROPOSAL, rep, lead };
+    task.cls.setop.output_diff = (struct DiffKey) { DIFF_KIND_LEADER_PROPOSAL, rep, lead };
+    put_task (session->taskmap, &task);
+  }
+
+  /* gcast phase 2: echo */
+  prev_step = step;
+  round += 1;
+  step = create_step (session, round, GNUNET_YES);
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_asprintf (&step->debug_name, "echo leader %u rep %u", lead, rep);
+#endif
+  step_depend_on (step, prev_step);
+
+  for (k = 0; k < n; k++)
+  {
+    p1 = k;
+    p2 = me;
+    arrange_peers (&p1, &p2, n);
+    task = ((struct TaskEntry) {
+      .step = step,
+      .key = (struct TaskKey) { PHASE_KIND_GRADECAST_ECHO, p1, p2, rep, lead },
+      .start = task_start_reconcile,
+      .cancel = task_cancel_reconcile,
+    });
+    task.cls.setop.input_set = (struct SetKey) { SET_KIND_LEADER_PROPOSAL, rep, lead };
+    task.cls.setop.output_rfn = (struct RfnKey) { RFN_KIND_ECHO, rep, lead };
+    put_task (session->taskmap, &task);
+  }
+
+  prev_step = step;
+  /* Same round, since step only has local tasks */
+  step = create_step (session, round, GNUNET_YES);
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_asprintf (&step->debug_name, "echo grade leader %u rep %u", lead, rep);
+#endif
+  step_depend_on (step, prev_step);
+
+  arrange_peers (&p1, &p2, n);
+  task = ((struct TaskEntry) {
+    .key = (struct TaskKey) { PHASE_KIND_GRADECAST_ECHO_GRADE, -1, -1, rep, lead },
+    .step = step,
+    .start = task_start_eval_echo
+  });
+  put_task (session->taskmap, &task);
+
+  prev_step = step;
+  round += 1;
+  step = create_step (session, round, GNUNET_YES);
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_asprintf (&step->debug_name, "confirm leader %u rep %u", lead, rep);
+#endif
+  step_depend_on (step, prev_step);
+
+  /* gcast phase 3: confirmation and grading */
+  for (k = 0; k < n; k++)
+  {
+    p1 = k;
+    p2 = me;
+    arrange_peers (&p1, &p2, n);
+    task = ((struct TaskEntry) {
+      .step = step,
+      .start = task_start_reconcile,
+      .cancel = task_cancel_reconcile,
+      .key = (struct TaskKey) { PHASE_KIND_GRADECAST_CONFIRM, p1, p2, rep, lead},
+    });
+    task.cls.setop.input_set = (struct SetKey) { SET_KIND_ECHO_RESULT, rep, lead };
+    task.cls.setop.output_rfn = (struct RfnKey) { RFN_KIND_CONFIRM, rep, lead };
+    /* If there was at least one element in the echo round that was
+       contested (i.e. it had no n-t majority), then we let the other peers
+       know, and other peers let us know.  The contested flag for each peer is
+       stored in the rfn. */
+    task.cls.setop.transceive_contested = GNUNET_YES;
+    put_task (session->taskmap, &task);
+  }
+
+  prev_step = step;
+  /* Same round, since step only has local tasks */
+  step = create_step (session, round, GNUNET_YES);
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_asprintf (&step->debug_name, "confirm grade leader %u rep %u", lead, rep);
+#endif
+  step_depend_on (step, prev_step);
+
+  task = ((struct TaskEntry) {
+    .step = step,
+    .key = (struct TaskKey) { PHASE_KIND_GRADECAST_CONFIRM_GRADE, -1, -1, rep, lead },
+    .start = task_start_grade,
+  });
+  put_task (session->taskmap, &task);
+
+  step_depend_on (step_after, step);
 }
 
 
 static void
-replay_premature_message (struct ConsensusPeerInformation *cpi)
+construct_task_graph (struct ConsensusSession *session)
 {
-  if (NULL != cpi->premature_strata_message)
-  {
-    struct StrataMessage *sm;
+  uint16_t n = session->num_peers;
+  uint16_t t = n / 3;
 
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "replaying premature SE\n");
-    sm = cpi->premature_strata_message;
-    cpi->premature_strata_message = NULL;
+  uint16_t me = session->local_peer_idx;
 
-    cpi->replaying_strata_message = GNUNET_YES;
-    handle_p2p_strata (cpi, sm);
-    cpi->replaying_strata_message = GNUNET_NO;
+  /* The task we're currently setting up. */
+  struct TaskEntry task;
 
-    GNUNET_free (sm);
-  }
-}
+  /* Current leader */
+  unsigned int lead;
 
+  struct Step *step;
+  struct Step *prev_step;
 
-/**
- * Do the next subround in the exp-scheme.
- * This function can be invoked as a timeout task, or called manually (tc will be NULL then).
- *
- * @param cls the session
- * @param tc task context, for when this task is invoked by the scheduler,
- *           NULL if invoked for another reason
- */
-static void
-subround_over (void *cls, const struct GNUNET_SCHEDULER_TaskContext *tc)
-{
-  struct ConsensusSession *session;
-  int i;
+  unsigned int round = 0;
 
-  /* don't kick off next subround if we're shutting down */
-  if ((NULL != tc) && (tc->reason & GNUNET_SCHEDULER_REASON_SHUTDOWN))
-    return;
-  session = cls;
-  /* don't send any messages from the last round */
-  /*
-  clear_peer_messages (session->partner_outgoing);
-  clear_peer_messages (session->partner_incoming);
-  for (i = 0; i < session->num_peers; i++)
-    clear_peer_messages (&session->info[i]);
-  */
-  /* cancel timeout */
-  if ((NULL == tc) && (session->round_timeout_tid != GNUNET_SCHEDULER_NO_TASK))
-    GNUNET_SCHEDULER_cancel (session->round_timeout_tid);
-  session->round_timeout_tid = GNUNET_SCHEDULER_NO_TASK;
-  /* check if we are done with the log phase, 2-peer consensus only does one log round */
-  if ( (session->exp_round == NUM_EXP_ROUNDS) ||
-       ((session->num_peers == 2) && (session->exp_round == 1)))
-  {
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: exp-round over\n", session->local_peer_idx);
-    round_over (session, NULL);
-    return;
-  }
-  if (session->exp_round == 0)
-  {
-    /* initialize everything for the log-rounds */
-    session->exp_round = 1;
-    session->exp_subround = 0;
-    if (NULL == session->shuffle)
-      session->shuffle = GNUNET_malloc ((sizeof (int)) * session->num_peers);
-    for (i = 0; i < session->num_peers; i++)
-      session->shuffle[i] = i;
-  }
-  else if (session->exp_subround + 1 >= (int) ceil (log2 (session->num_peers)))
-  {
-    /* subrounds done, start new log-round */
-    session->exp_round++;
-    session->exp_subround = 0;
-    shuffle (session);
-  }
-  else 
-  {
-    session->exp_subround++;
-  }
+  unsigned int i;
+
+  // XXX: introduce first step,
+  // where we wait for all insert acks
+  // from the set service
 
-  find_partners (session);
+  /* faster but brittle all-to-all */
+
+  // XXX: Not implemented yet
+
+  /* all-to-all step */
+
+  step = create_step (session, round, GNUNET_NO);
 
 #ifdef GNUNET_EXTRA_LOGGING
-  {
-    int in;
-    int out;
-    if (session->partner_outgoing == NULL)
-      out = -1;
-    else
-      out = (int) (session->partner_outgoing - session->info);
-    if (session->partner_incoming == NULL)
-      in = -1;
-    else
-      in = (int) (session->partner_incoming - session->info);
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%u: doing exp-round, r=%d, sub=%d, in: %d, out: %d\n", session->local_peer_idx,
-                session->exp_round, session->exp_subround, in, out);
-  }
-#endif /* GNUNET_EXTRA_LOGGING */
+  step->debug_name = GNUNET_strdup ("all to all");
+#endif
+
+  for (i = 0; i < n; i++)
+  {
+    uint16_t p1;
+    uint16_t p2;
+
+    p1 = me;
+    p2 = i;
+    arrange_peers (&p1, &p2, n);
+    task = ((struct TaskEntry) {
+      .key = (struct TaskKey) { PHASE_KIND_ALL_TO_ALL, p1, p2, -1, -1 },
+      .step = step,
+      .start = task_start_reconcile,
+      .cancel = task_cancel_reconcile,
+    });
+    task.cls.setop.input_set = (struct SetKey) { SET_KIND_CURRENT, 0 };
+    task.cls.setop.output_set = task.cls.setop.input_set;
+    task.cls.setop.do_not_remove = GNUNET_YES;
+    put_task (session->taskmap, &task);
+  }
+
+  round += 1;
+  prev_step = step;
+  step = create_step (session, round, GNUNET_NO);;
+#ifdef GNUNET_EXTRA_LOGGING
+  step->debug_name = GNUNET_strdup ("all to all 2");
+#endif
+  step_depend_on (step, prev_step);
+
 
-  if (NULL != session->partner_incoming)
+  for (i = 0; i < n; i++)
   {
-    session->partner_incoming->ibf_state = IBF_STATE_NONE;
-    session->partner_incoming->exp_subround_finished = GNUNET_NO;
-    session->partner_incoming->ibf_bucket_counter = 0;
+    uint16_t p1;
+    uint16_t p2;
 
-    /* maybe there's an early strata estimator? */
-    replay_premature_message (session->partner_incoming);
+    p1 = me;
+    p2 = i;
+    arrange_peers (&p1, &p2, n);
+    task = ((struct TaskEntry) {
+      .key = (struct TaskKey) { PHASE_KIND_ALL_TO_ALL_2, p1, p2, -1, -1 },
+      .step = step,
+      .start = task_start_reconcile,
+      .cancel = task_cancel_reconcile,
+    });
+    task.cls.setop.input_set = (struct SetKey) { SET_KIND_CURRENT, 0 };
+    task.cls.setop.output_set = task.cls.setop.input_set;
+    task.cls.setop.do_not_remove = GNUNET_YES;
+    put_task (session->taskmap, &task);
   }
 
-  if (NULL != session->partner_outgoing)
+  round += 1;
+
+  prev_step = step;
+  step = NULL;
+
+
+
+  /* Byzantine union */
+
+  /* sequential repetitions of the gradecasts */
+  for (i = 0; i < t + 1; i++)
   {
-    session->partner_outgoing->ibf_state = IBF_STATE_NONE;
-    session->partner_outgoing->ibf_bucket_counter = 0;
-    session->partner_outgoing->exp_subround_finished = GNUNET_NO;
+    struct Step *step_rep_start;
+    struct Step *step_rep_end;
 
-    if (NULL == session->partner_outgoing->socket)
-    {
-      session->partner_outgoing->socket =
-          GNUNET_STREAM_open (cfg, &session->partner_outgoing->peer_id, GNUNET_APPLICATION_TYPE_CONSENSUS,
-                              open_cb, session->partner_outgoing,
-                              GNUNET_STREAM_OPTION_END);
-    }
-    else if (GNUNET_YES == session->partner_outgoing->hello)
-    {
-      send_strata_estimator (session->partner_outgoing);
-    }
-    /* else: do nothing, the send hello cb will handle this */
+    /* Every repetition is in a separate round. */
+    step_rep_start = create_step (session, round, GNUNET_YES);
+#ifdef GNUNET_EXTRA_LOGGING
+    GNUNET_asprintf (&step_rep_start->debug_name, "gradecast start rep %u", i);
+#endif
+
+    step_depend_on (step_rep_start, prev_step);
+
+    /* gradecast has three rounds */
+    round += 3;
+    step_rep_end = create_step (session, round, GNUNET_YES);
+#ifdef GNUNET_EXTRA_LOGGING
+    GNUNET_asprintf (&step_rep_end->debug_name, "gradecast end rep %u", i);
+#endif
+
+    /* parallel gradecasts */
+    for (lead = 0; lead < n; lead++)
+      construct_task_graph_gradecast (session, i, lead, step_rep_start, step_rep_end);
+
+    task = ((struct TaskEntry) {
+      .step = step_rep_end,
+      .key = (struct TaskKey) { PHASE_KIND_APPLY_REP, -1, -1, i, -1},
+      .start = task_start_apply_round,
+    });
+    put_task (session->taskmap, &task);
+
+    prev_step = step_rep_end;
   }
 
-  /*
-  session->round_timeout_tid = GNUNET_SCHEDULER_add_delayed (GNUNET_TIME_relative_divide (session->conclude_timeout, 3 * NUM_EXP_ROUNDS),
-                                                                   subround_over, session);
-  */
+ /* There is no next gradecast round, thus the final
+    start step is the overall end step of the gradecasts */
+  round += 1;
+  step = create_step (session, round, GNUNET_NO);
+#ifdef GNUNET_EXTRA_LOGGING
+  GNUNET_asprintf (&step->debug_name, "finish");
+#endif
+  step_depend_on (step, prev_step);
+
+  task = ((struct TaskEntry) {
+    .step = step,
+    .key = (struct TaskKey) { PHASE_KIND_FINISH, -1, -1, -1, -1 },
+    .start = task_start_finish,
+  });
+  task.cls.finish.input_set = (struct SetKey) { SET_KIND_LAST_GRADECAST };
+
+  put_task (session->taskmap, &task);
 }
 
-static void
-contact_peer_a2a (struct ConsensusPeerInformation *cpi)
+
+
+/**
+ * Check join message.
+ *
+ * @param cls session of client that sent the message
+ * @param m message sent by the client
+ * @return #GNUNET_OK if @a m is well-formed
+ */
+static int
+check_client_join (void *cls,
+                   const struct GNUNET_CONSENSUS_JoinMessage *m)
 {
-  cpi->is_outgoing = GNUNET_YES;
-  if (NULL == cpi->socket)
-  {
-    cpi->socket = GNUNET_STREAM_open (cfg, &cpi->peer_id, GNUNET_APPLICATION_TYPE_CONSENSUS,
-                                      open_cb, cpi, GNUNET_STREAM_OPTION_END);
-  }
-  else if (GNUNET_YES == cpi->hello)
+  uint32_t listed_peers = ntohl (m->num_peers);
+
+  if ( (ntohs (m->header.size) - sizeof (*m)) !=
+       listed_peers * sizeof (struct GNUNET_PeerIdentity))
   {
-    send_strata_estimator (cpi);
+    GNUNET_break (0);
+    return GNUNET_SYSERR;
   }
+  return GNUNET_OK;
 }
 
+
 /**
- * Start the inventory round, contact all peers we are supposed to contact.
+ * Called when a client wants to join a consensus session.
  *
- * @param session the current session
+ * @param cls session of client that sent the message
+ * @param m message sent by the client
  */
 static void
-start_inventory (struct ConsensusSession *session)
-{
-  int i;
-  int last;
-
-  for (i = 0; i < session->num_peers; i++)
-  {
-    session->info[i].ibf_bucket_counter = 0;
-    session->info[i].ibf_state = IBF_STATE_NONE;
-    session->info[i].is_outgoing = GNUNET_NO;
+handle_client_join (void *cls,
+                    const struct GNUNET_CONSENSUS_JoinMessage *m)
+{
+  struct ConsensusSession *session = cls;
+  struct ConsensusSession *other_session;
+
+  initialize_session_peer_list (session,
+                                m);
+  compute_global_id (session,
+                     &m->session_id);
+
+  /* Check if some local client already owns the session.
+     It is only legal to have a session with an existing global id
+     if all other sessions with this global id are finished.*/
+  for (other_session = sessions_head;
+       NULL != other_session;
+       other_session = other_session->next)
+  {
+    if ( (other_session != session) &&
+         (0 == GNUNET_CRYPTO_hash_cmp (&session->global_id,
+                                       &other_session->global_id)) )
+      break;
   }
 
-  last = (session->local_peer_idx + ((session->num_peers - 1) / 2) + 1) % session->num_peers;
-  i = (session->local_peer_idx + 1) % session->num_peers;
-  while (i != last)
-  {
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d contacting P%d in all-to-all\n", session->local_peer_idx, i);
-    contact_peer_a2a (&session->info[i]);
-    session->info[i].is_outgoing = GNUNET_YES;
-    i = (i + 1) % session->num_peers;
-  }
-  // tie-breaker for even number of peers
-  if (((session->num_peers % 2) == 0) && (session->local_peer_idx < last))
-  {
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d contacting P%d in all-to-all (tie-breaker)\n", session->local_peer_idx, i);
-    session->info[last].is_outgoing = GNUNET_YES;
-    contact_peer_a2a (&session->info[last]);
-  }
+  session->conclude_deadline
+    = GNUNET_TIME_absolute_ntoh (m->deadline);
+  session->conclude_start
+    = GNUNET_TIME_absolute_ntoh (m->start);
+  session->local_peer_idx = get_peer_idx (&my_peer,
+                                          session);
+  GNUNET_assert (-1 != session->local_peer_idx);
+
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "Joining consensus session %s containing %u peers as %u with timeout %s\n",
+              GNUNET_h2s (&m->session_id),
+              session->num_peers,
+              session->local_peer_idx,
+              GNUNET_STRINGS_relative_time_to_string
+              (GNUNET_TIME_absolute_get_difference (session->conclude_start,
+                                                    session->conclude_deadline),
+               GNUNET_YES));
+
+  session->set_listener
+    = GNUNET_SET_listen (cfg,
+                         GNUNET_SET_OPERATION_UNION,
+                         &session->global_id,
+                         &set_listen_cb,
+                         session);
+
+  session->setmap = GNUNET_CONTAINER_multihashmap_create (1,
+                                                          GNUNET_NO);
+  session->taskmap = GNUNET_CONTAINER_multihashmap_create (1,
+                                                           GNUNET_NO);
+  session->diffmap = GNUNET_CONTAINER_multihashmap_create (1,
+                                                           GNUNET_NO);
+  session->rfnmap = GNUNET_CONTAINER_multihashmap_create (1,
+                                                          GNUNET_NO);
 
-  for (i = 0; i < session->num_peers; i++)
   {
-    if (GNUNET_NO == session->info[i].is_outgoing)
-      replay_premature_message (&session->info[i]);
+    struct SetEntry *client_set;
+
+    client_set = GNUNET_new (struct SetEntry);
+    client_set->h = GNUNET_SET_create (cfg,
+                                       GNUNET_SET_OPERATION_UNION);
+    client_set->key = ((struct SetKey) { SET_KIND_CURRENT, 0, 0 });
+    put_set (session,
+             client_set);
   }
+
+  session->peers_blacklisted = GNUNET_new_array (session->num_peers,
+                                                 int);
+
+  /* Just construct the task graph,
+     but don't run anything until the client calls conclude. */
+  construct_task_graph (session);
+  GNUNET_SERVICE_client_continue (session->client);
 }
 
+
 static void
-send_client_conclude_done (struct ConsensusSession *session)
+client_insert_done (void *cls)
 {
-  struct GNUNET_MessageHeader *msg;
-  session->current_round = CONSENSUS_ROUND_FINISH;
-  msg = GNUNET_malloc (sizeof *msg);
-  msg->type = htons (GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_CONCLUDE_DONE);
-  msg->size = htons (sizeof *msg);
-  queue_client_message (session, msg);
-  client_send_next (session);
+  // FIXME: implement
 }
 
+
 /**
- * Start the next round.
- * This function can be invoked as a timeout task, or called manually (tc will be NULL then).
+ * Called when a client performs an insert operation.
  *
- * @param cls the session
- * @param tc task context, for when this task is invoked by the scheduler,
- *           NULL if invoked for another reason
+ * @param cls client handle
+ * @param msg message sent by the client
+ * @return #GNUNET_OK (always well-formed)
  */
-static void 
-round_over (void *cls, const struct GNUNET_SCHEDULER_TaskContext *tc)
+static int
+check_client_insert (void *cls,
+                      const struct GNUNET_CONSENSUS_ElementMessage *msg)
 {
-  struct ConsensusSession *session;
-
-  /* don't kick off next round if we're shutting down */
-  if ((NULL != tc) && (tc->reason & GNUNET_SCHEDULER_REASON_SHUTDOWN))
-    return;
-
-  session = cls;
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: round over\n", session->local_peer_idx);
-
-  /*
-  for (i = 0; i < session->num_peers; i++)
-    clear_peer_messages (&session->info[i]);
-  */
-
-  if ((NULL == tc) && (session->round_timeout_tid != GNUNET_SCHEDULER_NO_TASK))
-  {
-    GNUNET_SCHEDULER_cancel (session->round_timeout_tid);
-    session->round_timeout_tid = GNUNET_SCHEDULER_NO_TASK;
-  }
-
-  switch (session->current_round)
-  {
-    case CONSENSUS_ROUND_BEGIN:
-      session->current_round = CONSENSUS_ROUND_EXCHANGE;
-      session->exp_round = 0;
-      subround_over (session, NULL);
-      break;
-    case CONSENSUS_ROUND_EXCHANGE:
-      /* handle two peers specially */
-      if (session->num_peers <= 2)
-      {
-        GNUNET_log (GNUNET_ERROR_TYPE_INFO, "P%d: done\n", session->local_peer_idx);
-        send_client_conclude_done (session);
-        return;
-      }
-      session->current_round = CONSENSUS_ROUND_INVENTORY;
-      start_inventory (session);
-      break;
-    case CONSENSUS_ROUND_INVENTORY:
-      session->current_round = CONSENSUS_ROUND_STOCK;
-      session->exp_round = 0;
-      subround_over (session, NULL);
-      break;
-    case CONSENSUS_ROUND_STOCK:
-      session->current_round = CONSENSUS_ROUND_FINISH;
-      send_client_conclude_done (session);
-      break;
-    default:
-      GNUNET_assert (0);
-  }
+  return GNUNET_OK;
 }
 
 
 /**
- * Called when a client performs the conclude operation.
+ * Called when a client performs an insert operation.
  *
- * @param cls (unused)
- * @param client client handle
- * @param message message sent by the client
+ * @param cls client handle
+ * @param msg message sent by the client
  */
 static void
-client_conclude (void *cls,
-                 struct GNUNET_SERVER_Client *client,
-                 const struct GNUNET_MessageHeader *message)
+handle_client_insert (void *cls,
+                      const struct GNUNET_CONSENSUS_ElementMessage *msg)
 {
-  struct ConsensusSession *session;
-  struct GNUNET_CONSENSUS_ConcludeMessage *cmsg;
+  struct ConsensusSession *session = cls;
+  ssize_t element_size;
+  struct GNUNET_SET_Handle *initial_set;
+  struct ConsensusElement *ce;
 
-  cmsg = (struct GNUNET_CONSENSUS_ConcludeMessage *) message;
-
-  session = sessions_head;
-  while ((session != NULL) && (session->client != client))
-    session = session->next;
-  if (NULL == session)
-  {
-    /* client not found */
-    GNUNET_break (0);
-    GNUNET_SERVER_client_disconnect (client);
-    return;
-  }
-
-  if (CONSENSUS_ROUND_BEGIN != session->current_round)
+  if (GNUNET_YES == session->conclude_started)
   {
-    /* client requested conclude twice */
     GNUNET_break (0);
-    /* client may still own a session, destroy it */
-    disconnect_client (client);
+    GNUNET_SERVICE_client_drop (session->client);
     return;
   }
 
-  if (session->num_peers <= 1)
-  {
-    send_client_conclude_done (session);
-  }
-  else
-  {
-    session->conclude_timeout = GNUNET_TIME_relative_ntoh (cmsg->timeout);
-    /* the 'begin' round is over, start with the next, real round */
-    round_over (session, NULL);
-  }
-
-  GNUNET_SERVER_receive_done (client, GNUNET_OK);
-  client_send_next (session);
-}
-
+  element_size = ntohs (msg->header.size) - sizeof (struct GNUNET_CONSENSUS_ElementMessage);
+  ce = GNUNET_malloc (sizeof (struct ConsensusElement) + element_size);
+  GNUNET_memcpy (&ce[1], &msg[1], element_size);
+  ce->payload_type = msg->element_type;
 
-/**
- * Called when a client sends an ack
- *
- * @param cls (unused)
- * @param client client handle
- * @param message message sent by the client
- */
-void
-client_ack (void *cls,
-             struct GNUNET_SERVER_Client *client,
-             const struct GNUNET_MessageHeader *message)
-{
-  struct ConsensusSession *session;
-  struct GNUNET_CONSENSUS_AckMessage *msg;
-  struct PendingElement *pending;
-  struct GNUNET_CONSENSUS_Element *element;
+  struct GNUNET_SET_Element element = {
+    .element_type = GNUNET_BLOCK_TYPE_CONSENSUS_ELEMENT,
+    .size = sizeof (struct ConsensusElement) + element_size,
+    .data = ce,
+  };
 
-  session = sessions_head;
-  while (NULL != session)
   {
-    if (session->client == client)
-      break;
-  }
+    struct SetKey key = { SET_KIND_CURRENT, 0, 0 };
+    struct SetEntry *entry;
 
-  if (NULL == session)
-  {
-    GNUNET_log (GNUNET_ERROR_TYPE_WARNING, "client tried to ack, but client is not in any session\n");
-    GNUNET_SERVER_client_disconnect (client);
-    return;
+    entry = lookup_set (session,
+                        &key);
+    GNUNET_assert (NULL != entry);
+    initial_set = entry->h;
   }
 
-  pending = session->client_approval_head;
-
-  GNUNET_CONTAINER_DLL_remove (session->client_approval_head, session->client_approval_tail, pending);
+  session->num_client_insert_pending++;
+  GNUNET_SET_add_element (initial_set,
+                          &element,
+                          &client_insert_done,
+                          session);
 
-  msg = (struct GNUNET_CONSENSUS_AckMessage *) message;
-
-  if (msg->keep)
+#ifdef GNUNET_EXTRA_LOGGING
   {
-    element = pending->element;
-    insert_element (session, element);
-    GNUNET_log (GNUNET_ERROR_TYPE_INFO, "got client ack\n");
+    GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+                "P%u: element %s added\n",
+                session->local_peer_idx,
+                debug_str_element (&element));
   }
-
-  GNUNET_free (pending);
-
-  GNUNET_SERVER_receive_done (client, GNUNET_OK);
+#endif
+  GNUNET_free (ce);
+  GNUNET_SERVICE_client_continue (session->client);
 }
 
 
 /**
- * Task that disconnects from core.
+ * Called when a client performs the conclude operation.
  *
- * @param cls core handle
- * @param tc context information (why was this task triggered now)
+ * @param cls client handle
+ * @param message message sent by the client
  */
 static void
-disconnect_core (void *cls,
-                 const struct GNUNET_SCHEDULER_TaskContext *tc)
+handle_client_conclude (void *cls,
+                        const struct GNUNET_MessageHeader *message)
 {
-  if (core != NULL)
-  {
-    GNUNET_CORE_disconnect (core);
-    core = NULL;
-  }
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "disconnected from core\n");
-}
-
-
-static void
-core_startup (void *cls,
-              struct GNUNET_CORE_Handle *core,
-              const struct GNUNET_PeerIdentity *peer)
-{
-  struct ConsensusSession *session;
+  struct ConsensusSession *session = cls;
 
-  my_peer = GNUNET_memdup(peer, sizeof (struct GNUNET_PeerIdentity));
-  /* core can't be disconnected directly in the core startup callback, schedule a task to do it! */
-  GNUNET_SCHEDULER_add_now (&disconnect_core, core);
-  GNUNET_log(GNUNET_ERROR_TYPE_INFO, "connected to core\n");
-
-  session = sessions_head;
-  while (NULL != session)
+  if (GNUNET_YES == session->conclude_started)
   {
-    if (NULL != session->join_msg)
-      initialize_session (session);
-    session = session->next;
+    /* conclude started twice */
+    GNUNET_break (0);
+    GNUNET_SERVICE_client_drop (session->client);
+    return;
   }
+  GNUNET_log (GNUNET_ERROR_TYPE_DEBUG,
+              "conclude requested\n");
+  session->conclude_started = GNUNET_YES;
+  install_step_timeouts (session);
+  run_ready_steps (session);
+  GNUNET_SERVICE_client_continue (session->client);
 }
 
 
@@ -2662,58 +3283,15 @@ core_startup (void *cls,
  * Called to clean up, after a shutdown has been requested.
  *
  * @param cls closure
- * @param tc context information (why was this task triggered now)
  */
 static void
-shutdown_task (void *cls,
-               const struct GNUNET_SCHEDULER_TaskContext *tc)
+shutdown_task (void *cls)
 {
-  /* FIXME: complete; write separate destructors for different data types */
-
-  while (NULL != incoming_sockets_head)
-  {
-    struct IncomingSocket *socket;
-    socket = incoming_sockets_head;
-    if (NULL != socket->rh)
-    {
-      GNUNET_STREAM_read_cancel (socket->rh);
-      socket->rh = NULL;
-    } 
-    if (NULL == socket->cpi)
-    {
-      GNUNET_STREAM_close (socket->socket);
-      socket->socket = NULL;
-      if (NULL != socket->mst)
-      {
-        GNUNET_SERVER_mst_destroy (socket->mst);
-        socket->mst = NULL;
-      }
-    }
-    incoming_sockets_head = incoming_sockets_head->next;
-    GNUNET_free (socket);
-  }
-
-  while (NULL != sessions_head)
-  {
-    struct ConsensusSession *session;
-    session = sessions_head->next;
-    destroy_session (sessions_head);
-    sessions_head = session;
-  }
-
-  if (NULL != core)
-  {
-    GNUNET_CORE_disconnect (core);
-    core = NULL;
-  }
-
-  if (NULL != listener)
-  {
-    GNUNET_STREAM_listen_close (listener);
-    listener = NULL;
-  } 
-
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "handled shutdown request\n");
+  GNUNET_log (GNUNET_ERROR_TYPE_INFO,
+              "shutting down\n");
+  GNUNET_STATISTICS_destroy (statistics,
+                             GNUNET_NO);
+  statistics = NULL;
 }
 
 
@@ -2721,58 +3299,103 @@ shutdown_task (void *cls,
  * Start processing consensus requests.
  *
  * @param cls closure
- * @param server the initialized server
  * @param c configuration to use
+ * @param service the initialized service
  */
 static void
-run (void *cls, struct GNUNET_SERVER_Handle *server, const struct GNUNET_CONFIGURATION_Handle *c)
+run (void *cls,
+     const struct GNUNET_CONFIGURATION_Handle *c,
+     struct GNUNET_SERVICE_Handle *service)
 {
-  /* core is only used to retrieve the peer identity */
-  static const struct GNUNET_CORE_MessageHandler core_handlers[] = {
-    {NULL, 0, 0}
-  };
-  static const struct GNUNET_SERVER_MessageHandler server_handlers[] = {
-    {&client_join, NULL, GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_JOIN, 0},
-    {&client_insert, NULL, GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_INSERT, 0},
-    {&client_conclude, NULL, GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_CONCLUDE,
-        sizeof (struct GNUNET_CONSENSUS_ConcludeMessage)},
-    {&client_ack, NULL, GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_ACK,
-        sizeof (struct GNUNET_CONSENSUS_AckMessage)},
-    {NULL, NULL, 0, 0}
-  };
-
   cfg = c;
-  srv = server;
-
-  GNUNET_SERVER_add_handlers (server, server_handlers);
-
-  GNUNET_SCHEDULER_add_delayed (GNUNET_TIME_UNIT_FOREVER_REL, &shutdown_task, NULL);
+  if (GNUNET_OK !=
+      GNUNET_CRYPTO_get_peer_identity (cfg,
+                                       &my_peer))
+  {
+    GNUNET_log (GNUNET_ERROR_TYPE_ERROR,
+                "Could not retrieve host identity\n");
+    GNUNET_SCHEDULER_shutdown ();
+    return;
+  }
+  statistics = GNUNET_STATISTICS_create ("consensus",
+                                         cfg);
+  GNUNET_SCHEDULER_add_shutdown (&shutdown_task,
+                                 NULL);
+}
 
-  listener = GNUNET_STREAM_listen (cfg, GNUNET_APPLICATION_TYPE_CONSENSUS,
-                                   listen_cb, NULL,
-                                   GNUNET_STREAM_OPTION_END);
 
-  /* we have to wait for the core_startup callback before proceeding with the consensus service startup */
-  core = GNUNET_CORE_connect (c, NULL, &core_startup, NULL, NULL, NULL, GNUNET_NO, NULL, GNUNET_NO, core_handlers);
-  GNUNET_assert (NULL != core);
+/**
+ * Callback called when a client connects to the service.
+ *
+ * @param cls closure for the service
+ * @param c the new client that connected to the service
+ * @param mq the message queue used to send messages to the client
+ * @return @a c
+ */
+static void *
+client_connect_cb (void *cls,
+                  struct GNUNET_SERVICE_Client *c,
+                  struct GNUNET_MQ_Handle *mq)
+{
+  struct ConsensusSession *session = GNUNET_new (struct ConsensusSession);
 
-  GNUNET_log(GNUNET_ERROR_TYPE_INFO, "consensus running\n");
+  session->client = c;
+  session->client_mq = mq;
+  GNUNET_CONTAINER_DLL_insert (sessions_head,
+                               sessions_tail,
+                               session);
+  return session;
 }
 
 
 /**
- * The main function for the consensus service.
+ * Callback called when a client disconnected from the service
  *
- * @param argc number of arguments from the command line
- * @param argv command line arguments
- * @return 0 ok, 1 on error
+ * @param cls closure for the service
+ * @param c the client that disconnected
+ * @param internal_cls should be equal to @a c
  */
-int
-main (int argc, char *const *argv)
+static void
+client_disconnect_cb (void *cls,
+                     struct GNUNET_SERVICE_Client *c,
+                     void *internal_cls)
 {
-  int ret;
-  ret = GNUNET_SERVICE_run (argc, argv, "consensus", GNUNET_SERVICE_OPTION_NONE, &run, NULL);
-  GNUNET_log (GNUNET_ERROR_TYPE_INFO, "exit\n");
-  return (GNUNET_OK == ret) ? 0 : 1;
+  struct ConsensusSession *session = internal_cls;
+
+  if (NULL != session->set_listener)
+  {
+    GNUNET_SET_listen_cancel (session->set_listener);
+    session->set_listener = NULL;
+  }
+  GNUNET_CONTAINER_DLL_remove (sessions_head,
+                               sessions_tail,
+                               session);
+  GNUNET_free (session);
 }
 
+
+/**
+ * Define "main" method using service macro.
+ */
+GNUNET_SERVICE_MAIN
+("consensus",
+ GNUNET_SERVICE_OPTION_NONE,
+ &run,
+ &client_connect_cb,
+ &client_disconnect_cb,
+ NULL,
+ GNUNET_MQ_hd_fixed_size (client_conclude,
+                          GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_CONCLUDE,
+                          struct GNUNET_MessageHeader,
+                          NULL),
+ GNUNET_MQ_hd_var_size (client_insert,
+                        GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_INSERT,
+                        struct GNUNET_CONSENSUS_ElementMessage,
+                        NULL),
+ GNUNET_MQ_hd_var_size (client_join,
+                        GNUNET_MESSAGE_TYPE_CONSENSUS_CLIENT_JOIN,
+                        struct GNUNET_CONSENSUS_JoinMessage,
+                        NULL),
+ GNUNET_MQ_handler_end ());
+
+/* end of gnunet-service-consensus.c */