Move data structures from execution.h to history.h
[c11tester.git] / execution.cc
index b62f69443dff0f10df28ca58e97cfa02926dbe00..7b8461226a8860d60a0311c2fa2538607c9b1363 100644 (file)
@@ -6,7 +6,6 @@
 #include "model.h"
 #include "execution.h"
 #include "action.h"
-#include "nodestack.h"
 #include "schedule.h"
 #include "common.h"
 #include "clockvector.h"
@@ -14,7 +13,9 @@
 #include "datarace.h"
 #include "threads-model.h"
 #include "bugmessage.h"
+#include "history.h"
 #include "fuzzer.h"
+#include "newfuzzer.h"
 
 #define INITIAL_THREAD_ID       0
 
@@ -27,7 +28,6 @@ struct model_snapshot_members {
                next_thread_id(INITIAL_THREAD_ID),
                used_sequence_numbers(0),
                bugs(),
-               bad_synchronization(false),
                asserted(false)
        { }
 
@@ -41,37 +41,39 @@ struct model_snapshot_members {
        modelclock_t used_sequence_numbers;
        SnapVector<bug_message *> bugs;
        /** @brief Incorrectly-ordered synchronization was made */
-       bool bad_synchronization;
        bool asserted;
 
        SNAPSHOTALLOC
 };
 
 /** @brief Constructor */
-ModelExecution::ModelExecution(ModelChecker *m, Scheduler *scheduler, NodeStack *node_stack) :
+ModelExecution::ModelExecution(ModelChecker *m, Scheduler *scheduler) :
        model(m),
        params(NULL),
        scheduler(scheduler),
        action_trace(),
        thread_map(2),  /* We'll always need at least 2 threads */
        pthread_map(0),
-       pthread_counter(0),
+       pthread_counter(1),
        obj_map(),
        condvar_waiters_map(),
        obj_thrd_map(),
        mutex_map(),
        thrd_last_action(1),
        thrd_last_fence_release(),
-       node_stack(node_stack),
        priv(new struct model_snapshot_members ()),
-                        mo_graph(new CycleGraph()),
-       fuzzer(new Fuzzer())
+       mo_graph(new CycleGraph()),
+       fuzzer(new NewFuzzer()),
+       isfinished(false)
 {
        /* Initialize a model-checker thread, for special ModelActions */
        model_thread = new Thread(get_next_id());
        add_thread(model_thread);
+       fuzzer->register_engine(m->get_history(), this);
        scheduler->register_engine(this);
-       node_stack->register_engine(this);
+#ifdef TLS
+       pthread_key_create(&pthreadkey, tlsdestructor);
+#endif
 }
 
 /** @brief Destructor */
@@ -89,7 +91,7 @@ int ModelExecution::get_execution_number() const
        return model->get_execution_number();
 }
 
