You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ac...@apache.org on 2007/11/01 18:35:19 UTC

svn commit: r591098 - in /incubator/qpid/trunk/qpid/cpp/src: qpid/sys/RefCountedMap.h tests/RefCountedMap.cpp

Author: aconway
Date: Thu Nov  1 10:35:18 2007
New Revision: 591098

URL: http://svn.apache.org/viewvc?rev=591098&view=rev
Log:

Simplified/improved qpid::sys::RefCountedMap, unit test session attachment use case.

Modified:
    incubator/qpid/trunk/qpid/cpp/src/qpid/sys/RefCountedMap.h
    incubator/qpid/trunk/qpid/cpp/src/tests/RefCountedMap.cpp

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/sys/RefCountedMap.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/sys/RefCountedMap.h?rev=591098&r1=591097&r2=591098&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/sys/RefCountedMap.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/sys/RefCountedMap.h Thu Nov  1 10:35:18 2007
@@ -25,9 +25,6 @@
 #include "qpid/sys/Mutex.h"
 #include "qpid/RefCounted.h"
 
-#include <boost/call_traits.hpp>
-#include <boost/iterator/iterator_facade.hpp>
-
 #include <map>
 
 namespace qpid {
@@ -35,134 +32,110 @@
 
 /**
  * A thread-safe, RefCounted map of RefCounted entries.  Entries are
- * automatically erased when all iterators to them are destroyed.  The
- * entire map is released when all its entries are erased.
+ * automatically erased when released The entire map is released when
+ * all its entries are erased.
  *
  * The assumption is that some other object/thread holds an iterator
  * to each entry for as long as it is useful.
  *
- * API is a subset of std::map
+ * The map can be cleared with close()
  *
- * WARNING: Modifying iterators locks the map.  To avoid deadlock, you
- * MUST NOT modify an iterator while holding another lock that could be
- * locked as a result of erasing the entry and destroying its value.
+ * WARNING: Assigning an intrusive_ptr<D> returned by the map locks the
+ * map.  To avoid deadlock, you MUST NOT modify an iterator while
+ * holding another lock that could be locked as a result of erasing
+ * the entry and destroying its value.
  *
+ * @param D must be public RefCounted 
  */
-
-template <class K, class D>
+template <class Key, class Data>
 class RefCountedMap : public RefCounted
 {
-    typedef Mutex::ScopedLock Lock;
-
   public:
-    typedef K key_type;
-    typedef D data_type;
-    typedef std::pair<key_type,data_type> value_type;
-
-    /** Bidirectional iterator maintains a reference count on the map entry.
-     * Provides operator!() and operator bool() to test for end() iterator.
-     */
-    class iterator : 
-        public boost::iterator_facade<iterator, value_type,
-                                      boost::bidirectional_traversal_tag>
-    {
-      public:
-        iterator() {}
-        bool operator!() const { return !ptr; }
-        operator bool() const { return ptr; }
-        void reset() { ptr=0; }
-
-      private:
-        typedef typename RefCountedMap::Entry Entry;
-
-        iterator(intrusive_ptr<Entry> entry) : ptr(entry) {}
-
-        // boost::iterator_facade functions.
-        value_type& dereference() const { return ptr->value; }
-        bool equal(iterator j) const { return ptr==j.ptr; }
-        void increment() { assert(ptr); *this=ptr->map->next(ptr->self); }
-        void decrement() { assert(ptr); *this=ptr->map->prev(ptr->self); }
-
-        intrusive_ptr<Entry> ptr;
-
-      friend class boost::iterator_core_access;
-      friend class RefCountedMap<K,D>;
-    };
-
-    iterator begin() { Lock l(lock); return makeIterator(map.begin()); }
-
-    iterator end() { Lock l(lock); return makeIterator(map.end()); }
-
-    size_t size() { Lock l(lock); return map.size(); }
-
-    bool empty() { return size() == 0u; }
-    
-    iterator find(const key_type& key) {
-        Lock l(lock); return makeIterator(map.find(key));
-    }
-
-    std::pair<iterator, bool> insert(const value_type& x) {
-        Lock l(lock);
-        std::pair<typename Map::iterator,bool> ib=
-            map.insert(make_pair(x.first, Entry(x, this)));
-        ib.first->second.self = ib.first;
-        return std::make_pair(makeIterator(ib.first), ib.second);
-    }
+    typedef intrusive_ptr<Data> DataPtr;
 
   private:
-
-    //
-    // INVARIANT:
-    //  - All entries in the map have non-0 refcounts.
-    //  - Each entry holds an intrusive_ptr to the map 
-    //
-
-    struct Entry : public RefCounted {
-        typedef typename RefCountedMap::Map::iterator Iterator;
-
-        Entry(const value_type& v, RefCountedMap* m) : value(v), map(m) {}
-        
-        value_type value;
+    struct Entry : public Data {
+        typedef typename RefCountedMap::Iterator Iterator;
         intrusive_ptr<RefCountedMap> map;
         Iterator self;
-
-        // RefCounts are modified with map locked. 
-        struct MapLock : public Lock {
-            MapLock(RefCountedMap& m) : Lock(m.lock) {}
-        };
-
+        void init(intrusive_ptr<RefCountedMap> m, Iterator s) {
+            map=m; self=s;
+        }
         void released() const {
-            intrusive_ptr<RefCountedMap> protect(map);
-            map->map.erase(self);
+            if (map) {
+                intrusive_ptr<RefCountedMap> protect(map);
+                map->map.erase(self);
+            }
         }
     };
 
-    typedef std::map<K,Entry> Map;
+    typedef std::map<Key,Entry> Map;
+    typedef typename Map::iterator Iterator;
 
-    iterator makeIterator(typename Map::iterator i) {
-        // Call with lock held.
-        return iterator(i==map.end() ? 0 : &i->second);
-    }
+    typedef Mutex::ScopedLock Lock;
+    struct OpenLock : public Lock {
+        OpenLock(RefCountedMap& m) : Lock(m.lock) { assert(!m.closed); }
+    };
+    
+    DataPtr ptr_(Iterator i) { return i==map.end() ? 0 : &i->second; }
 
-     void erase(typename RefCountedMap::Map::iterator i) {
-        // Make sure this is not deleted till lock is released.
-        intrusive_ptr<RefCountedMap> self(this);
-        { Lock l(lock); map.erase(i); }
-    }
+    Mutex lock;
+    Map map;
+    bool closed;
+    
+  friend struct Entry;
+  friend class iterator;
 
-    iterator next(typename RefCountedMap::Map::iterator i) {
-        { Lock l(lock); return makeIterator(++i); }
+  public:
+    RefCountedMap() : closed(false) {}
+    
+    /** Return 0 if not found
+     * @pre !isClosed()
+     */
+    DataPtr find(const Key& k) {
+        OpenLock l(*this);
+        return ptr_(map.find(k));
     }
 
-    iterator prev(typename RefCountedMap::Map::iterator i) {
-        { Lock l(lock); return makeIterator(--i); }
+    /** Find existing or create new entry for k 
+     * @pre !isClosed()
+     */
+    DataPtr get(const Key& k)  {
+        OpenLock l(*this);
+        std::pair<Iterator,bool> ib=
+            map.insert(std::make_pair(k, Entry()));
+        if (ib.second)
+            ib.first->second.init(this, ib.first);
+        return ptr_(ib.first);
     }
+    
+    size_t size() { Lock l(lock); return map.size(); }
 
-    Mutex lock;
-    Map map;
+    bool empty() { return size() == 0u; }
 
-  friend struct Entry;
-  friend class iterator;
+    bool isClosed() { Lock l(lock); return closed; }
+    
+    /**
+     * Close the map and call functor on each remaining entry.
+     * Note the map will not be deleted until all entries are
+     * released, the assumption is that functor takes some
+     * action that will release the entry.
+     *
+     * close() does nothing if isClosed() 
+     */
+    template <class F>
+    void close(F functor) {
+        Lock l(lock);
+        if (closed) return;
+        closed=true;            // No more inserts
+        intrusive_ptr<RefCountedMap> protect(this);
+        Iterator i=map.begin();
+        while (i != map.end()) {
+            Iterator j=i;
+            ++i;
+            functor(j->second); // May erase j
+        }
+    }
 };
 
 

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/RefCountedMap.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/RefCountedMap.cpp?rev=591098&r1=591097&r2=591098&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/RefCountedMap.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/RefCountedMap.cpp Thu Nov  1 10:35:18 2007
@@ -19,89 +19,105 @@
 #include "qpid/sys/RefCountedMap.h"
 
 #include <boost/test/auto_unit_test.hpp>
+#include <boost/bind.hpp>
+
 BOOST_AUTO_TEST_SUITE(RefCountedMap);
 
 using namespace std;
 using namespace qpid;
 using namespace qpid::sys;
 
-struct TestMap : public  RefCountedMap<int,int> {
+template <int ID> struct CountEm {
     static int instances;
-    TestMap() { ++instances; }
-    ~TestMap() { --instances; }
+    CountEm() { instances++; }
+    ~CountEm() { instances--; }
+    CountEm(const CountEm&) { instances++; }
+};
+template <int ID> int CountEm<ID>::instances = 0;
+    
+struct Data;
+
+struct Attachment : public RefCounted, public CountEm<1> {
+    intrusive_ptr<Data> link;
 };
 
-int TestMap::instances=0;
+struct Data : public RefCounted, public CountEm<2> {
+    intrusive_ptr<Attachment> link;
+    void attach(intrusive_ptr<Attachment> a) {
+        if (!a) return;
+        a->link=this;
+        link=a;
+    }
+    void detach() {
+        if (!link) return;
+        intrusive_ptr<Data> protect(this);
+        link->link=0;
+        link=0;
+    }
+};
+typedef intrusive_ptr<Data> DataPtr;
+
+struct Map : public  RefCountedMap<int,Data>, public CountEm<3> {};
+
+
+
 
 BOOST_AUTO_TEST_CASE(testRefCountedMap) {
-    BOOST_CHECK_EQUAL(0, TestMap::instances);
-    intrusive_ptr<TestMap> map=new TestMap();
-    BOOST_CHECK_EQUAL(1, TestMap::instances);
+    BOOST_CHECK_EQUAL(0, Map::instances);
+    BOOST_CHECK_EQUAL(0, Data::instances);
+
+    intrusive_ptr<Map> map=new Map();
+    BOOST_CHECK_EQUAL(1, Map::instances);
 
     // Empty map
+    BOOST_CHECK(!map->isClosed());
     BOOST_CHECK(map->empty());
     BOOST_CHECK_EQUAL(map->size(), 0u);
-    BOOST_CHECK(map->begin()==map->end());
-    BOOST_CHECK(!map->begin());
-    BOOST_CHECK(!map->end());
-    BOOST_CHECK(map->find(1)==map->end());
     BOOST_CHECK(!map->find(1));
 
     {
-        // Add and modify an entry
-        pair<TestMap::iterator, bool> ib=map->insert(TestMap::value_type(1,11));
-        BOOST_CHECK(ib.second);
-        TestMap::iterator p = ib.first;
-        ib.first.reset();
-        BOOST_CHECK(p);
-        BOOST_CHECK_EQUAL(p->second, 11);
-        p->second=22;
-        BOOST_CHECK_EQUAL(22, map->find(1)->second);
-        BOOST_CHECK(!map->empty());
+        // Add entries
+        DataPtr p=map->get(1);
+        DataPtr q=map->get(2);
+
+        BOOST_CHECK_EQUAL(Data::instances, 2);
+        BOOST_CHECK_EQUAL(map->size(), 2u);
+
+        p=0;                    // verify erased
+        BOOST_CHECK_EQUAL(Data::instances, 1);
         BOOST_CHECK_EQUAL(map->size(), 1u);
 
-        // Find an entry
-        TestMap::iterator q=map->find(1);
+        p=map->find(2);
         BOOST_CHECK(q==p);
-        BOOST_CHECK_EQUAL(q->first, 1);
     }
 
     BOOST_CHECK(map->empty());
-    BOOST_CHECK_EQUAL(1, TestMap::instances); 
+    BOOST_CHECK_EQUAL(1, Map::instances); 
+    BOOST_CHECK_EQUAL(0, Data::instances); 
 
     {
         // Hold the map via a reference to an entry.
-        TestMap::iterator p=map->insert(TestMap::value_type(2,22)).first;
-        map=0;                      // Release the map->
-        BOOST_CHECK_EQUAL(1, TestMap::instances); // Held by entry.
-        BOOST_CHECK_EQUAL(p->second, 22);
+        DataPtr p=map->get(3);
+        map=0;               
+        BOOST_CHECK_EQUAL(1, Map::instances); // Held by entry.
     }
-
-    BOOST_CHECK_EQUAL(0, TestMap::instances); 
+    BOOST_CHECK_EQUAL(0, Map::instances); // entry released
 }
 
 
-BOOST_AUTO_TEST_CASE(testRefCountedMapIterator) {
-    BOOST_CHECK_EQUAL(TestMap::instances, 0);
-    {
-        intrusive_ptr<TestMap> map=new TestMap();
-        TestMap::iterator iter[4], p, q;
-        for (int i = 0; i < 4; ++i) 
-            iter[i] = map->insert(make_pair(i, 10+i)).first;
-        int j=0;
-        for (p = map->begin(); p != map->end(); ++p, ++j)  {
-            BOOST_CHECK_EQUAL(p->first, j);
-            BOOST_CHECK_EQUAL(p->second, 10+j);
-        }
-        BOOST_CHECK_EQUAL(4, j);
-
-        // Release two entries.
-        iter[0]=iter[2]=TestMap::iterator();
-        
-        p=map->begin();
-        BOOST_CHECK_EQUAL(p->second, 11);
-        ++p;
-        BOOST_CHECK_EQUAL(p->second, 13);
-    }
-    BOOST_CHECK_EQUAL(TestMap::instances, 0);
+BOOST_AUTO_TEST_CASE(testRefCountedMapAttachClose) {
+    intrusive_ptr<Map> map=new Map();
+    DataPtr d=map->get(5);
+    d->attach(new Attachment());
+    d=0;
+    // Attachment keeps entry pinned
+    BOOST_CHECK_EQUAL(1u, map->size());
+    BOOST_CHECK(map->find(5));
+
+    // Close breaks attachment
+    map->close(boost::bind(&Data::detach, _1));
+    BOOST_CHECK(map->empty());
+    BOOST_CHECK(map->isClosed());
+    BOOST_CHECK_EQUAL(0, Data::instances);
+    BOOST_CHECK_EQUAL(0, Attachment::instances);
 }