You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ch...@apache.org on 2009/10/22 23:00:21 UTC

svn commit: r828852 - in /activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal: index/BTreeIndex.java page/HawtPageFile.java page/HawtTransaction.java

Author: chirino
Date: Thu Oct 22 21:00:21 2009
New Revision: 828852

URL: http://svn.apache.org/viewvc?rev=828852&view=rev
Log:
Better update merging implemented.

Modified:
    activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/index/BTreeIndex.java
    activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtPageFile.java
    activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtTransaction.java

Modified: activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/index/BTreeIndex.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/index/BTreeIndex.java?rev=828852&r1=828851&r2=828852&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/index/BTreeIndex.java (original)
+++ activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/index/BTreeIndex.java Thu Oct 22 21:00:21 2009
@@ -63,6 +63,11 @@
         this.prefixer = factory.getPrefixer();
     }
     
+    @Override
+    public String toString() {
+        return "{ page: "+page+", deferredEncoding: "+deferredEncoding+" }";
+    }
+    
     public void create() {
         // Store the root page..
         BTreeNode<Key, Value> root = new BTreeNode<Key, Value>(null, page);

Modified: activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtPageFile.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtPageFile.java?rev=828852&r1=828851&r2=828852&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtPageFile.java (original)
+++ activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtPageFile.java Thu Oct 22 21:00:21 2009
@@ -22,6 +22,7 @@
 import java.io.ObjectInputStream;
 import java.io.ObjectOutput;
 import java.io.ObjectOutputStream;
+import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -39,6 +40,7 @@
 import org.apache.activemq.util.buffer.Buffer;
 import org.apache.activemq.util.list.LinkedNode;
 import org.apache.activemq.util.list.LinkedNodeList;
+import org.apache.hawtdb.api.Allocator;
 import org.apache.hawtdb.api.EncoderDecoder;
 import org.apache.hawtdb.api.IOPagingException;
 import org.apache.hawtdb.api.OptimisticUpdateException;
@@ -73,8 +75,6 @@
     private static final String MAGIC = "HawtDB:1.0\n";
     private static final int FILE_HEADER_SIZE = 1024 * 4;
 
-    public static final int PAGE_ALLOCATED = -1;
-    public static final int PAGE_FREED = -2;
     public static final int HEADER_SIZE = 1024*4;
 
     /**
@@ -131,14 +131,14 @@
      * 
      * @author chirino
      */
-    static class Commit extends RedoEntry implements Externalizable {
+    final static class Commit extends RedoEntry implements Externalizable {
 
         /** oldest revision in the commit range. */
         private long base; 
         /** newest revision in the commit range, will match base if this only tracks one commit */ 
         private long head;
         /** all the page updates that are part of the redo */
-        private ConcurrentHashMap<Integer, Integer> updates;
+        private ConcurrentHashMap<Integer, Update> updates;
         /** the deferred updates that need to be done in this redo */
         private ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates;
 
@@ -146,7 +146,7 @@
         public Commit() {
         }
         
-        public Commit(long version, ConcurrentHashMap<Integer, Integer> updates, ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates) {
+        public Commit(long version, ConcurrentHashMap<Integer, Update> updates, ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates) {
             this.head = this.base = version;
             this.updates = updates;
             this.deferredUpdates = deferredUpdates;
@@ -165,7 +165,7 @@
             return "{ base: "+this.base+", head: "+this.head+", updates: "+updateSize+", cache: "+cacheSize+" }";
         }
 
-        public long commitCheck(Map<Integer, Integer> newUpdate) {
+        public long commitCheck(Map<Integer, Update> newUpdate) {
             for (Integer page : newUpdate.keySet()) {
                 if( updates.containsKey( page ) ) {
                     throw new OptimisticUpdateException();
@@ -174,7 +174,7 @@
             return head;
         }
 
-        public void merge(Paged paged, long rev, ConcurrentHashMap<Integer, Integer> updates, ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates) {
+        public void merge(Allocator allocator, long rev, ConcurrentHashMap<Integer, Update> updates, ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates) {
             assert head+1 == rev;
             head=rev;
             if (deferredUpdates != null) {
@@ -191,9 +191,9 @@
                             // TODO: There was a previous deferred update in the redo...  we can just use it's 
                             // redo allocation and release the new allocation.
                             if (previous != null) {
-                                Integer allocated = updates.remove(page);
-                                assert allocated == du.page; // these should match...
-                                paged.allocator().free(du.page, 1);
+                                Update allocated = updates.remove(page);
+                                assert allocated.update_location == du.page; // these should match...
+                                allocator.free(du.page, 1);
                                 // since we replaced the previous entry,  
                                 du.page = previous.page;
                             }
@@ -201,7 +201,43 @@
                     }
                 }
             }
-            this.updates.putAll(updates);
+            
+            // merge all the entries in the update..
+            for (Entry<Integer, Update> entry : updates.entrySet()) {
+                merge(allocator, entry.getKey(), entry.getValue());
+            }
+        }
+
+        /**
+         * merges one update..
+         * 
+         * @param page
+         * @param update
+         */
+        private void merge(Allocator allocator, int page, Update update) {
+            Update previous = this.updates.put(page, update);
+            if (previous != null) {
+                if( update.wasFreed() ) {
+                    // we can undo the previous update
+                    if( previous.update_location != page ) {
+                        allocator.free(previous.update_location, 1);
+                    }
+                    if( previous.wasAllocated() ) {
+                        allocator.free(page, 1);
+                    }
+                    this.updates.remove(page);
+                } else {
+                    // we are undoing the previous update /w this new update.
+                    if( previous.update_location != page ) {
+                        allocator.free(previous.update_location, 1);
+                    }
+                    // we may be updating a previously allocated page,
+                    // if so we need to mark the new page as allocated too.
+                    if( previous.wasAllocated() ) {
+                        update.flags = PAGE_ALLOCATED;
+                    }                    
+                }
+            }
         }
 
         @SuppressWarnings("unchecked")
@@ -209,7 +245,7 @@
         public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             base = in.readLong();
             head = in.readLong();
-            updates = (ConcurrentHashMap<Integer, Integer>) in.readObject();
+            updates = (ConcurrentHashMap<Integer, Update>) in.readObject();
         }
 
         @Override
@@ -221,7 +257,7 @@
         
     }
     
-    class Snapshot {
+    final class Snapshot {
         final SnapshotHead head;
         final Redo base;
         
@@ -247,7 +283,7 @@
      *  
      * @author chirino
      */
-    class SnapshotHead extends RedoEntry {
+    final class SnapshotHead extends RedoEntry {
         final Redo parent;
         
         public SnapshotHead(Redo parent) {
@@ -314,16 +350,9 @@
                 while( curEntry!=null ) {
                     Commit commit = curEntry.isCommit();
                     if( commit !=null ) {
-                        Integer updatedPage = commit.updates.get(page);
-                        if (updatedPage != null) {
-                            switch (updatedPage) {
-                            case PAGE_FREED:
-                                throw new PagingException("You should never try to read page that has been freed.");
-                            case PAGE_ALLOCATED:
-                                return page;
-                            default:
-                                return updatedPage;
-                            }
+                        Update update = commit.updates.get(page);
+                        if( update!=null ) {
+                            return update.page();
                         }
                     }
                     curEntry = curEntry.getPrevious();
@@ -367,7 +396,7 @@
             return readCache.cacheLoad(marshaller, page);
         }
         
-        public long commitCheck(Map<Integer, Integer> pageUpdates) {
+        public long commitCheck(Map<Integer, Update> pageUpdates) {
             long rc=0;
             Redo curRedo = parent;
             RedoEntry curEntry = getNext();
@@ -517,16 +546,14 @@
                         if( cu.value == null ) {
                             List<Integer> freePages = cu.marshaller.remove(pageFile, cu.page);
                             for (Integer page : freePages) {
-                                // add any allocated pages to the update list so that the free 
-                                // list gets properly adjusted.
-                                commit.updates.put(page, PAGE_FREED);
+                                commit.merge(pageFile.allocator(), page, Update.freed(page));
                             }
                         } else {
                             List<Integer> allocatedPages = cu.store(pageFile);
                             for (Integer page : allocatedPages) {
                                 // add any allocated pages to the update list so that the free 
                                 // list gets properly adjusted.
-                                commit.updates.put(page, PAGE_ALLOCATED);
+                                commit.merge(pageFile.allocator(), page, Update.allocated(page));
                             }
                         }
                     }
@@ -536,23 +563,14 @@
 
         public void freeRedoSpace(SimpleAllocator allocator) {
             for (Commit commit : this) {
-                for (Entry<Integer, Integer> entry : commit.updates.entrySet()) {
+                for (Entry<Integer, Update> entry : commit.updates.entrySet()) {
                     int key = entry.getKey();
-                    int value = entry.getValue();
-            
-                    switch( value ) {
-                    case PAGE_ALLOCATED:
-                        // It was a new page that was written.. we don't need to 
-                        // free it.
-                        break;
-                    case PAGE_FREED:
-                        // update freed a previous page.. now is when we actually free it.
+                    Update value = entry.getValue();
+                    if( value.wasFreed() ) {
                         allocator.free(key, 1);
-                        break;
-                    default:
-                        // This updated the 'key' page, now we can release the 'value' page
-                        // since it has been copied over the 'key' page and is no longer needed.
-                        allocator.free(value, 1);
+                    } else if( key != value.update_location ) {
+                        // need to free the udpate page..
+                        allocator.free(value.update_location, 1);
                     }
                 }
             }
@@ -629,7 +647,7 @@
      * @param pageUpdates
      * @param deferredUpdates
      */
-    void commit(Snapshot snapshot, ConcurrentHashMap<Integer, Integer> pageUpdates, ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates) {
+    void commit(Snapshot snapshot, ConcurrentHashMap<Integer, Update> pageUpdates, ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates) {
         
         boolean fullRedo=false;
         synchronized (TRANSACTION_MUTEX) {
@@ -664,7 +682,7 @@
             
             if( commit!=null ) {
                 // TODO: figure out how to do the merge outside the TRANSACTION_MUTEX
-                commit.merge(pageFile, rev, pageUpdates, deferredUpdates);
+                commit.merge(pageFile.allocator(), rev, pageUpdates, deferredUpdates);
             } else {
                 buildingRedo.entries.addLast(new Commit(rev, pageUpdates, deferredUpdates) );
             }
@@ -923,41 +941,42 @@
             
             // Performing the redo actually applies the updates to the original page locations.
             for (Commit commit : syncedRedos) {
-                for (Entry<Integer, Integer> entry : commit.updates.entrySet()) {
-                    int key = entry.getKey();
-                    int value = entry.getValue();
-                    switch( value ) {
-                    case PAGE_ALLOCATED:
-                        if( syncedRedos.recovered ) {
-                            // If we are recovering, the allocator MIGHT not have this 
-                            // page as being allocated.  This makes sure it's allocated so that
-                            // new transaction to get this page and overwrite it in error.
-                            allocator.unfree(key, 1);
-                        }
-                        // Update the persistent free list.  This gets stored on the next sync.
-                        baseRevisionFreePages.remove(key, 1);
-                        break;
-                    case PAGE_FREED:
-                        // The actual free gets done on the next file sync.
-                        // Update the persistent free list.  This gets stored on the next sync.
-                        baseRevisionFreePages.add(key, 1);
-                        break;
-                    default:
+                for (Entry<Integer, Update> entry : commit.updates.entrySet()) {
+                    int page = entry.getKey();
+                    Update update = entry.getValue();
+                    
+                    if( page != update.update_location ) {
+                        
                         if( syncedRedos.recovered ) {
                             // If we are recovering, the allocator MIGHT not have this 
                             // page as being allocated.  This makes sure it's allocated so that
                             // new transaction to get this page and overwrite it in error.
-                            allocator.unfree(key, 1);
+                            allocator.unfree(page, 1);
                         }
                         
-                        // Perform the update by copying the updated 'redo page' to the original
+                        // Perform the update by copying the updated page the original
                         // page location.
-                        ByteBuffer slice = pageFile.slice(SliceType.READ, value, 1);
+                        ByteBuffer slice = pageFile.slice(SliceType.READ, update.update_location, 1);
                         try {
-                            pageFile.write(key, slice);
+                            pageFile.write(page, slice);
                         } finally { 
                             pageFile.unslice(slice);
                         }
+                        
+                    }
+                    if( update.wasAllocated() ) {
+                        
+                        if( syncedRedos.recovered ) {
+                            // If we are recovering, the allocator MIGHT not have this 
+                            // page as being allocated.  This makes sure it's allocated so that
+                            // new transaction to get this page and overwrite it in error.
+                            allocator.unfree(page, 1);
+                        }
+                        // Update the persistent free list.  This gets stored on the next sync.
+                        baseRevisionFreePages.remove(page, 1);
+                        
+                    } else if( update.wasFreed() ) {
+                        baseRevisionFreePages.add(page, 1);
                     }
                 }
             }
@@ -1082,7 +1101,7 @@
     // Simple Helper Classes
     // /////////////////////////////////////////////////////////////////
 
-    class ReadCache {
+    final class ReadCache {
         private final Map<Integer, Object> map = Collections.synchronizedMap(new LRUCache<Integer, Object>(1024));
 
         @SuppressWarnings("unchecked")
@@ -1100,7 +1119,7 @@
         }        
     }
     
-    static class DeferredUpdate {
+    final static class DeferredUpdate {
         int page;
         Object value;
         EncoderDecoder<?> marshaller;
@@ -1130,4 +1149,55 @@
         public List<Integer> store(Paged paged) {
             return ((EncoderDecoder)marshaller).store(paged, page, value);
         }
-    }}
+    }
+    
+    public static final byte PAGE_ALLOCATED = 1;
+    public static final byte PAGE_FREED = 2;
+    
+    final static class Update implements Serializable {
+
+        private static final long serialVersionUID = -1128410792448869134L;
+        
+        byte flags;
+        final int update_location;
+       
+        public Update(int updateLocation, byte flags) {
+            this.update_location = updateLocation;
+            this.flags = flags;
+        }
+
+        public static Update updated(int page) {
+            return new Update(page, (byte) 0);
+        }
+
+        public static Update allocated(int page) {
+            return new Update(page, PAGE_ALLOCATED);
+        }
+
+        public static Update freed(int page) {
+            return new Update(page, PAGE_FREED);
+        }
+
+        public boolean wasFreed() {
+            return flags == PAGE_FREED;
+        }
+        
+        public boolean wasAllocated() {
+            return flags == PAGE_ALLOCATED;
+        }
+        
+        public int page() {
+            if( wasFreed() ) {
+                throw new PagingException("You should never try to read or write page that has been freed.");
+            }
+            return update_location;
+        }
+
+        @Override
+        public String toString() {
+            return "{ page: "+update_location+", flags: "+flags+" }";
+        }
+
+    }
+    
+}

Modified: activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtTransaction.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtTransaction.java?rev=828852&r1=828851&r2=828852&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtTransaction.java (original)
+++ activemq/sandbox/activemq-apollo/hawtdb/src/main/java/org/apache/hawtdb/internal/page/HawtTransaction.java Thu Oct 22 21:00:21 2009
@@ -17,7 +17,6 @@
 package org.apache.hawtdb.internal.page;
 
 import java.nio.ByteBuffer;
-import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.activemq.util.buffer.Buffer;
@@ -29,6 +28,7 @@
 import org.apache.hawtdb.api.Transaction;
 import org.apache.hawtdb.internal.page.HawtPageFile.DeferredUpdate;
 import org.apache.hawtdb.internal.page.HawtPageFile.Snapshot;
+import org.apache.hawtdb.internal.page.HawtPageFile.Update;
 
 /**
  * Transaction objects are NOT thread safe. Users of this object should
@@ -50,7 +50,7 @@
     }
 
     private ConcurrentHashMap<Integer, DeferredUpdate> deferredUpdates;
-    private ConcurrentHashMap<Integer, Integer> updates;
+    private ConcurrentHashMap<Integer, Update> updates;
     private Snapshot snapshot;
     
     private final Allocator txallocator = new Allocator() {
@@ -59,8 +59,8 @@
             // TODO: this is not a very efficient way to handle allocation ranges.
             int end = pageId+count;
             for (int key = pageId; key < end; key++) {
-                Integer previous = getUpdates().put(key, HawtPageFile.PAGE_FREED);
-                if( previous!=null && previous==HawtPageFile.PAGE_ALLOCATED) {
+                Update previous = getUpdates().put(key, Update.freed(key));
+                if( previous!=null && previous.wasAllocated() ) {
                     getUpdates().remove(key);
                     HawtTransaction.this.parent.allocator.free(key, 1);
                 }
@@ -72,7 +72,7 @@
             // TODO: this is not a very efficient way to handle allocation ranges.
             int end = pageId+count;
             for (int key = pageId; key < end; key++) {
-                getUpdates().put(key, HawtPageFile.PAGE_ALLOCATED);
+                getUpdates().put(key, Update.allocated(key));
             }
             return pageId;
         }
@@ -107,22 +107,21 @@
     }
 
     public <T> void put(EncoderDecoder<T> marshaller, int page, T value) {
-        Integer update = getUpdates().get(page);
+        Update update = getUpdates().get(page);
         if (update == null) {
             // This is the first time this transaction updates the page...
             snapshot();
-            update = parent.allocator.alloc(1);
+            update = Update.allocated(parent.allocator.alloc(1));
             getUpdates().put(page, update);
-            getCacheUpdates().put(page, new HawtPageFile.DeferredUpdate(update, value, marshaller));
+            getCacheUpdates().put(page, new HawtPageFile.DeferredUpdate(update.update_location, value, marshaller));
         } else {
             // We have updated it before...
-            switch (update) {
-            case HawtPageFile.PAGE_FREED:
+            if( update.wasFreed() ) {
                 throw new PagingException("You should never try to write a page that has been freed.");
-            case HawtPageFile.PAGE_ALLOCATED:
+            }
+            if( update.wasAllocated() ) {
                 getCacheUpdates().put(page, new HawtPageFile.DeferredUpdate(page, value, marshaller));
-                break;
-            default:
+            } else {
                 DeferredUpdate cu = getCacheUpdates().get(page);
                 if( cu == null ) {
                     throw new PagingException("You should never try to store mix using the cached objects with normal page updates.");
@@ -151,19 +150,9 @@
     }
 
     public void read(int pageId, Buffer buffer) throws IOPagingException {
-       
-        Integer updatedPageId = updates == null ? null : updates.get(pageId);
-        if (updatedPageId != null) {
-            switch (updatedPageId) {
-            case HawtPageFile.PAGE_FREED:
-                throw new PagingException("You should never try to read a page that has been freed.");
-            case HawtPageFile.PAGE_ALLOCATED:
-                parent.pageFile.read(pageId, buffer);
-                break;
-            default:
-                // read back in the updated we had done.
-                parent.pageFile.read(updatedPageId, buffer);
-            }
+        Update update = updates == null ? null : updates.get(pageId);
+        if (update != null) {
+            parent.pageFile.read(update.page(), buffer);
         } else {
             // Get the data from the snapshot.
             snapshot().head.read(pageId, buffer);
@@ -173,31 +162,22 @@
     public ByteBuffer slice(SliceType type, int page, int count) throws IOPagingException {
         //TODO: need to improve the design of ranged ops..
         if( type==SliceType.READ ) {
-            Integer udpate = updates == null ? null : updates.get(page);
+            Update udpate = updates == null ? null : updates.get(page);
             if (udpate != null) {
-                switch (udpate) {
-                case HawtPageFile.PAGE_FREED:
-                    throw new PagingException("You should never try to read a page that has been allocated or freed.");
-                case HawtPageFile.PAGE_ALLOCATED:
-                    break;
-                default:
-                    page = udpate;
-                }
-                return parent.pageFile.slice(type, page, count);
+                return parent.pageFile.slice(type, udpate.page(), count);
             } else {
                 // Get the data from the snapshot.
                 return snapshot().head.slice(page, count);
             }
             
         } else {
-            Integer update = getUpdates().get(page);
+            Update update = getUpdates().get(page);
             if (update == null) {
-                update = parent.allocator.alloc(count);
-                
+                update = Update.allocated(parent.allocator.alloc(count));
                 if (type==SliceType.READ_WRITE) {
                     ByteBuffer slice = snapshot().head.slice(page, count);
                     try {
-                        parent.pageFile.write(update, slice);
+                        parent.pageFile.write(update.update_location, slice);
                     } finally { 
                         parent.pageFile.unslice(slice);
                     }
@@ -205,23 +185,13 @@
                 
                 int end = page+count;
                 for (int i = page; i < end; i++) {
-                    getUpdates().put(i, HawtPageFile.PAGE_ALLOCATED);
+                    getUpdates().put(i, Update.allocated(i));
                 }
                 getUpdates().put(page, update);
                 
-                return parent.pageFile.slice(type, update, count);
-            } else {
-                switch (update) {
-                case HawtPageFile.PAGE_FREED:
-                    throw new PagingException("You should never try to write a page that has been freed.");
-                case HawtPageFile.PAGE_ALLOCATED:
-                    break;
-                default:
-                    page = update;
-                }
+                return parent.pageFile.slice(type, update.update_location, count);
             }
-            return parent.pageFile.slice(type, page, count);
-            
+            return parent.pageFile.slice(type, update.page(), count);
         }
         
     }
@@ -231,24 +201,14 @@
     }
 
     public void write(int page, Buffer buffer) throws IOPagingException {
-        Integer update = getUpdates().get(page);
+        Update update = getUpdates().get(page);
         if (update == null) {
             // We are updating an existing page in the snapshot...
             snapshot();
-            update = parent.allocator.alloc(1);
+            update = Update.allocated(parent.allocator.alloc(1));
             getUpdates().put(page, update);
-            page = update;
-        } else {
-            switch (update) {
-            case HawtPageFile.PAGE_FREED:
-                throw new PagingException("You should never try to write a page that has been freed.");
-            case HawtPageFile.PAGE_ALLOCATED:
-                break;
-            default:
-                page = update;
-            }
         }
-        parent.pageFile.write(page, buffer);
+        parent.pageFile.write(update.page(), buffer);
     }
 
 
@@ -275,16 +235,9 @@
     public void rollback() throws IOPagingException {
         try {
             if (updates!=null) {
-                for (Entry<Integer, Integer> entry : updates.entrySet()) {
-                    switch (entry.getValue()) {
-                    case HawtPageFile.PAGE_FREED:
-                        // Don't need to do anything..
-                        break;
-                    case HawtPageFile.PAGE_ALLOCATED:
-                    default:
-                        // We need to free the page that was allocated for the
-                        // update..
-                        parent.allocator.free(entry.getKey(), 1);
+                for (Update update : updates.values()) {
+                    if( !update.wasFreed() ) {
+                        parent.allocator.free(update.update_location, 1);
                     }
                 }
             }
@@ -316,9 +269,9 @@
         return deferredUpdates;
     }
 
-    private ConcurrentHashMap<Integer, Integer> getUpdates() {
+    private ConcurrentHashMap<Integer, Update> getUpdates() {
         if (updates == null) {
-            updates = new ConcurrentHashMap<Integer, Integer>();
+            updates = new ConcurrentHashMap<Integer, Update>();
         }
         return updates;
     }