Introducing a separate Scheduler class

SyncLogic almost works...
diff --git a/model/sync-ccnx-wrapper.cc b/model/sync-ccnx-wrapper.cc
index 1b0e5b5..e911e2e 100644
--- a/model/sync-ccnx-wrapper.cc
+++ b/model/sync-ccnx-wrapper.cc
@@ -126,6 +126,7 @@
 int
 CcnxWrapper::publishData (const string &name, const string &dataBuffer, int freshness)
 {
+  // cout << "Publish: " << name << endl;
   ccn_charbuf *pname = ccn_charbuf_create();
   ccn_charbuf *signed_info = ccn_charbuf_create();
   ccn_charbuf *content = ccn_charbuf_create();
diff --git a/model/sync-logic-event-container.h b/model/sync-logic-event-container.h
new file mode 100644
index 0000000..3b99d76
--- /dev/null
+++ b/model/sync-logic-event-container.h
@@ -0,0 +1,74 @@
+/* -*- Mode: C++; c-file-style: "gnu"; indent-tabs-mode:nil -*- */
+/*
+ * Copyright (c) 2012 University of California, Los Angeles
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License version 2 as
+ * published by the Free Software Foundation;
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+ *
+ * Author: Zhenkai Zhu <zhenkai@cs.ucla.edu>
+ *         卞超轶 Chaoyi Bian <bcy@pku.edu.cn>
+ *	   Alexander Afanasyev <alexander.afanasyev@ucla.edu>
+ */
+
+#ifndef SYNC_LOGIC_EVENT_CONTAINER_H
+#define SYNC_LOGIC_EVENT_CONTAINER_H
+
+#include <boost/function.hpp>
+#include <boost/date_time/posix_time/posix_time_types.hpp>
+
+#include <boost/multi_index_container.hpp>
+// #include <boost/multi_index/tag.hpp>
+#include <boost/multi_index/ordered_index.hpp>
+// #include <boost/multi_index/composite_key.hpp>
+// #include <boost/multi_index/hashed_index.hpp>
+// #include <boost/multi_index/random_access_index.hpp>
+#include <boost/multi_index/member.hpp>
+// #include <boost/multi_index/mem_fun.hpp>
+
+namespace mi = boost::multi_index;
+
+namespace Sync
+{
+
+typedef boost::function< void ( ) > Event;
+
+struct LogicEvent
+{
+  LogicEvent (const boost::system_time &_time, Event _event)
+    : time (_time)
+    , event (_event)
+  { }
+  
+  boost::system_time time;
+  Event event;
+};
+
+/**
+ * \ingroup sync
+ * @brief ???
+ */
+struct EventsContainer : public mi::multi_index_container<
+  LogicEvent,
+  mi::indexed_by<
+
+    mi::ordered_non_unique<
+      mi::member<LogicEvent, boost::system_time, &LogicEvent::time>
+      >
+    >
+  >
+{
+};
+
+} // Sync
+
+#endif // SYNC_LOGIC_EVENT_CONTAINER_H
diff --git a/model/sync-logic.cc b/model/sync-logic.cc
index e39b86b..9491a5c 100644
--- a/model/sync-logic.cc
+++ b/model/sync-logic.cc
@@ -35,9 +35,6 @@
 namespace Sync
 {
 
-const boost::posix_time::time_duration SyncLogic::m_delayedCheckTime = boost::posix_time::seconds (4.0);
-
-
 SyncLogic::SyncLogic (const std::string &syncPrefix,
                       LogicUpdateCallback onUpdate,
                       LogicRemoveCallback onRemove,
@@ -46,66 +43,17 @@
   , m_onUpdate (onUpdate)
   , m_onRemove (onRemove)
   , m_ccnxHandle (ccnxHandle)
-  , m_delayedCheckThreadRunning (true)
+  , m_randomGenerator (static_cast<unsigned int> (std::time (0)))
+  , m_rangeUniformRandom (m_randomGenerator, uniform_int<> (20,100))
 {
-  srandom(time(NULL));
   m_ccnxHandle->setInterestFilter (syncPrefix,
                                    bind (&SyncLogic::respondSyncInterest, this, _1));
-
-  m_delayedCheckThread = thread (&SyncLogic::delayedChecksLoop, this);
 }
 
 SyncLogic::~SyncLogic ()
 {
-  m_delayedCheckThreadRunning = false;
-  // cout << "Requested stop" << this_thread::get_id () << endl;
-  m_delayedCheckThread.interrupt ();
-  m_delayedCheckThread.join ();
 }
 
-void
-SyncLogic::delayedChecksLoop ()
-{
-  while (m_delayedCheckThreadRunning)
-    {
-      try
-        {
-          DelayedChecksList::value_type tuple;
-          
-          {
-            unique_lock<mutex> lock (m_listChecksMutex);
-            while (m_delayedCheckThreadRunning && m_listChecks.size () == 0)
-              {
-                m_listChecksCondition.wait (lock);
-                // cout << "Got something" << endl;
-              }
-
-            if (m_listChecks.size () == 0) continue;
-
-            tuple = m_listChecks.front ();
-            m_listChecks.pop_front ();
-            // cout << "pop" << endl;
-            // release the mutex
-          }
-
-          // waiting and calling
-          
-          // cout << "Duration: " << tuple.get<0> () - get_system_time () << endl;
-          this_thread::sleep (tuple.get<0> ()); 
-          
-          if (!m_delayedCheckThreadRunning) continue;
-          tuple.get<1> () (); // call the scheduled function
-        }
-      catch (thread_interrupted e)
-        {
-          // cout << "interrupted: " << this_thread::get_id () << endl;
-          // do nothing
-        }
-    }
-  // cout << "Exited...\n";
-}
-
-
 
 void
 SyncLogic::respondSyncInterest (const string &interest)
@@ -150,19 +98,9 @@
 
   if (!timedProcessing)
     {
-      {
-        // Alex: Should we ignore interests if interest with the same digest is already in the wait queue?
-        
-        lock_guard<mutex> lock (m_listChecksMutex);
-        system_time delay = get_system_time () + m_delayedCheckTime;
-        // do we need randomization??
-        // delay += boost::ptime::milliseconds (rand() % 80 + 20);
+      m_delayedChecksScheduler.schedule (posix_time::milliseconds (m_rangeUniformRandom ()) /*from 20 to 100ms*/,
+                                         bind (&SyncLogic::processSyncInterest, this, digest, interestName, true));
       
-        m_listChecks.push_back (make_tuple (delay,
-                                            bind (&SyncLogic::processSyncInterest, this, digest, interestName, true))
-                                );
-      }
-      m_listChecksCondition.notify_one ();
     }
   else
     {
diff --git a/model/sync-logic.h b/model/sync-logic.h
index 6c112dd..bbf3fad 100644
--- a/model/sync-logic.h
+++ b/model/sync-logic.h
@@ -22,16 +22,17 @@
 
 #ifndef SYNC_LOGIC_H
 #define SYNC_LOGIC_H
+
 #include <boost/shared_ptr.hpp>
-#include <boost/function.hpp>
-#include "boost/date_time/posix_time/posix_time_types.hpp"
 #include <boost/thread/recursive_mutex.hpp>
+#include <boost/random.hpp>
 
 #include "sync-ccnx-wrapper.h"
 #include "sync-interest-table.h"
 #include "sync-diff-state.h"
 #include "sync-full-state.h"
 #include "sync-std-name-info.h"
+#include "sync-scheduler.h"
 
 #include "sync-diff-state-container.h"
 
@@ -84,31 +85,27 @@
    * @brief remove a participant's subtree from the sync tree
    * @param the name prefix for the participant
    */
-	void remove(const std::string &prefix);
+  void remove(const std::string &prefix);
 
 #ifdef _DEBUG
-  size_t
-  getListChecksSize ()
-  {
-    boost::lock_guard<boost::mutex> lock (m_listChecksMutex);
-    return m_listChecks.size ();
-  }
+  Scheduler &
+  getScheduler () { return m_delayedChecksScheduler; }
 #endif
   
 private:
-  void delayedChecksLoop ();
+  void
+  delayedChecksLoop ();
 
   void
   processSyncInterest (DigestConstPtr digest, const std::string &interestname, bool timedProcessing=false);
   
-  void sendSyncInterest ();
+  void
+  sendSyncInterest ();
 
-	void 
-	processPendingSyncInterests(DiffStatePtr &diff);
+  void 
+  processPendingSyncInterests(DiffStatePtr &diff);
 
 private:
-  typedef std::list< boost::tuple< boost::system_time, boost::function< void ( ) > > > DelayedChecksList;
-
   FullState m_state;
   DiffStateContainer m_log;
   boost::recursive_mutex m_stateMutex;
@@ -120,13 +117,11 @@
   LogicRemoveCallback m_onRemove;
   CcnxWrapperPtr m_ccnxHandle;
 
-  boost::thread m_delayedCheckThread;
-  bool          m_delayedCheckThreadRunning;
-  DelayedChecksList m_listChecks;
-  boost::condition_variable m_listChecksCondition;
-  boost::mutex  m_listChecksMutex;
+  Scheduler m_delayedChecksScheduler;
 
-  static const boost::posix_time::time_duration m_delayedCheckTime;
+  boost::mt19937 m_randomGenerator;
+  boost::variate_generator<boost::mt19937&, boost::uniform_int<> > m_rangeUniformRandom;
+  
   static const int m_syncResponseFreshness = 2;
 };
 
diff --git a/model/sync-scheduler.cc b/model/sync-scheduler.cc
new file mode 100644
index 0000000..155e839
--- /dev/null
+++ b/model/sync-scheduler.cc
@@ -0,0 +1,119 @@
+/* -*- Mode: C++; c-file-style: "gnu"; indent-tabs-mode:nil -*- */
+/*
+ * Copyright (c) 2012 University of California, Los Angeles
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License version 2 as
+ * published by the Free Software Foundation;
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+ *
+ * Author: Zhenkai Zhu <zhenkai@cs.ucla.edu>
+ *         卞超轶 Chaoyi Bian <bcy@pku.edu.cn>
+ *	   Alexander Afanasyev <alexander.afanasyev@ucla.edu>
+ */
+
+#include "sync-scheduler.h"
+#include <boost/date_time/posix_time/posix_time.hpp>
+
+using namespace boost;
+using namespace std;
+
+namespace Sync {
+
+Scheduler::Scheduler ()
+  : m_threadRunning (true)
+{
+  m_thread = thread (&Scheduler::threadLoop, this);
+}
+
+Scheduler::~Scheduler ()
+{
+  m_threadRunning = false;
+  // cout << "Requested stop" << this_thread::get_id () << endl;
+  m_thread.interrupt ();
+  m_thread.join ();
+}
+
+void
+Scheduler::threadLoop ()
+{
+  while (m_threadRunning)
+    {
+      try
+        {
+	  boost::system_time nextTime;
+          {
+            unique_lock<mutex> lock (m_eventsMutex);
+            while (m_threadRunning && m_events.size () == 0)
+              {
+                m_eventsCondition.wait (lock);
+                // cout << "Got something" << endl;
+              }
+
+            if (m_events.size () == 0) continue;
+
+	    nextTime = m_events.begin ()->time;
+          }
+
+	  if (nextTime - get_system_time () > posix_time::time_duration (0,0,0,0))
+	    {
+	      this_thread::sleep (nextTime - get_system_time ());
+
+	      // sleeping
+
+	      if (nextTime - get_system_time () > posix_time::time_duration (0,0,0,0))
+		continue; // something changes, try again
+	    }
+
+	  if (!m_threadRunning) continue;
+
+	  Event event;
+	  
+	  {
+	    lock_guard<mutex> lock (m_eventsMutex);
+
+	    BOOST_ASSERT (m_events.size () != 0);
+	    
+	    event = m_events.begin ()->event;
+	    m_events.erase (m_events.begin ());
+	  }
+
+	  event (); // calling the event outside the locked mutex
+        }
+      catch (thread_interrupted e)
+        {
+          // cout << "interrupted: " << this_thread::get_id () << endl;
+          // do nothing
+        }
+    }
+  // cout << "Exited...\n";  
+}
+
+
+void
+Scheduler::schedule (const boost::system_time &abstime, Event event)
+{
+  {
+    lock_guard<mutex> lock (m_eventsMutex);
+    m_events.insert (LogicEvent (abstime, event));
+  }
+  m_eventsCondition.notify_one ();
+  m_thread.interrupt (); // interrupt sleep, if currently sleeping
+}
+
+void
+Scheduler::schedule (const boost::posix_time::time_duration &reltime, Event event)
+{
+  // cout << reltime << endl;
+  schedule (boost::get_system_time () + reltime, event);
+}
+
+} // Sync
diff --git a/model/sync-scheduler.h b/model/sync-scheduler.h
new file mode 100644
index 0000000..7cc23da
--- /dev/null
+++ b/model/sync-scheduler.h
@@ -0,0 +1,92 @@
+/* -*- Mode: C++; c-file-style: "gnu"; indent-tabs-mode:nil -*- */
+/*
+ * Copyright (c) 2012 University of California, Los Angeles
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License version 2 as
+ * published by the Free Software Foundation;
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+ *
+ * Author: Zhenkai Zhu <zhenkai@cs.ucla.edu>
+ *         卞超轶 Chaoyi Bian <bcy@pku.edu.cn>
+ *	   Alexander Afanasyev <alexander.afanasyev@ucla.edu>
+ */
+
+#ifndef SYNC_SCHEDULER_H
+#define SYNC_SCHEDULER_H
+
+#include <boost/thread/thread.hpp>
+#include <boost/thread/mutex.hpp>
+
+#include "sync-logic-event-container.h"
+
+namespace Sync {
+
+/**
+ * @ingroup sync
+ * @brief General purpose event scheduler
+ *
+ * This class internally runs a thread and events can be scheduled by specifying an absolute or relative time of the event
+ */
+class Scheduler
+{
+public:
+  /**
+   * @brief Default constructor. Thread will be created
+   */
+  Scheduler ();
+  /**
+   * @brief Destructor. Thread will be nicely stopped
+   */
+  ~Scheduler ();
+
+  /**
+   * @brief Schedule an event at absolute time 'abstime'
+   * @param abstime Absolute time
+   * @param event function to be called at the time
+   */
+  void
+  schedule (const boost::system_time &abstime, Event event);
+
+  /**
+   * @brief Schedule an event at relative time 'reltime'
+   * @param abstime Relative time
+   * @param event function to be called at the time
+   */
+  void
+  schedule (const boost::posix_time::time_duration &reltime, Event event); 
+
+
+#ifdef _DEBUG
+  size_t
+  getEventsSize ()
+  {
+    boost::lock_guard<boost::mutex> lock (m_eventsMutex);
+    return m_events.size ();
+  }
+#endif
+  
+private:
+  void
+  threadLoop ();
+    
+private:
+  EventsContainer m_events;
+  boost::condition_variable m_eventsCondition;
+  boost::mutex  m_eventsMutex;
+
+  boost::thread m_thread;
+  bool          m_threadRunning;
+};
+  
+}
+
+#endif // SYNC_SCHEDULER_H
diff --git a/test/test_interest_table.cc b/test/test_interest_table.cc
index 3c39046..ee48ccb 100644
--- a/test/test_interest_table.cc
+++ b/test/test_interest_table.cc
@@ -28,7 +28,6 @@
 #include <boost/make_shared.hpp>
 
 #include "../model/sync-interest-table.h"
-#include "../model/sync-logic.h"
 
 using namespace Sync;
 using namespace std;
@@ -42,49 +41,3 @@
   BOOST_CHECK_NO_THROW (delete table);
 }
 
-void funcUpdate (const std::string &, const SeqNo &newSeq, const SeqNo &oldSeq)
-{
-  cout << "funcUpdate\n";
-}
-
-void funcRemove (const std::string &)
-{
-  cout << "funcRemove\n";
-}
-
-BOOST_AUTO_TEST_CASE (SyncLogicTest)
-{  
-  SyncLogic *logic = 0;
-  BOOST_CHECK_NO_THROW (logic = new SyncLogic ("/prefix", funcUpdate, funcRemove, make_shared<CcnxWrapper> ()));
-  BOOST_CHECK_EQUAL (logic->getListChecksSize (), 0);
-
-  // 0s
-  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e")); 
-  sleep (1);
-
-  // 1s
-  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e")); 
-  sleep (1);
-
-  // 2s
-  // cout << "Wait queue size: " << logic->getListChecksSize () << endl;
-  BOOST_CHECK_EQUAL (logic->getListChecksSize (), 1);
-
-  // 2s
-  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e"));  
-  // cout << "Wait queue size: " << logic->getListChecksSize () << endl;
-  BOOST_CHECK_EQUAL (logic->getListChecksSize (), 2);
-
-  this_thread::sleep (posix_time::milliseconds (2500)); // make two interests expire
-
-  // 4.5s
-  // cout << "(after 3.3s) Wait queue size: " << logic->getListChecksSize () << endl;
-  BOOST_CHECK_EQUAL (logic->getListChecksSize (), 1);
-
-  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e"));
-  sleep (5);
-  BOOST_CHECK_EQUAL (logic->getListChecksSize (), 0);
-  
-  BOOST_CHECK_NO_THROW (delete logic);
-}
-
diff --git a/test/test_scheduler.cc b/test/test_scheduler.cc
new file mode 100644
index 0000000..9a72251
--- /dev/null
+++ b/test/test_scheduler.cc
@@ -0,0 +1,153 @@
+/* -*- Mode: C++; c-file-style: "gnu"; indent-tabs-mode:nil -*- */
+/*
+ * Copyright (c) 2012 University of California, Los Angeles
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License version 2 as
+ * published by the Free Software Foundation;
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+ *
+ * Author: Zhenkai Zhu <zhenkai@cs.ucla.edu>
+ *         卞超轶 Chaoyi Bian <bcy@pku.edu.cn>
+ *	   Alexander Afanasyev <alexander.afanasyev@ucla.edu>
+ */
+
+#include <boost/test/unit_test.hpp>
+#include <boost/test/output_test_stream.hpp> 
+#include <map>
+using boost::test_tools::output_test_stream;
+
+#include <boost/make_shared.hpp>
+#include "../model/sync-scheduler.h"
+#include "../model/sync-logic.h"
+
+using namespace Sync;
+using namespace std;
+using namespace boost;
+
+
+
+// void funcUpdate (const std::string &, const SeqNo &newSeq, const SeqNo &oldSeq)
+// {
+//   cout << "funcUpdate\n";
+// }
+
+// void funcRemove (const std::string &)
+// {
+//   cout << "funcRemove\n";
+// }
+
+struct SchedulerFixture
+{
+  SchedulerFixture ()
+    : counter (0)
+  {}
+
+  int counter;
+  
+  Scheduler *scheduler;
+
+  void everySecond ()
+  {
+    // cout << "." << flush;
+    counter ++;
+
+    if (counter < 9)
+    scheduler->schedule (boost::posix_time::milliseconds (100), boost::bind (&SchedulerFixture::everySecond, this));
+  }
+
+  void setCounterFive ()
+  {
+    counter = 5;
+  }
+
+  void setCounterThree ()
+  {
+    counter = 3;
+  }
+};
+
+
+#ifdef _DEBUG
+
+BOOST_FIXTURE_TEST_SUITE (SchedulerTestSuite, SchedulerFixture)
+
+BOOST_AUTO_TEST_CASE (BasicTest)
+{
+  BOOST_CHECK_NO_THROW (scheduler = new Scheduler ());
+
+  scheduler->schedule (posix_time::milliseconds (100), bind (&SchedulerFixture::everySecond, this));
+
+  sleep (1);
+  // cout << counter << endl;
+  BOOST_CHECK_EQUAL (counter, 9); // generally, should be 9
+
+  scheduler->schedule (posix_time::seconds (2),
+		       bind (&SchedulerFixture::setCounterFive, this));
+
+  this_thread::sleep (posix_time::milliseconds (400)); // just in case
+
+  scheduler->schedule (posix_time::milliseconds (600),
+		       bind (&SchedulerFixture::setCounterThree, this));
+
+  this_thread::sleep (posix_time::milliseconds (500));
+  BOOST_CHECK_EQUAL (counter, 9); // still 9
+
+  this_thread::sleep (posix_time::milliseconds (200));
+  BOOST_CHECK_EQUAL (counter, 3);
+
+  this_thread::sleep (posix_time::milliseconds (1000));
+  BOOST_CHECK_EQUAL (counter, 5);
+  
+  BOOST_CHECK_NO_THROW (delete scheduler);
+}
+
+BOOST_AUTO_TEST_SUITE_END ()
+
+
+void funcUpdate( const std::string &/*prefix*/, const SeqNo &/*newSeq*/, const SeqNo &/*oldSeq*/ )
+{
+}
+
+void funcRemove( const std::string &/*prefix*/ )
+{
+}
+
+BOOST_AUTO_TEST_CASE (SyncLogicTest)
+{  
+  SyncLogic *logic = 0;
+  BOOST_CHECK_NO_THROW (logic = new SyncLogic ("/prefix", funcUpdate, funcRemove, make_shared<CcnxWrapper> ()));
+
+  Scheduler &scheduler = logic->getScheduler ();
+  BOOST_CHECK_EQUAL (scheduler.getEventsSize (), 0);
+
+  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e")); 
+  BOOST_CHECK_EQUAL (scheduler.getEventsSize (), 1);
+
+  this_thread::sleep (posix_time::milliseconds (100)); // max waiting time
+  BOOST_CHECK_EQUAL (scheduler.getEventsSize (), 0);
+
+  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e")); 
+  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e")); 
+  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e")); 
+  BOOST_CHECK_NO_THROW (logic->respondSyncInterest ("/prefix/e5fa44f2b31c1fb553b6021e7360d07d5d91ff5e"));
+  BOOST_CHECK_EQUAL (scheduler.getEventsSize (), 4);  
+
+  this_thread::sleep (posix_time::milliseconds (19)); // min waiting time is 20
+  BOOST_CHECK_EQUAL (scheduler.getEventsSize (), 4);  
+
+  this_thread::sleep (posix_time::milliseconds (100)); // max waiting time
+  BOOST_CHECK_EQUAL (scheduler.getEventsSize (), 0);
+  
+  BOOST_CHECK_NO_THROW (delete logic);
+}
+
+#endif