-static action_list_t * get_safe_ptr_action(HashTable<const void *, action_list_t *, uintptr_t, 4> * hash, void * ptr)
+static action_list_t * get_safe_ptr_action(HashTable<const void *, action_list_t *, uintptr_t, 2> * hash, void * ptr)
 {
        action_list_t *tmp = hash->get(ptr);
        if (tmp == NULL) {
@@ -99,7 +101,7 @@ static action_list_t * get_safe_ptr_action(HashTable<const void *, action_list_t
        return tmp;
 }
 
-static SnapVector<action_list_t> * get_safe_ptr_vect_action(HashTable<void *, SnapVector<action_list_t> *, uintptr_t, 4> * hash, void * ptr)
+static SnapVector<action_list_t> * get_safe_ptr_vect_action(HashTable<const void *, SnapVector<action_list_t> *, uintptr_t, 2> * hash, void * ptr)
 {
        SnapVector<action_list_t> *tmp = hash->get(ptr);
        if (tmp == NULL) {
@@ -109,18 +111,6 @@ static SnapVector<action_list_t> * get_safe_ptr_vect_action(HashTable<void *, Sn
        return tmp;
 }
 
-action_list_t * ModelExecution::get_actions_on_obj(void * obj, thread_id_t tid) const
-{
-       SnapVector<action_list_t> *wrv = obj_thrd_map.get(obj);
-       if (wrv==NULL)
-               return NULL;
-       unsigned int thread=id_to_int(tid);
-       if (thread < wrv->size())
-               return &(*wrv)[thread];
-       else
-               return NULL;
-}
-
 /** @return a thread ID for a new Thread */
 thread_id_t ModelExecution::get_next_id()
 {
@@ -139,6 +129,12 @@ modelclock_t ModelExecution::get_next_seq_num()
        return ++priv->used_sequence_numbers;
 }
 
+/** Restore the last used sequence number when actions of a thread are postponed by Fuzzer */
+void ModelExecution::restore_last_seq_num()
+{
+       priv->used_sequence_numbers--;
+}
+
 /**
  * @brief Should the current action wake up a given thread?
  *
@@ -164,6 +160,12 @@ bool ModelExecution::should_wake_up(const ModelAction *curr, const Thread *threa
                if (fence_release && *(get_last_action(thread->get_id())) < *fence_release)
                        return true;
        }
+       /* The sleep is literally sleeping */
+       if (asleep->is_sleep()) {
+               if (fuzzer->shouldWake(asleep))
+                       return true;
+       }
+
        return false;
 }
 
@@ -172,29 +174,20 @@ void ModelExecution::wake_up_sleeping_actions(ModelAction *curr)
        for (unsigned int i = 0;i < get_num_threads();i++) {
                Thread *thr = get_thread(int_to_id(i));
                if (scheduler->is_sleep_set(thr)) {
-                       if (should_wake_up(curr, thr))
+                       if (should_wake_up(curr, thr)) {
                                /* Remove this thread from sleep set */
                                scheduler->remove_sleep(thr);
+                               if (thr->get_pending()->is_sleep())
+                                       thr->set_wakeup_state(true);
+                       }
                }
        }
 }
 
-/** @brief Alert the model-checker that an incorrectly-ordered
- * synchronization was made */
-void ModelExecution::set_bad_synchronization()
-{
-       priv->bad_synchronization = true;
-}
-
-bool ModelExecution::assert_bug(const char *msg)
+void ModelExecution::assert_bug(const char *msg)
 {
        priv->bugs.push_back(new bug_message(msg));
-
-       if (isfeasibleprefix()) {
-               set_assert();
-               return true;
-       }
-       return false;
+       set_assert();
 }
 
 /** @return True, if any bugs have been reported for this execution */
@@ -265,6 +258,20 @@ bool ModelExecution::is_complete_execution() const
        return true;
 }
 
+ModelAction * ModelExecution::convertNonAtomicStore(void * location) {
+       uint64_t value = *((const uint64_t *) location);
+       modelclock_t storeclock;
+       thread_id_t storethread;
+       getStoreThreadAndClock(location, &storethread, &storeclock);
+       setAtomicStoreFlag(location);
+       ModelAction * act = new ModelAction(NONATOMIC_WRITE, memory_order_relaxed, location, value, get_thread(storethread));
+       act->set_seq_number(storeclock);
+       add_normal_write_to_lists(act);
+       add_write_to_lists(act);
+       w_modification_order(act);
+       model->get_history()->process_action(act, act->get_tid());
+       return act;
+}
 
 /**
  * Processes a read model action.
@@ -274,23 +281,48 @@ bool ModelExecution::is_complete_execution() const
  */
 bool ModelExecution::process_read(ModelAction *curr, SnapVector<ModelAction *> * rf_set)
 {
-       while(true) {
+       SnapVector<const ModelAction *> * priorset = new SnapVector<const ModelAction *>();
+       bool hasnonatomicstore = hasNonAtomicStore(curr->get_location());
+       if (hasnonatomicstore) {
+               ModelAction * nonatomicstore = convertNonAtomicStore(curr->get_location());
+               rf_set->push_back(nonatomicstore);
+       }
 
+       // Remove writes that violate read modification order
+       uint i = 0;
+       while (i < rf_set->size()) {
+               ModelAction * rf = (*rf_set)[i];
+               if (!r_modification_order(curr, rf, NULL, NULL, true)) {
+                       (*rf_set)[i] = rf_set->back();
+                       rf_set->pop_back();
+               } else
+                       i++;
+       }
+
+       while(true) {
                int index = fuzzer->selectWrite(curr, rf_set);
-               const ModelAction *rf = (*rf_set)[index];
 
+               ModelAction *rf = (*rf_set)[index];
 
                ASSERT(rf);
-
-               mo_graph->startChanges();
-               r_modification_order(curr, rf);
-               if (!is_infeasible()) {
+               bool canprune = false;
+               if (r_modification_order(curr, rf, priorset, &canprune)) {
+                       for(unsigned int i=0;i<priorset->size();i++) {
+                               mo_graph->addEdge((*priorset)[i], rf);
+                       }
                        read_from(curr, rf);
-                       mo_graph->commitChanges();
                        get_thread(curr)->set_return_value(curr->get_return_value());
-                       return updated;
+                       delete priorset;
+                       if (canprune && curr->get_type() == ATOMIC_READ) {
+                               int tid = id_to_int(curr->get_tid());
+                               (*obj_thrd_map.get(curr->get_location()))[tid].pop_back();
+                       }
+                       return true;
                }
-               mo_graph->rollbackChanges();
+
+               ASSERT(false);
+               /* TODO: Following code not needed anymore */
+               priorset->clear();
                (*rf_set)[index] = rf_set->back();
                rf_set->pop_back();
        }
@@ -332,8 +364,9 @@ bool ModelExecution::process_mutex(ModelAction *curr)
        }
        //otherwise fall into the lock case
        case ATOMIC_LOCK: {
-               if (curr->get_cv()->getClock(state->alloc_tid) <= state->alloc_clock)
-                       assert_bug("Lock access before initialization");
+               //TODO: FIND SOME BETTER WAY TO CHECK LOCK INITIALIZED OR NOT
+               //if (curr->get_cv()->getClock(state->alloc_tid) <= state->alloc_clock)
+               //      assert_bug("Lock access before initialization");
                state->locked = get_thread(curr);
                ModelAction *unlock = get_last_unlock(curr);
                //synchronize with the previous unlock statement
@@ -343,8 +376,7 @@ bool ModelExecution::process_mutex(ModelAction *curr)
                }
                break;
        }
-       case ATOMIC_WAIT:
-       case ATOMIC_UNLOCK: {
+       case ATOMIC_WAIT: {
                /* wake up the other threads */
                for (unsigned int i = 0;i < get_num_threads();i++) {
                        Thread *t = get_thread(int_to_id(i));
@@ -356,24 +388,45 @@ bool ModelExecution::process_mutex(ModelAction *curr)
                /* unlock the lock - after checking who was waiting on it */
                state->locked = NULL;
 
-               if (!curr->is_wait())
-                       break;/* The rest is only for ATOMIC_WAIT */
+               if (fuzzer->shouldWait(curr)) {
+                       /* disable this thread */
+                       get_safe_ptr_action(&condvar_waiters_map, curr->get_location())->push_back(curr);
+                       scheduler->sleep(get_thread(curr));
+               }
 
                break;
        }
+       case ATOMIC_TIMEDWAIT:
+       case ATOMIC_UNLOCK: {
+               //TODO: FIX WAIT SITUATION...WAITS CAN SPURIOUSLY FAIL...TIMED WAITS SHOULD PROBABLY JUST BE THE SAME AS NORMAL WAITS...THINK ABOUT PROBABILITIES THOUGH....AS IN TIMED WAIT MUST FAIL TO GUARANTEE PROGRESS...NORMAL WAIT MAY FAIL...SO NEED NORMAL WAIT TO WORK CORRECTLY IN THE CASE IT SPURIOUSLY FAILS AND IN THE CASE IT DOESN'T...  TIMED WAITS MUST EVENMTUALLY RELEASE...
+
+               /* wake up the other threads */
+               for (unsigned int i = 0;i < get_num_threads();i++) {
+                       Thread *t = get_thread(int_to_id(i));
+                       Thread *curr_thrd = get_thread(curr);
+                       if (t->waiting_on() == curr_thrd && t->get_pending()->is_lock())
+                               scheduler->wake(t);
+               }
+
+               /* unlock the lock - after checking who was waiting on it */
+               state->locked = NULL;
+               break;
+       }
        case ATOMIC_NOTIFY_ALL: {
                action_list_t *waiters = get_safe_ptr_action(&condvar_waiters_map, curr->get_location());
                //activate all the waiting threads
-               for (action_list_t::iterator rit = waiters->begin();rit != waiters->end();rit++) {
-                       scheduler->wake(get_thread(*rit));
+               for (sllnode<ModelAction *> * rit = waiters->begin();rit != NULL;rit=rit->getNext()) {
+                       scheduler->wake(get_thread(rit->getVal()));
                }
                waiters->clear();
                break;
        }
        case ATOMIC_NOTIFY_ONE: {
                action_list_t *waiters = get_safe_ptr_action(&condvar_waiters_map, curr->get_location());
-               Thread * thread = fuzzer->selectNotify(waiters);
-               scheduler->wake(thread);
+               if (waiters->size() != 0) {
+                       Thread * thread = fuzzer->selectNotify(waiters);
+                       scheduler->wake(thread);
+               }
                break;
        }
 
@@ -388,15 +441,10 @@ bool ModelExecution::process_mutex(ModelAction *curr)
  * @param curr The ModelAction to process
  * @return True if the mo_graph was updated or promises were resolved
  */
-bool ModelExecution::process_write(ModelAction *curr)
+void ModelExecution::process_write(ModelAction *curr)
 {
-
-       bool updated_mod_order = w_modification_order(curr);
-
-       mo_graph->commitChanges();
-
+       w_modification_order(curr);
        get_thread(curr)->set_return_value(VALUE_NONE);
-       return updated_mod_order;
 }
 
 /**
@@ -417,10 +465,10 @@ bool ModelExecution::process_fence(ModelAction *curr)
        bool updated = false;
        if (curr->is_acquire()) {
                action_list_t *list = &action_trace;
-               action_list_t::reverse_iterator rit;
+               sllnode<ModelAction *> * rit;
                /* Find X : is_read(X) && X --sb-> curr */
-               for (rit = list->rbegin();rit != list->rend();rit++) {
-                       ModelAction *act = *rit;
+               for (rit = list->end();rit != NULL;rit=rit->getPrev()) {
+                       ModelAction *act = rit->getVal();
                        if (act == curr)
                                continue;
                        if (act->get_tid() != curr->get_tid())
@@ -438,11 +486,8 @@ bool ModelExecution::process_fence(ModelAction *curr)
                                continue;
 
                        /* Establish hypothetical release sequences */
-                       rel_heads_list_t release_heads;
-                       get_release_seq_heads(curr, act, &release_heads);
-                       for (unsigned int i = 0;i < release_heads.size();i++)
-                               synchronize(release_heads[i], curr);
-                       if (release_heads.size() != 0)
+                       ClockVector *cv = get_hb_from_write(act->get_reads_from());
+                       if (cv != NULL && curr->get_cv()->merge(cv))
                                updated = true;
                }
        }
@@ -460,10 +505,8 @@ bool ModelExecution::process_fence(ModelAction *curr)
  * @param curr The current action
  * @return True if synchronization was updated or a thread completed
  */
-bool ModelExecution::process_thread_action(ModelAction *curr)
+void ModelExecution::process_thread_action(ModelAction *curr)
 {
-       bool updated = false;
-
        switch (curr->get_type()) {
        case THREAD_CREATE: {
                thrd_t *thrd = (thrd_t *)curr->get_location();
@@ -493,19 +536,25 @@ bool ModelExecution::process_thread_action(ModelAction *curr)
                Thread *blocking = curr->get_thread_operand();
                ModelAction *act = get_last_action(blocking->get_id());
                synchronize(act, curr);
-               updated = true; /* trigger rel-seq checks */
                break;
        }
        case PTHREAD_JOIN: {
                Thread *blocking = curr->get_thread_operand();
                ModelAction *act = get_last_action(blocking->get_id());
                synchronize(act, curr);
-               updated = true; /* trigger rel-seq checks */
                break;  // WL: to be add (modified)
        }
 
+       case THREADONLY_FINISH:
        case THREAD_FINISH: {
                Thread *th = get_thread(curr);
+               if (curr->get_type() == THREAD_FINISH &&
+                               th == model->getInitThread()) {
+                       th->complete();
+                       setFinished();
+                       break;
+               }
+
                /* Wake up any joining threads */
                for (unsigned int i = 0;i < get_num_threads();i++) {
                        Thread *waiting = get_thread(int_to_id(i));
@@ -514,23 +563,26 @@ bool ModelExecution::process_thread_action(ModelAction *curr)
                                scheduler->wake(waiting);
                }
                th->complete();
-               updated = true; /* trigger rel-seq checks */
                break;
        }
        case THREAD_START: {
                break;
        }
+       case THREAD_SLEEP: {
+               Thread *th = get_thread(curr);
+               th->set_pending(curr);
+               scheduler->add_sleep(th);
+               break;
+       }
        default:
                break;
        }
-
-       return updated;
 }
 
 /**
  * Initialize the current action by performing one or more of the following
- * actions, as appropriate: merging RMWR and RMWC/RMW actions, stepping forward
- * in the NodeStack, manipulating backtracking sets, allocating and
+ * actions, as appropriate: merging RMWR and RMWC/RMW actions,
+ * manipulating backtracking sets, allocating and
  * initializing clock vectors, and computing the promises to fulfill.
  *
  * @param curr The current action, as passed from the user context; may be
@@ -540,38 +592,16 @@ bool ModelExecution::process_thread_action(ModelAction *curr)
  */
 bool ModelExecution::initialize_curr_action(ModelAction **curr)
 {
-       ModelAction *newcurr;
-
        if ((*curr)->is_rmwc() || (*curr)->is_rmw()) {
-               newcurr = process_rmw(*curr);
+               ModelAction *newcurr = process_rmw(*curr);
                delete *curr;
 
                *curr = newcurr;
                return false;
-       }
-
-       (*curr)->set_seq_number(get_next_seq_num());
-
-       newcurr = node_stack->explore_action(*curr);
-       if (newcurr) {
-               /* First restore type and order in case of RMW operation */
-               if ((*curr)->is_rmwr())
-                       newcurr->copy_typeandorder(*curr);
-
-               ASSERT((*curr)->get_location() == newcurr->get_location());
-               newcurr->copy_from_new(*curr);
-
-               /* Discard duplicate ModelAction; use action from NodeStack */
-               delete *curr;
-
-               /* Always compute new clock vector */
-               newcurr->create_cv(get_parent_action(newcurr->get_tid()));
-
-               *curr = newcurr;
-               return false;   /* Action was explored previously */
        } else {
-               newcurr = *curr;
+               ModelAction *newcurr = *curr;
 
+               newcurr->set_seq_number(get_next_seq_num());
                /* Always compute new clock vector */
                newcurr->create_cv(get_parent_action(newcurr->get_tid()));
 
@@ -594,22 +624,18 @@ bool ModelExecution::initialize_curr_action(ModelAction **curr)
  * @return True if this read established synchronization
  */
 
-bool ModelExecution::read_from(ModelAction *act, const ModelAction *rf)
+void ModelExecution::read_from(ModelAction *act, ModelAction *rf)
 {
        ASSERT(rf);
        ASSERT(rf->is_write());
 
        act->set_read_from(rf);
        if (act->is_acquire()) {
-               rel_heads_list_t release_heads;
-               get_release_seq_heads(act, act, &release_heads);
-               int num_heads = release_heads.size();
-               for (unsigned int i = 0;i < release_heads.size();i++)
-                       if (!synchronize(release_heads[i], act))
-                               num_heads--;
-               return num_heads > 0;
+               ClockVector *cv = get_hb_from_write(rf);
+               if (cv == NULL)
+                       return;
+               act->get_cv()->merge(cv);
        }
-       return false;
 }
 
 /**
@@ -627,7 +653,7 @@ bool ModelExecution::read_from(ModelAction *act, const ModelAction *rf)
 bool ModelExecution::synchronize(const ModelAction *first, ModelAction *second)
 {
        if (*second < *first) {
-               set_bad_synchronization();
+               ASSERT(0);      //This should not happend
                return false;
        }
        return second->synchronize_with(first);
@@ -655,6 +681,9 @@ bool ModelExecution::check_action_enabled(ModelAction *curr) {
                if (!blocking->is_complete()) {
                        return false;
                }
+       } else if (curr->is_sleep()) {
+               if (!fuzzer->shouldSleep(curr))
+                       return false;
        }
 
        return true;
@@ -681,23 +710,29 @@ ModelAction * ModelExecution::check_current_action(ModelAction *curr)
 
        wake_up_sleeping_actions(curr);
 
-       /* Add the action to lists before any other model-checking tasks */
+       /* Add uninitialized actions to lists */
        if (!second_part_of_rmw)
-               add_action_to_lists(curr);
+               add_uninit_action_to_lists(curr);
 
        SnapVector<ModelAction *> * rf_set = NULL;
        /* Build may_read_from set for newly-created actions */
        if (newly_explored && curr->is_read())
                rf_set = build_may_read_from(curr);
 
-       process_thread_action(curr);
-
        if (curr->is_read() && !second_part_of_rmw) {
                process_read(curr, rf_set);
                delete rf_set;
-       } else {
+       } else
                ASSERT(rf_set == NULL);
-       }
+
+       /* Add the action to lists */
+       if (!second_part_of_rmw)
+               add_action_to_lists(curr);
+
+       if (curr->is_write())
+               add_write_to_lists(curr);
+
+       process_thread_action(curr);
 
        if (curr->is_write())
                process_write(curr);
@@ -711,52 +746,12 @@ ModelAction * ModelExecution::check_current_action(ModelAction *curr)
        return curr;
 }
 
-/**
- * This is the strongest feasibility check available.
- * @return whether the current trace (partial or complete) must be a prefix of
- * a feasible trace.
- */
-bool ModelExecution::isfeasibleprefix() const
-{
-       return !is_infeasible();
-}
-
-/**
- * Print disagnostic information about an infeasible execution
- * @param prefix A string to prefix the output with; if NULL, then a default
- * message prefix will be provided
- */
-void ModelExecution::print_infeasibility(const char *prefix) const
-{
-       char buf[100];
-       char *ptr = buf;
-       if (mo_graph->checkForCycles())
-               ptr += sprintf(ptr, "[mo cycle]");
-       if (priv->bad_synchronization)
-               ptr += sprintf(ptr, "[bad sw ordering]");
-       if (ptr != buf)
-               model_print("%s: %s", prefix ? prefix : "Infeasible", buf);
-}
-
-/**
- * Check if the current partial trace is infeasible. Does not check any
- * end-of-execution flags, which might rule out the execution. Thus, this is
- * useful only for ruling an execution as infeasible.
- * @return whether the current partial trace is infeasible.
- */
-bool ModelExecution::is_infeasible() const
-{
-       return mo_graph->checkForCycles() ||
-                                priv->bad_synchronization;
-}
-
 /** Close out a RMWR by converting previous RMWR into a RMW or READ. */
 ModelAction * ModelExecution::process_rmw(ModelAction *act) {
        ModelAction *lastread = get_last_action(act->get_tid());
        lastread->process_rmw(act);
        if (act->is_rmw()) {
                mo_graph->addRMWEdge(lastread->get_reads_from(), lastread);
-               mo_graph->commitChanges();
        }
        return lastread;
 }
@@ -775,10 +770,14 @@ ModelAction * ModelExecution::process_rmw(ModelAction *act) {
  *
  * @param curr The current action. Must be a read.
  * @param rf The ModelAction or Promise that curr reads from. Must be a write.
+ * @param check_only If true, then only check whether the current action satisfies
+ *        read modification order or not, without modifiying priorset and canprune.
+ *        False by default.
  * @return True if modification order edges were added; false otherwise
  */
 
-bool ModelExecution::r_modification_order(ModelAction *curr, const ModelAction *rf, SnapVector<ModelAction *> * priorset)
+bool ModelExecution::r_modification_order(ModelAction *curr, const ModelAction *rf,
+                                                                                                                                                                       SnapVector<const ModelAction *> * priorset, bool * canprune, bool check_only)
 {
        SnapVector<action_list_t> *thrd_lists = obj_thrd_map.get(curr->get_location());
        unsigned int i;
@@ -786,27 +785,33 @@ bool ModelExecution::r_modification_order(ModelAction *curr, const ModelAction *
 
        /* Last SC fence in the current thread */
        ModelAction *last_sc_fence_local = get_last_seq_cst_fence(curr->get_tid(), NULL);
-       ModelAction *last_sc_write = NULL;
-       if (curr->is_seqcst())
-               last_sc_write = get_last_seq_cst_write(curr);
 
+       int tid = curr->get_tid();
+       ModelAction *prev_same_thread = NULL;
        /* Iterate over all threads */
-       for (i = 0;i < thrd_lists->size();i++) {
-               /* Last SC fence in thread i */
+       for (i = 0;i < thrd_lists->size();i++, tid = (((unsigned int)(tid+1)) == thrd_lists->size()) ? 0 : tid + 1) {
+               /* Last SC fence in thread tid */
                ModelAction *last_sc_fence_thread_local = NULL;
-               if (int_to_id((int)i) != curr->get_tid())
-                       last_sc_fence_thread_local = get_last_seq_cst_fence(int_to_id(i), NULL);
+               if (i != 0)
+                       last_sc_fence_thread_local = get_last_seq_cst_fence(int_to_id(tid), NULL);
 
-               /* Last SC fence in thread i, before last SC fence in current thread */
+               /* Last SC fence in thread tid, before last SC fence in current thread */
                ModelAction *last_sc_fence_thread_before = NULL;
                if (last_sc_fence_local)
-                       last_sc_fence_thread_before = get_last_seq_cst_fence(int_to_id(i), last_sc_fence_local);
+                       last_sc_fence_thread_before = get_last_seq_cst_fence(int_to_id(tid), last_sc_fence_local);
+
+               //Only need to iterate if either hb has changed for thread in question or SC fence after last operation...
+               if (prev_same_thread != NULL &&
+                               (prev_same_thread->get_cv()->getClock(tid) == curr->get_cv()->getClock(tid)) &&
+                               (last_sc_fence_thread_local == NULL || *last_sc_fence_thread_local < *prev_same_thread)) {
+                       continue;
+               }
 
                /* Iterate over actions in thread, starting from most recent */
-               action_list_t *list = &(*thrd_lists)[i];
-               action_list_t::reverse_iterator rit;
-               for (rit = list->rbegin();rit != list->rend();rit++) {
-                       ModelAction *act = *rit;
+               action_list_t *list = &(*thrd_lists)[tid];
+               sllnode<ModelAction *> * rit;
+               for (rit = list->end();rit != NULL;rit=rit->getPrev()) {
+                       ModelAction *act = rit->getVal();
 
                        /* Skip curr */
                        if (act == curr)
@@ -823,26 +828,29 @@ bool ModelExecution::r_modification_order(ModelAction *curr, const ModelAction *
                                /* C++, Section 29.3 statement 5 */
                                if (curr->is_seqcst() && last_sc_fence_thread_local &&
                                                *act < *last_sc_fence_thread_local) {
-                                 if (mo_graph->checkReachable(rf, act))
-                                   return false;
-                                 priorset->add(act);
+                                       if (mo_graph->checkReachable(rf, act))
+                                               return false;
+                                       if (!check_only)
+                                               priorset->push_back(act);
                                        break;
                                }
                                /* C++, Section 29.3 statement 4 */
                                else if (act->is_seqcst() && last_sc_fence_local &&
                                                                 *act < *last_sc_fence_local) {
-                                 if (mo_graph->checkReachable(rf, act))
-                                   return false;
-                                 priorset->add(act);
-                                 break;
+                                       if (mo_graph->checkReachable(rf, act))
+                                               return false;
+                                       if (!check_only)
+                                               priorset->push_back(act);
+                                       break;
                                }
                                /* C++, Section 29.3 statement 6 */
                                else if (last_sc_fence_thread_before &&
                                                                 *act < *last_sc_fence_thread_before) {
-                                 if (mo_graph->checkReachable(rf, act))
-                                   return false;
-                                 priorset->add(act);
-                                 break;
+                                       if (mo_graph->checkReachable(rf, act))
+                                               return false;
+                                       if (!check_only)
+                                               priorset->push_back(act);
+                                       break;
                                }
                        }
 
@@ -851,16 +859,30 @@ bool ModelExecution::r_modification_order(ModelAction *curr, const ModelAction *
                         * before" curr
                         */
                        if (act->happens_before(curr)) {
+                               if (i==0) {
+                                       if (last_sc_fence_local == NULL ||
+                                                       (*last_sc_fence_local < *act)) {
+                                               prev_same_thread = act;
+                                       }
+                               }
                                if (act->is_write()) {
-                                 if (mo_graph->checkReachable(rf, act))
-                                   return false;
-                                 priorset->add(act);
+                                       if (mo_graph->checkReachable(rf, act))
+                                               return false;
+                                       if (!check_only)
+                                               priorset->push_back(act);
                                } else {
                                        const ModelAction *prevrf = act->get_reads_from();
                                        if (!prevrf->equals(rf)) {
-                                         if (mo_graph->checkReachable(rf, prevrf))
-                                           return false;
-                                         priorset->add(prevrf);
+                                               if (mo_graph->checkReachable(rf, prevrf))
+                                                       return false;
+                                               if (!check_only)
+                                                       priorset->push_back(prevrf);
+                                       } else {
+                                               if (act->get_tid() == curr->get_tid()) {
+                                                       //Can prune curr from obj list
+                                                       if (!check_only)
+                                                               *canprune = true;
+                                               }
                                        }
                                }
                                break;
@@ -900,13 +922,17 @@ void ModelExecution::w_modification_order(ModelAction *curr)
        unsigned int i;
        ASSERT(curr->is_write());
 
+       SnapList<ModelAction *> edgeset;
+
        if (curr->is_seqcst()) {
                /* We have to at least see the last sequentially consistent write,
                         so we are initialized. */
                ModelAction *last_seq_cst = get_last_seq_cst_write(curr);
                if (last_seq_cst != NULL) {
-                       mo_graph->addEdge(last_seq_cst, curr);
+                       edgeset.push_back(last_seq_cst);
                }
+               //update map for next query
+               obj_last_sc_map.put(curr->get_location(), curr);
        }
 
        /* Last SC fence in the current thread */
@@ -921,9 +947,9 @@ void ModelExecution::w_modification_order(ModelAction *curr)
 
                /* Iterate over actions in thread, starting from most recent */
                action_list_t *list = &(*thrd_lists)[i];
-               action_list_t::reverse_iterator rit;
-               for (rit = list->rbegin();rit != list->rend();rit++) {
-                       ModelAction *act = *rit;
+               sllnode<ModelAction*>* rit;
+               for (rit = list->end();rit != NULL;rit=rit->getPrev()) {
+                       ModelAction *act = rit->getVal();
                        if (act == curr) {
                                /*
                                 * 1) If RMW and it actually read from something, then we
@@ -948,7 +974,7 @@ void ModelExecution::w_modification_order(ModelAction *curr)
                        /* C++, Section 29.3 statement 7 */
                        if (last_sc_fence_thread_before && act->is_write() &&
                                        *act < *last_sc_fence_thread_before) {
-                               mo_graph->addEdge(act, curr);
+                               edgeset.push_back(act);
                                break;
                        }
 
@@ -964,30 +990,17 @@ void ModelExecution::w_modification_order(ModelAction *curr)
                                 *   readfrom(act) --mo--> act
                                 */
                                if (act->is_write())
-                                       mo_graph->addEdge(act, curr);
+                                       edgeset.push_back(act);
                                else if (act->is_read()) {
                                        //if previous read accessed a null, just keep going
-                                       mo_graph->addEdge(act->get_reads_from(), curr);
+                                       edgeset.push_back(act->get_reads_from());
                                }
                                break;
-                       } else if (act->is_read() && !act->could_synchronize_with(curr) &&
-                                                                !act->same_thread(curr)) {
-                               /* We have an action that:
-                                  (1) did not happen before us
-                                  (2) is a read and we are a write
-                                  (3) cannot synchronize with us
-                                  (4) is in a different thread
-                                  =>
-                                  that read could potentially read from our write.  Note that
-                                  these checks are overly conservative at this point, we'll
-                                  do more checks before actually removing the
-                                  pendingfuturevalue.
-
-                                */
-
                        }
                }
        }
+       mo_graph->addEdges(&edgeset, curr);
+
 }
 
 /**
@@ -1008,9 +1021,9 @@ bool ModelExecution::mo_may_allow(const ModelAction *writer, const ModelAction *
 
                /* Iterate over actions in thread, starting from most recent */
                action_list_t *list = &(*thrd_lists)[i];
-               action_list_t::reverse_iterator rit;
-               for (rit = list->rbegin();rit != list->rend();rit++) {
-                       ModelAction *act = *rit;
+               sllnode<ModelAction *>* rit;
+               for (rit = list->end();rit != NULL;rit=rit->getPrev()) {
+                       ModelAction *act = rit->getVal();
 
                        /* Don't disallow due to act == reader */
                        if (!reader->happens_before(act) || reader == act)
@@ -1028,97 +1041,111 @@ bool ModelExecution::mo_may_allow(const ModelAction *writer, const ModelAction *
 }
 
 /**
- * Finds the head(s) of the release sequence(s) containing a given ModelAction.
- * The ModelAction under consideration is expected to be taking part in
- * release/acquire synchronization as an object of the "reads from" relation.
- * Note that this can only provide release sequence support for RMW chains
- * which do not read from the future, as those actions cannot be traced until
- * their "promise" is fulfilled. Similarly, we may not even establish the
- * presence of a release sequence with certainty, as some modification order
- * constraints may be decided further in the future. Thus, this function
- * "returns" two pieces of data: a pass-by-reference vector of @a release_heads
- * and a boolean representing certainty.
+ * Computes the clock vector that happens before propagates from this write.
  *
  * @param rf The action that might be part of a release sequence. Must be a
  * write.
- * @param release_heads A pass-by-reference style return parameter. After
- * execution of this function, release_heads will contain the heads of all the
- * relevant release sequences, if any exists with certainty
- * @return true, if the ModelExecution is certain that release_heads is complete;
- * false otherwise
+ * @return ClockVector of happens before relation.
  */
-bool ModelExecution::release_seq_heads(const ModelAction *rf,
-                                                                                                                                                        rel_heads_list_t *release_heads) const
-{
-       /* Only check for release sequences if there are no cycles */
-       if (mo_graph->checkForCycles())
-               return false;
 
+ClockVector * ModelExecution::get_hb_from_write(ModelAction *rf) const {
+       SnapVector<ModelAction *> * processset = NULL;
        for ( ;rf != NULL;rf = rf->get_reads_from()) {
                ASSERT(rf->is_write());
+               if (!rf->is_rmw() || (rf->is_acquire() && rf->is_release()) || rf->get_rfcv() != NULL)
+                       break;
+               if (processset == NULL)
+                       processset = new SnapVector<ModelAction *>();
+               processset->push_back(rf);
+       }
 
-               if (rf->is_release())
-                       release_heads->push_back(rf);
-               else if (rf->get_last_fence_release())
-                       release_heads->push_back(rf->get_last_fence_release());
-               if (!rf->is_rmw())
-                       break;/* End of RMW chain */
-
-               /** @todo Need to be smarter here...  In the linux lock
-                * example, this will run to the beginning of the program for
-                * every acquire. */
-               /** @todo The way to be smarter here is to keep going until 1
-                * thread has a release preceded by an acquire and you've seen
-                *       both. */
-
-               /* acq_rel RMW is a sufficient stopping condition */
-               if (rf->is_acquire() && rf->is_release())
-                       return true;/* complete */
-       };
-       ASSERT(rf);     // Needs to be real write
-
-       if (rf->is_release())
-               return true;/* complete */
-
-       /* else relaxed write
-        * - check for fence-release in the same thread (29.8, stmt. 3)
-        * - check modification order for contiguous subsequence
-        *   -> rf must be same thread as release */
-
-       const ModelAction *fence_release = rf->get_last_fence_release();
-       /* Synchronize with a fence-release unconditionally; we don't need to
-        * find any more "contiguous subsequence..." for it */
-       if (fence_release)
-               release_heads->push_back(fence_release);
-
-       return true;    /* complete */
+       int i = (processset == NULL) ? 0 : processset->size();
+
+       ClockVector * vec = NULL;
+       while(true) {
+               if (rf->get_rfcv() != NULL) {
+                       vec = rf->get_rfcv();
+               } else if (rf->is_acquire() && rf->is_release()) {
+                       vec = rf->get_cv();
+               } else if (rf->is_release() && !rf->is_rmw()) {
+                       vec = rf->get_cv();
+               } else if (rf->is_release()) {
+                       //have rmw that is release and doesn't have a rfcv
+                       (vec = new ClockVector(vec, NULL))->merge(rf->get_cv());
+                       rf->set_rfcv(vec);
+               } else {
+                       //operation that isn't release
+                       if (rf->get_last_fence_release()) {
+                               if (vec == NULL)
+                                       vec = rf->get_last_fence_release()->get_cv();
+                               else
+                                       (vec=new ClockVector(vec, NULL))->merge(rf->get_last_fence_release()->get_cv());
+                       }
+                       rf->set_rfcv(vec);
+               }
+               i--;
+               if (i >= 0) {
+                       rf = (*processset)[i];
+               } else
+                       break;
+       }
+       if (processset != NULL)
+               delete processset;
+       return vec;
 }
 
 /**
- * An interface for getting the release sequence head(s) with which a
- * given ModelAction must synchronize. This function only returns a non-empty
- * result when it can locate a release sequence head with certainty. Otherwise,
- * it may mark the internal state of the ModelExecution so that it will handle
- * the release sequence at a later time, causing @a acquire to update its
- * synchronization at some later point in execution.
+ * Performs various bookkeeping operations for the current ModelAction when it is
+ * the first atomic action occurred at its memory location.
  *
- * @param acquire The 'acquire' action that may synchronize with a release
- * sequence
- * @param read The read action that may read from a release sequence; this may
- * be the same as acquire, or else an earlier action in the same thread (i.e.,
- * when 'acquire' is a fence-acquire)
- * @param release_heads A pass-by-reference return parameter. Will be filled
- * with the head(s) of the release sequence(s), if they exists with certainty.
- * @see ModelExecution::release_seq_heads
+ * For instance, adds uninitialized action to the per-object, per-thread action vector
+ * and to the action trace list of all thread actions.
+ *
+ * @param act is the ModelAction to process.
  */
-void ModelExecution::get_release_seq_heads(ModelAction *acquire,
-                                                                                                                                                                        ModelAction *read, rel_heads_list_t *release_heads)
+void ModelExecution::add_uninit_action_to_lists(ModelAction *act)
 {
-       const ModelAction *rf = read->get_reads_from();
+       int tid = id_to_int(act->get_tid());
+       ModelAction *uninit = NULL;
+       int uninit_id = -1;
+       SnapVector<action_list_t> *objvec = get_safe_ptr_vect_action(&obj_thrd_map, act->get_location());
+       if (objvec->empty() && act->is_atomic_var()) {
+               uninit = get_uninitialized_action(act);
+               uninit_id = id_to_int(uninit->get_tid());
+               SnapVector<action_list_t> *vec = get_safe_ptr_vect_action(&obj_wr_thrd_map, act->get_location());
+               if ((int)vec->size() <= uninit_id) {
+                       int oldsize = (int) vec->size();
+                       vec->resize(uninit_id + 1);
+                       for(int i = oldsize;i < uninit_id + 1;i++) {
+                               new (&(*vec)[i]) action_list_t();
+                       }
+               }
+               (*vec)[uninit_id].push_front(uninit);
+       }
 
-       release_seq_heads(rf, release_heads);
+       // Update action trace, a total order of all actions
+       if (uninit)
+               action_trace.push_front(uninit);
+
+       // Update obj_thrd_map, a per location, per thread, order of actions
+       SnapVector<action_list_t> *vec = get_safe_ptr_vect_action(&obj_thrd_map, act->get_location());
+       if ((int)vec->size() <= tid) {
+               uint oldsize = vec->size();
+               vec->resize(priv->next_thread_id);
+               for(uint i = oldsize;i < priv->next_thread_id;i++)
+                       new (&(*vec)[i]) action_list_t();
+       }
+       if (uninit)
+               (*vec)[uninit_id].push_front(uninit);
+
+       // Update thrd_last_action, the last action taken by each thrad
+       if ((int)thrd_last_action.size() <= tid)
+               thrd_last_action.resize(get_num_threads());
+       if (uninit)
+               thrd_last_action[uninit_id] = uninit;
 }
 
+
 /**
  * Performs various bookkeeping operations for the current ModelAction. For
  * instance, adds action to the per-object, per-thread action vector and to the
@@ -1129,33 +1156,30 @@ void ModelExecution::get_release_seq_heads(ModelAction *acquire,
 void ModelExecution::add_action_to_lists(ModelAction *act)
 {
        int tid = id_to_int(act->get_tid());
-       ModelAction *uninit = NULL;
-       int uninit_id = -1;
-       action_list_t *list = get_safe_ptr_action(&obj_map, act->get_location());
-       if (list->empty() && act->is_atomic_var()) {
-               uninit = get_uninitialized_action(act);
-               uninit_id = id_to_int(uninit->get_tid());
-               list->push_front(uninit);
+       if ((act->is_fence() && act->is_seqcst()) || act->is_unlock()) {
+         action_list_t *list = get_safe_ptr_action(&obj_map, act->get_location());
+         list->push_back(act);
        }
-       list->push_back(act);
 
+       // Update action trace, a total order of all actions
        action_trace.push_back(act);
-       if (uninit)
-               action_trace.push_front(uninit);
 
+       // Update obj_thrd_map, a per location, per thread, order of actions
        SnapVector<action_list_t> *vec = get_safe_ptr_vect_action(&obj_thrd_map, act->get_location());
-       if (tid >= (int)vec->size())
+       if ((int)vec->size() <= tid) {
+               uint oldsize = vec->size();
                vec->resize(priv->next_thread_id);
+               for(uint i = oldsize;i < priv->next_thread_id;i++)
+                       new (&(*vec)[i]) action_list_t();
+       }
        (*vec)[tid].push_back(act);
-       if (uninit)
-               (*vec)[uninit_id].push_front(uninit);
 
+       // Update thrd_last_action, the last action taken by each thrad
        if ((int)thrd_last_action.size() <= tid)
                thrd_last_action.resize(get_num_threads());
        thrd_last_action[tid] = act;
-       if (uninit)
-               thrd_last_action[uninit_id] = uninit;
 
+       // Update thrd_last_fence_release, the last release fence taken by each thread
        if (act->is_fence() && act->is_release()) {
                if ((int)thrd_last_fence_release.size() <= tid)
                        thrd_last_fence_release.resize(get_num_threads());
@@ -1167,12 +1191,91 @@ void ModelExecution::add_action_to_lists(ModelAction *act)
                get_safe_ptr_action(&obj_map, mutex_loc)->push_back(act);
 
                SnapVector<action_list_t> *vec = get_safe_ptr_vect_action(&obj_thrd_map, mutex_loc);
-               if (tid >= (int)vec->size())
+               if ((int)vec->size() <= tid) {
+                       uint oldsize = vec->size();
                        vec->resize(priv->next_thread_id);
+                       for(uint i = oldsize;i < priv->next_thread_id;i++)
+                               new (&(*vec)[i]) action_list_t();
+               }
                (*vec)[tid].push_back(act);
        }
 }
 
+void insertIntoActionList(action_list_t *list, ModelAction *act) {
+       sllnode<ModelAction*> * rit = list->end();
+       modelclock_t next_seq = act->get_seq_number();
+       if (rit == NULL || (rit->getVal()->get_seq_number() == next_seq))
+               list->push_back(act);
+       else {
+               for(;rit != NULL;rit=rit->getPrev()) {
+                       if (rit->getVal()->get_seq_number() == next_seq) {
+                               list->insertAfter(rit, act);
+                               break;
+                       }
+               }
+       }
+}
+
+void insertIntoActionListAndSetCV(action_list_t *list, ModelAction *act) {
+       sllnode<ModelAction*> * rit = list->end();
+       modelclock_t next_seq = act->get_seq_number();
+       if (rit == NULL) {
+               act->create_cv(NULL);
+       } else if (rit->getVal()->get_seq_number() == next_seq) {
+               act->create_cv(rit->getVal());
+               list->push_back(act);
+       } else {
+               for(;rit != NULL;rit=rit->getPrev()) {
+                       if (rit->getVal()->get_seq_number() == next_seq) {
+                               act->create_cv(rit->getVal());
+                               list->insertAfter(rit, act);
+                               break;
+                       }
+               }
+       }
+}
+
+/**
+ * Performs various bookkeeping operations for a normal write.  The
+ * complication is that we are typically inserting a normal write
+ * lazily, so we need to insert it into the middle of lists.
+ *
+ * @param act is the ModelAction to add.
+ */
+
+void ModelExecution::add_normal_write_to_lists(ModelAction *act)
+{
+       int tid = id_to_int(act->get_tid());
+       insertIntoActionListAndSetCV(&action_trace, act);
+
+       // Update obj_thrd_map, a per location, per thread, order of actions
+       SnapVector<action_list_t> *vec = get_safe_ptr_vect_action(&obj_thrd_map, act->get_location());
+       if (tid >= (int)vec->size()) {
+               uint oldsize =vec->size();
+               vec->resize(priv->next_thread_id);
+               for(uint i=oldsize;i<priv->next_thread_id;i++)
+                       new (&(*vec)[i]) action_list_t();
+       }
+       insertIntoActionList(&(*vec)[tid],act);
+
+       // Update thrd_last_action, the last action taken by each thrad
+       if (thrd_last_action[tid]->get_seq_number() == act->get_seq_number())
+               thrd_last_action[tid] = act;
+}
+
+
+void ModelExecution::add_write_to_lists(ModelAction *write) {
+       SnapVector<action_list_t> *vec = get_safe_ptr_vect_action(&obj_wr_thrd_map, write->get_location());
+       int tid = id_to_int(write->get_tid());
+       if (tid >= (int)vec->size()) {
+               uint oldsize =vec->size();
+               vec->resize(priv->next_thread_id);
+               for(uint i=oldsize;i<priv->next_thread_id;i++)
+                       new (&(*vec)[i]) action_list_t();
+       }
+       (*vec)[tid].push_back(write);
+}
+
 /**
  * @brief Get the last action performed by a particular Thread
  * @param tid The thread ID of the Thread in question
@@ -1212,16 +1315,7 @@ ModelAction * ModelExecution::get_last_fence_release(thread_id_t tid) const
 ModelAction * ModelExecution::get_last_seq_cst_write(ModelAction *curr) const
 {
        void *location = curr->get_location();
-       action_list_t *list = obj_map.get(location);
-       /* Find: max({i in dom(S) | seq_cst(t_i) && isWrite(t_i) && samevar(t_i, t)}) */
-       action_list_t::reverse_iterator rit;
-       for (rit = list->rbegin();(*rit) != curr;rit++)
-               ;
-       rit++;  /* Skip past curr */
-       for ( ;rit != list->rend();rit++)
-               if ((*rit)->is_write() && (*rit)->is_seqcst())
-                       return *rit;
-       return NULL;
+       return obj_last_sc_map.get(location);
 }
 
 /**
@@ -1240,20 +1334,22 @@ ModelAction * ModelExecution::get_last_seq_cst_fence(thread_id_t tid, const Mode
        if (!list)
                return NULL;
 
-       action_list_t::reverse_iterator rit = list->rbegin();
+       sllnode<ModelAction*>* rit = list->end();
 
        if (before_fence) {
-               for (;rit != list->rend();rit++)
-                       if (*rit == before_fence)
+               for (;rit != NULL;rit=rit->getPrev())
+                       if (rit->getVal() == before_fence)
                                break;
 
-               ASSERT(*rit == before_fence);
-               rit++;
+               ASSERT(rit->getVal() == before_fence);
+               rit=rit->getPrev();
        }
 
-       for (;rit != list->rend();rit++)
-               if ((*rit)->is_fence() && (tid == (*rit)->get_tid()) && (*rit)->is_seqcst())
-                       return *rit;
+       for (;rit != NULL;rit=rit->getPrev()) {
+               ModelAction *act = rit->getVal();
+               if (act->is_fence() && (tid == act->get_tid()) && act->is_seqcst())
+                       return act;
+       }
        return NULL;
 }
 
@@ -1271,10 +1367,10 @@ ModelAction * ModelExecution::get_last_unlock(ModelAction *curr) const
 
        action_list_t *list = obj_map.get(location);
        /* Find: max({i in dom(S) | isUnlock(t_i) && samevar(t_i, t)}) */
-       action_list_t::reverse_iterator rit;
-       for (rit = list->rbegin();rit != list->rend();rit++)
-               if ((*rit)->is_unlock() || (*rit)->is_wait())
-                       return *rit;
+       sllnode<ModelAction*>* rit;
+       for (rit = list->end();rit != NULL;rit=rit->getPrev())
+               if (rit->getVal()->is_unlock() || rit->getVal()->is_wait())
+                       return rit->getVal();
        return NULL;
 }
 
@@ -1293,7 +1389,8 @@ ModelAction * ModelExecution::get_parent_action(thread_id_t tid) const
  */
 ClockVector * ModelExecution::get_cv(thread_id_t tid) const
 {
-       return get_parent_action(tid)->get_cv();
+       ModelAction *firstaction=get_parent_action(tid);
+       return firstaction != NULL ? firstaction->get_cv() : NULL;
 }
 
 bool valequals(uint64_t val1, uint64_t val2, int size) {
@@ -1321,7 +1418,7 @@ bool valequals(uint64_t val1, uint64_t val2, int size) {
  */
 SnapVector<ModelAction *> *  ModelExecution::build_may_read_from(ModelAction *curr)
 {
-       SnapVector<action_list_t> *thrd_lists = obj_thrd_map.get(curr->get_location());
+       SnapVector<action_list_t> *thrd_lists = obj_wr_thrd_map.get(curr->get_location());
        unsigned int i;
        ASSERT(curr->is_read());
 
@@ -1331,18 +1428,16 @@ SnapVector<ModelAction *> *  ModelExecution::build_may_read_from(ModelAction *cu
                last_sc_write = get_last_seq_cst_write(curr);
 
        SnapVector<ModelAction *> * rf_set = new SnapVector<ModelAction *>();
-       SnapVector<ModelAction *> * priorset = new SnapVector<ModelAction *>();
 
        /* Iterate over all threads */
        for (i = 0;i < thrd_lists->size();i++) {
                /* Iterate over actions in thread, starting from most recent */
                action_list_t *list = &(*thrd_lists)[i];
-               action_list_t::reverse_iterator rit;
-               for (rit = list->rbegin();rit != list->rend();rit++) {
-                       ModelAction *act = *rit;
+               sllnode<ModelAction *> * rit;
+               for (rit = list->end();rit != NULL;rit=rit->getPrev()) {
+                       ModelAction *act = rit->getVal();
 
-                       /* Only consider 'write' actions */
-                       if (!act->is_write() || act == curr)
+                       if (act == curr)
                                continue;
 
                        /* Don't consider more than one seq_cst write if we are a seq_cst read. */
@@ -1367,11 +1462,7 @@ SnapVector<ModelAction *> *  ModelExecution::build_may_read_from(ModelAction *cu
 
                        if (allow_read) {
                                /* Only add feasible reads */
-                               mo_graph->startChanges();
-                               r_modification_order(curr, act);
-                               if (!is_infeasible())
-                                       rf_set->push_back(act);
-                               mo_graph->rollbackChanges();
+                               rf_set->push_back(act);
                        }
 
                        /* Include at most one act per-thread that "happens before" curr */
@@ -1391,26 +1482,25 @@ SnapVector<ModelAction *> *  ModelExecution::build_may_read_from(ModelAction *cu
 /**
  * @brief Get an action representing an uninitialized atomic
  *
- * This function may create a new one or try to retrieve one from the NodeStack
+ * This function may create a new one.
  *
  * @param curr The current action, which prompts the creation of an UNINIT action
  * @return A pointer to the UNINIT ModelAction
  */
-ModelAction * ModelExecution::get_uninitialized_action(const ModelAction *curr) const
+ModelAction * ModelExecution::get_uninitialized_action(ModelAction *curr) const
 {
-       Node *node = curr->get_node();
-       ModelAction *act = node->get_uninit_action();
+       ModelAction *act = curr->get_uninit_action();
        if (!act) {
                act = new ModelAction(ATOMIC_UNINIT, std::memory_order_relaxed, curr->get_location(), params->uninitvalue, model_thread);
-               node->set_uninit_action(act);
+               curr->set_uninit_action(act);
        }
        act->create_cv(NULL);
        return act;
 }
 
-static void print_list(const action_list_t *list)
+static void print_list(action_list_t *list)
 {
-       action_list_t::const_iterator it;
+       sllnode<ModelAction*> *it;
 
        model_print("------------------------------------------------------------------------------------\n");
        model_print("#    t    Action type     MO       Location         Value               Rf  CV\n");
@@ -1418,18 +1508,18 @@ static void print_list(const action_list_t *list)
 
        unsigned int hash = 0;
 
-       for (it = list->begin();it != list->end();it++) {
-               const ModelAction *act = *it;
+       for (it = list->begin();it != NULL;it=it->getNext()) {
+               const ModelAction *act = it->getVal();
                if (act->get_seq_number() > 0)
                        act->print();
-               hash = hash^(hash<<3)^((*it)->hash());
+               hash = hash^(hash<<3)^(it->getVal()->hash());
        }
        model_print("HASH %u\n", hash);
        model_print("------------------------------------------------------------------------------------\n");
 }
 
 #if SUPPORT_MOD_ORDER_DUMP
-void ModelExecution::dumpGraph(char *filename) const
+void ModelExecution::dumpGraph(char *filename)
 {
        char buffer[200];
        sprintf(buffer, "%s.dot", filename);
@@ -1438,8 +1528,8 @@ void ModelExecution::dumpGraph(char *filename) const
        mo_graph->dumpNodes(file);
        ModelAction **thread_array = (ModelAction **)model_calloc(1, sizeof(ModelAction *) * get_num_threads());
 
-       for (action_list_t::const_iterator it = action_trace.begin();it != action_trace.end();it++) {
-               ModelAction *act = *it;
+       for (sllnode<ModelAction*>* it = action_trace.begin();it != NULL;it=it->getNext()) {
+               ModelAction *act = it->getVal();
                if (act->is_read()) {
                        mo_graph->dot_print_node(file, act);
                        mo_graph->dot_print_edge(file,
@@ -1463,7 +1553,7 @@ void ModelExecution::dumpGraph(char *filename) const
 #endif
 
 /** @brief Prints an execution trace summary. */
-void ModelExecution::print_summary() const
+void ModelExecution::print_summary()
 {
 #if SUPPORT_MOD_ORDER_DUMP
        char buffername[100];
@@ -1474,13 +1564,11 @@ void ModelExecution::print_summary() const
 #endif
 
        model_print("Execution trace %d:", get_execution_number());
-       if (isfeasibleprefix()) {
-               if (scheduler->all_threads_sleeping())
-                       model_print(" SLEEP-SET REDUNDANT");
-               if (have_bug_reports())
-                       model_print(" DETECTED BUG(S)");
-       } else
-               print_infeasibility(" INFEASIBLE");
+       if (scheduler->all_threads_sleeping())
+               model_print(" SLEEP-SET REDUNDANT");
+       if (have_bug_reports())
+               model_print(" DETECTED BUG(S)");
+
        model_print("\n");
 
        print_list(&action_trace);
@@ -1576,20 +1664,8 @@ bool ModelExecution::is_enabled(thread_id_t tid) const
 Thread * ModelExecution::action_select_next_thread(const ModelAction *curr) const
 {
        /* Do not split atomic RMW */
-       if (curr->is_rmwr())
+       if (curr->is_rmwr() && !paused_by_fuzzer(curr))
                return get_thread(curr);
-       if (curr->is_write()) {
-               std::memory_order order = curr->get_mo();
-               switch(order) {
-               case std::memory_order_relaxed:
-                       return get_thread(curr);
-               case std::memory_order_release:
-                       return get_thread(curr);
-               default:
-                       return NULL;
-               }
-       }
-
        /* Follow CREATE with the created thread */
        /* which is not needed, because model.cc takes care of this */
        if (curr->get_type() == THREAD_CREATE)
@@ -1600,6 +1676,15 @@ Thread * ModelExecution::action_select_next_thread(const ModelAction *curr) cons
        return NULL;
 }
 
+/** @param act A read atomic action */
+bool ModelExecution::paused_by_fuzzer(const ModelAction * act) const
+{
+       ASSERT(act->is_read());
+
+       // Actions paused by fuzzer have their sequence number reset to 0
+       return act->get_seq_number() == 0;
+}
+
 /**
  * Takes the next step in the execution, if possible.
  * @param curr The current step to take
@@ -1615,6 +1700,9 @@ Thread * ModelExecution::take_step(ModelAction *curr)
        curr = check_current_action(curr);
        ASSERT(curr);
 
+       /* Process this action in ModelHistory for records */
+       model->get_history()->process_action( curr, curr->get_tid() );
+
        if (curr_thrd->is_blocked() || curr_thrd->is_complete())
                scheduler->remove_thread(curr_thrd);