You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2009/03/02 07:11:21 UTC

svn commit: r749204 [2/2] - /incubator/cassandra/src/org/apache/cassandra/utils/

Added: incubator/cassandra/src/org/apache/cassandra/utils/FastHash.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/FastHash.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/FastHash.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/FastHash.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,384 @@
+package org.apache.cassandra.utils;
+
+import java.util.Random;
+
+
+/**
+ * Base class for hashtables that use open addressing to resolve collisions.
+ * 
+ * @author Avinash Lakshman
+ * 
+ */
+
+abstract public class FastHash implements Cloneable
+{
+    /** the current number of occupied slots in the hash. */
+    protected transient int size_;
+    
+    /** the current number of free slots in the hash. */
+    protected transient int free_;
+    
+    /** the load above which rehashing occurs. */
+    protected static final float DEFAULT_LOAD_FACTOR = 0.5f;
+    
+    /**
+     * the default initial capacity for the hash table. This is one less than a
+     * prime value because one is added to it when searching for a prime
+     * capacity to account for the free slot required by open addressing. Thus,
+     * the real default capacity is 11.
+     */
+    protected static final int DEFAULT_INITIAL_CAPACITY = 10;
+    
+    /**
+     * Determines how full the internal table can become before rehashing is
+     * required. This must be a value in the range: 0.0 < loadFactor < 1.0. The
+     * default value is 0.5, which is about as large as you can get in open
+     * addressing without hurting performance. Cf. Knuth, Volume 3., Chapter 6.
+     */
+    protected float loadFactor_;
+    
+    /**
+     * The maximum number of elements allowed without allocating more space.
+     */
+    protected int maxSize_;
+    
+    /**
+     * The number of removes that should be performed before an auto-compaction
+     * occurs.
+     */
+    protected int autoCompactRemovesRemaining_;
+    
+    /**
+     * The auto-compaction factor for the table.
+     * 
+     * @see #setAutoCompactionFactor
+     */
+    protected float autoCompactionFactor_;
+    
+    /**
+     * @see
+     */
+    private boolean autoCompactTemporaryDisable_ = false;
+    
+    /**
+     * Creates a new <code>THash</code> instance with the default capacity and
+     * load factor.
+     */
+    public FastHash()
+    {
+        this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR);
+    }
+    
+    /**
+     * Creates a new <code>THash</code> instance with a prime capacity at or
+     * near the specified capacity and with the default load factor.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     */
+    public FastHash(int initialCapacity)
+    {
+        this(initialCapacity, DEFAULT_LOAD_FACTOR);
+    }
+    
+    /**
+     * Creates a new <code>THash</code> instance with a prime capacity at or
+     * near the minimum needed to hold <tt>initialCapacity</tt> elements with
+     * load factor <tt>loadFactor</tt> without triggering a rehash.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     * @param loadFactor
+     *            a <code>float</code> value
+     */
+    public FastHash(int initialCapacity, float loadFactor)
+    {
+        super();
+        loadFactor_ = loadFactor;
+        
+        // Through testing, the load factor (especially the default load factor)
+        // has been
+        // found to be a pretty good starting auto-compaction factor.
+        autoCompactionFactor_ = loadFactor;
+        
+        setUp((int) Math.ceil(initialCapacity / loadFactor));
+    }
+    
+    public Object clone()
+    {
+        try
+        {
+            return super.clone();
+        }
+        catch (CloneNotSupportedException cnse)
+        {
+            return null; // it's supported
+        }
+    }
+    
+    /**
+     * Tells whether this set is currently holding any elements.
+     * 
+     * @return a <code>boolean</code> value
+     */
+    public boolean isEmpty()
+    {
+        return 0 == size_;
+    }
+    
+    /**
+     * Returns the number of distinct elements in this collection.
+     * 
+     * @return an <code>int</code> value
+     */
+    public int size()
+    {
+        return size_;
+    }
+    
+    /**
+     * @return the current physical capacity of the hash table.
+     */
+    abstract protected int capacity();
+    
+    /**
+     * Ensure that this hashtable has sufficient capacity to hold
+     * <tt>desiredCapacity<tt> <b>additional</b> elements without
+     * requiring a rehash.  This is a tuning method you can call
+     * before doing a large insert.
+     *
+     * @param desiredCapacity an <code>int</code> value
+     */
+    public void ensureCapacity(int desiredCapacity)
+    {
+        if (desiredCapacity > (maxSize_ - size()))
+        {
+            rehash(PrimeFinder.nextPrime((int) Math.ceil(desiredCapacity
+                    + size() / loadFactor_) + 1));
+            computeMaxSize(capacity());
+        }
+    }
+    
+    /**
+     * Compresses the hashtable to the minimum prime size (as defined by
+     * PrimeFinder) that will hold all of the elements currently in the table.
+     * If you have done a lot of <tt>remove</tt> operations and plan to do a
+     * lot of queries or insertions or iteration, it is a good idea to invoke
+     * this method. Doing so will accomplish two things:
+     * 
+     * <ol>
+     * <li> You'll free memory allocated to the table but no longer needed
+     * because of the remove()s.</li>
+     * 
+     * <li> You'll get better query/insert/iterator performance because there
+     * won't be any <tt>REMOVED</tt> slots to skip over when probing for
+     * indices in the table.</li>
+     * </ol>
+     */
+    public void compact()
+    {
+        // need at least one free spot for open addressing
+        rehash(PrimeFinder.nextPrime((int) Math.ceil(size() / loadFactor_) + 1));
+        computeMaxSize(capacity());
+        
+        // If auto-compaction is enabled, re-determine the compaction interval
+        if (autoCompactionFactor_ != 0)
+        {
+            computeNextAutoCompactionAmount(size());
+        }
+    }
+    
+    /**
+     * The auto-compaction factor controls whether and when a table performs a
+     * {@link #compact} automatically after a certain number of remove
+     * operations. If the value is non-zero, the number of removes that need to
+     * occur for auto-compaction is the size of table at the time of the
+     * previous compaction (or the initial capacity) multiplied by this factor.
+     * <p>
+     * Setting this value to zero will disable auto-compaction.
+     */
+    public void setAutoCompactionFactor(float factor)
+    {
+        if (factor < 0)
+        {
+            throw new IllegalArgumentException("Factor must be >= 0: " + factor);
+        }
+        
+        autoCompactionFactor_ = factor;
+    }
+    
+    /**
+     * @see #setAutoCompactionFactor
+     */
+    public float getAutoCompactionFactor()
+    {
+        return autoCompactionFactor_;
+    }
+    
+    /**
+     * This simply calls {@link #compact compact}. It is included for symmetry
+     * with other collection classes. Note that the name of this method is
+     * somewhat misleading (which is why we prefer <tt>compact</tt>) as the
+     * load factor may require capacity above and beyond the size of this
+     * collection.
+     * 
+     * @see #compact
+     */
+    public final void trimToSize()
+    {
+        compact();
+    }
+    
+    /**
+     * Delete the record at <tt>index</tt>. Reduces the size of the
+     * collection by one.
+     * 
+     * @param index
+     *            an <code>int</code> value
+     */
+    protected void removeAt(int index)
+    {
+        size_--;
+        
+        // If auto-compaction is enabled, see if we need to compact
+        if (autoCompactionFactor_ != 0)
+        {
+            autoCompactRemovesRemaining_--;
+            
+            if (!autoCompactTemporaryDisable_
+                    && autoCompactRemovesRemaining_ <= 0)
+            {
+                // Do the compact
+                // NOTE: this will cause the next compaction interval to be
+                // calculated
+                compact();
+            }
+        }
+    }
+    
+    /**
+     * Empties the collection.
+     */
+    public void clear()
+    {
+        size_ = 0;
+        free_ = capacity();
+    }
+    
+    /**
+     * initializes the hashtable to a prime capacity which is at least
+     * <tt>initialCapacity + 1</tt>.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     * @return the actual capacity chosen
+     */
+    protected int setUp(int initialCapacity)
+    {
+        int capacity;
+        
+        capacity = PrimeFinder.nextPrime(initialCapacity);
+        computeMaxSize(capacity);
+        computeNextAutoCompactionAmount(initialCapacity);
+        
+        return capacity;
+    }
+    
+    /**
+     * Rehashes the set.
+     * 
+     * @param newCapacity
+     *            an <code>int</code> value
+     */
+    protected abstract void rehash(int newCapacity);
+    
+    /**
+     * Temporarily disables auto-compaction. MUST be followed by calling
+     * {@link #reenableAutoCompaction}.
+     */
+    protected void tempDisableAutoCompaction()
+    {
+        autoCompactTemporaryDisable_ = true;
+    }
+    
+    /**
+     * Re-enable auto-compaction after it was disabled via
+     * {@link #tempDisableAutoCompaction()}.
+     * 
+     * @param check_for_compaction
+     *            True if compaction should be performed if needed before
+     *            returning. If false, no compaction will be performed.
+     */
+    protected void reenableAutoCompaction(boolean check_for_compaction)
+    {
+        autoCompactTemporaryDisable_ = false;
+        
+        if (check_for_compaction && autoCompactRemovesRemaining_ <= 0
+                && autoCompactionFactor_ != 0)
+        {
+            
+            // Do the compact
+            // NOTE: this will cause the next compaction interval to be
+            // calculated
+            compact();
+        }
+    }
+    
+    /**
+     * Computes the values of maxSize. There will always be at least one free
+     * slot required.
+     * 
+     * @param capacity
+     *            an <code>int</code> value
+     */
+    private final void computeMaxSize(int capacity)
+    {
+        // need at least one free slot for open addressing
+        maxSize_ = Math.min(capacity - 1, (int) Math.floor(capacity
+                * loadFactor_));
+        free_ = capacity - size_; // reset the free element count
+    }
+    
+    /**
+     * Computes the number of removes that need to happen before the next
+     * auto-compaction will occur.
+     */
+    private void computeNextAutoCompactionAmount(int size)
+    {
+        if (autoCompactionFactor_ != 0)
+        {
+            autoCompactRemovesRemaining_ = Math.round(size
+                    * autoCompactionFactor_);
+        }
+    }
+    
+    /**
+     * After an insert, this hook is called to adjust the size/free values of
+     * the set and to perform rehashing if necessary.
+     */
+    protected final void postInsertHook(boolean usedFreeSlot)
+    {
+        if (usedFreeSlot)
+        {
+            free_--;
+        }
+        
+        // rehash whenever we exhaust the available space in the table
+        if (++size_ > maxSize_ || free_ == 0)
+        {
+            // choose a new capacity suited to the new state of the table
+            // if we've grown beyond our maximum size, double capacity;
+            // if we've exhausted the free spots, rehash to the same capacity,
+            // which will free up any stale removed slots for reuse.
+            int newCapacity = size_ > maxSize_ ? PrimeFinder
+                    .nextPrime(capacity() << 1) : capacity();
+                    rehash(newCapacity);
+                    computeMaxSize(capacity());
+        }
+    }
+    
+    protected int calculateGrownCapacity()
+    {
+        return capacity() << 1;
+    }
+}// THash

Added: incubator/cassandra/src/org/apache/cassandra/utils/FastHashMap.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/FastHashMap.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/FastHashMap.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/FastHashMap.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,583 @@
+package org.apache.cassandra.utils;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * An implementation of the Map interface which uses an open addressed hash
+ * table to store its contents
+ * @author Avinash Lakshman
+ */
+public class FastHashMap<K, V> extends FastObjectHash<K> implements Map<K, V>, Serializable
+{
+    static final long serialVersionUID = 1L;
+
+    /** the values of the map */
+    protected transient V[] values_;
+
+    /**
+     * Creates a new <code>FastHashMap</code> instance with the default capacity
+     * and load factor.
+     */
+    public FastHashMap()
+    {
+        super();
+    }
+
+    /**
+     * Creates a new <code>FastHashMap</code> instance with a prime capacity
+     * equal to or greater than <tt>initialCapacity</tt> and with the default
+     * load factor.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     */
+    public FastHashMap(int initialCapacity)
+    {
+        super(initialCapacity);
+    }
+
+    /**
+     * Creates a new <code>FastHashMap</code> instance with a prime capacity
+     * equal to or greater than <tt>initialCapacity</tt> and with the
+     * specified load factor.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     * @param loadFactor
+     *            a <code>float</code> value
+     */
+    public FastHashMap(int initialCapacity, float loadFactor)
+    {
+        super(initialCapacity, loadFactor);
+    }
+
+    /**
+     * Creates a new <code>FastHashMap</code> instance which contains the
+     * key/value pairs in <tt>map</tt>.
+     * 
+     * @param map
+     *            a <code>Map</code> value
+     */
+    public FastHashMap(Map<K, V> map)
+    {
+        this(map.size());
+        putAll(map);
+    }
+
+    /**
+     * @return a shallow clone of this collection
+     */
+    public FastHashMap<K, V> clone()
+    {
+        FastHashMap<K, V> m = (FastHashMap<K, V>) super.clone();
+        m.values_ = this.values_.clone();
+        return m;
+    }
+
+    /**
+     * initialize the value array of the map.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     * @return an <code>int</code> value
+     */
+    protected int setUp(int initialCapacity)
+    {
+        int capacity;
+
+        capacity = super.setUp(initialCapacity);
+        values_ = (V[]) new Object[capacity];
+        return capacity;
+    }
+    
+    void addEntry(Object key, int index)
+    {    	
+    }
+    
+    void removeEntry(Object key, int index)
+    {
+    }
+
+    /**
+     * Inserts a key/value pair into the map.
+     * 
+     * @param key
+     *            an <code>Object</code> value
+     * @param value
+     *            an <code>Object</code> value
+     * @return the previous value associated with <tt>key</tt>, or null if
+     *         none was found.
+     */
+    public V put(K key, V value)
+    {
+        V previous = null;
+        Object oldKey;
+        int index = insertionIndex(key);
+        boolean isNewMapping = true;
+        if (index < 0)
+        {
+            index = -index - 1;
+            previous = values_[index];
+            isNewMapping = false;
+        }
+        oldKey = set_[index];
+        
+        if ( oldKey == FREE )
+        {
+        	/* This is used as a hook to process new put() operations */
+        	addEntry(key, index);
+        }
+    
+        set_[index] = key;
+        values_[index] = value;
+        if (isNewMapping)
+        {
+            postInsertHook(oldKey == FREE);
+        }
+
+        return previous;
+    }
+
+    /**
+     * rehashes the map to the new capacity.
+     * 
+     * @param newCapacity
+     *            an <code>int</code> value
+     */
+    protected void rehash(int newCapacity)
+    {
+        int oldCapacity = set_.length;
+        Object oldKeys[] = set_;
+        V oldVals[] = values_;
+
+        set_ = new Object[newCapacity];
+        Arrays.fill(set_, FREE);
+        values_ = (V[]) new Object[newCapacity];
+
+        for (int i = oldCapacity; i-- > 0;)
+        {
+            if (oldKeys[i] != FREE && oldKeys[i] != REMOVED)
+            {
+                Object o = oldKeys[i];
+                int index = insertionIndex((K) o);
+                if (index < 0)
+                {
+                    throwObjectContractViolation(set_[(-index - 1)], o);
+                }
+                set_[index] = o;
+                values_[index] = oldVals[i];
+            }
+        }
+    }
+
+    /**
+     * retrieves the value for <tt>key</tt>
+     * 
+     * @param key
+     *            an <code>Object</code> value
+     * @return the value of <tt>key</tt> or null if no such mapping exists.
+     */
+    public V get(Object key)
+    {
+        int index = index((K) key);
+        return index < 0 ? null : values_[index];
+    }
+
+    /**
+     * Empties the map.
+     * 
+     */
+    public void clear()
+    {
+        if (size() == 0)
+            return; // optimization
+
+        super.clear();
+        Object[] keys = set_;
+        V[] vals = values_;
+
+        for (int i = keys.length; i-- > 0;)
+        {
+            keys[i] = FREE;
+            vals[i] = null;
+        }
+    }
+
+    /**
+     * Deletes a key/value pair from the map.
+     * 
+     * @param key an <code>Object</code> value
+     * @return an <code>Object</code> value
+     */
+    public V remove(Object key)
+    {
+        V prev = null;
+        int index = index((K)key);
+        if (index >= 0)
+        {
+            prev = values_[index];
+            /* clear key,state; adjust size */
+            removeAt(index);
+            /* This is used as hook to process deleted items */
+            removeEntry(key, index);
+        }
+        return prev;
+    }
+
+    /**
+     * removes the mapping at <tt>index</tt> from the map.
+     * 
+     * @param index an <code>int</code> value
+     */
+    protected void removeAt(int index)
+    {
+        values_[index] = null;
+        /* clear key, state; adjust size */
+        super.removeAt(index); 
+    }
+
+    /**
+     * Returns a view on the values of the map.
+     * 
+     * @return a <code>Collection</code> value
+     */
+    public Collection<V> values()
+    {
+        return Arrays.asList(values_);
+    }
+
+    /**
+     * returns a Set view on the keys of the map.
+     * 
+     * @return a <code>Set</code> value
+     */
+    public Set<K> keySet()
+    {
+        return new KeyView();
+    }
+
+    /**
+     * Returns a Set view on the entries of the map.
+     * 
+     * @return a <code>Set</code> value
+     */
+    public Set<Map.Entry<K, V>> entrySet()
+    {
+        throw new UnsupportedOperationException(
+                "This operation is currently not supported.");
+    }
+
+    /**
+     * checks for the presence of <tt>val</tt> in the values of the map.
+     * 
+     * @param val
+     *            an <code>Object</code> value
+     * @return a <code>boolean</code> value
+     */
+    public boolean containsValue(Object val)
+    {
+        Object[] set = set_;
+        V[] vals = values_;
+
+        // special case null values so that we don't have to
+        // perform null checks before every call to equals()
+        if (null == val)
+        {
+            for (int i = vals.length; i-- > 0;)
+            {
+                if ((set[i] != FREE && set[i] != REMOVED) && val == vals[i])
+                {
+                    return true;
+                }
+            }
+        }
+        else
+        {
+            for (int i = vals.length; i-- > 0;)
+            {
+                if ((set[i] != FREE && set[i] != REMOVED)
+                        && (val == vals[i] || val.equals(vals[i])))
+                {
+                    return true;
+                }
+            }
+        } // end of else
+        return false;
+    }
+
+    /**
+     * checks for the present of <tt>key</tt> in the keys of the map.
+     * 
+     * @param key
+     *            an <code>Object</code> value
+     * @return a <code>boolean</code> value
+     */
+    public boolean containsKey(Object key)
+    {
+        return contains(key);
+    }
+
+    /**
+     * copies the key/value mappings in <tt>map</tt> into this map.
+     * 
+     * @param map
+     *            a <code>Map</code> value
+     */
+    public void putAll(Map<? extends K, ? extends V> map)
+    {
+        ensureCapacity(map.size());
+        // could optimize this for cases when map instanceof FastHashMap
+        for (Iterator<? extends Map.Entry<? extends K, ? extends V>> i = map
+                .entrySet().iterator(); i.hasNext();)
+        {
+            Map.Entry<? extends K, ? extends V> e = i.next();
+            put(e.getKey(), e.getValue());
+        }
+    }
+
+    private abstract class MapBackedView<E> extends AbstractSet<E> implements Set<E>, Iterable<E>
+    {
+        public abstract Iterator<E> iterator();
+        public abstract boolean removeElement(E key);
+        public abstract boolean containsElement(E key);
+
+        public boolean contains(Object key)
+        {
+            return containsElement((E) key);
+        }
+
+        public boolean remove(Object o)
+        {
+            return removeElement((E) o);
+        }
+
+        public boolean containsAll(Collection<?> collection)
+        {
+            for (Iterator i = collection.iterator(); i.hasNext();)
+            {
+                if (!contains(i.next()))
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+
+        public void clear()
+        {
+            FastHashMap.this.clear();
+        }
+
+        public boolean add(E obj)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public int size()
+        {
+            return FastHashMap.this.size();
+        }
+
+        public Object[] toArray()
+        {
+            Object[] result = new Object[size()];
+            Iterator e = iterator();
+            for (int i = 0; e.hasNext(); i++)
+                result[i] = e.next();
+            return result;
+        }
+
+        public <T> T[] toArray(T[] a)
+        {
+            int size = size();
+            if (a.length < size)
+                a = (T[]) java.lang.reflect.Array.newInstance(a.getClass().getComponentType(), size);
+
+            Iterator<E> it = iterator();
+            Object[] result = a;
+            for (int i = 0; i < size; i++)
+            {
+                result[i] = it.next();
+            }
+
+            if (a.length > size)
+            {
+                a[size] = null;
+            }
+
+            return a;
+        }
+
+        public boolean isEmpty()
+        {
+            return FastHashMap.this.isEmpty();
+        }
+
+        public boolean addAll(Collection<? extends E> collection)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public boolean retainAll(Collection<?> collection)
+        {
+            boolean changed = false;
+            Iterator i = iterator();
+            while (i.hasNext())
+            {
+                if (!collection.contains(i.next()))
+                {
+                    i.remove();
+                    changed = true;
+                }
+            }
+            return changed;
+        }
+    }
+    
+    protected class FastHashMapIterator<T> implements Iterator<T>
+    {
+        private int nextIndex_;
+        private int expectedSize_;
+        private FastObjectHash<T> tMap_;
+        
+        FastHashMapIterator(FastObjectHash<T> tMap)
+        {
+            nextIndex_ = -1;
+            expectedSize_ = tMap.size();
+            tMap_ = tMap;
+        }
+        
+        public boolean hasNext()
+        {
+            return (expectedSize_ > 0);
+        }
+        
+        public T next()
+        {
+            moveToNextIndex();
+            int index = nextIndex_;
+            /* 
+             * Decrement so that we can track how many 
+             * elements we have already looked at.
+            */
+            --expectedSize_;
+            return (T)tMap_.set_[index];
+        }
+        
+        private void moveToNextIndex()
+        {
+            int i = nextIndex_ + 1;
+            for ( ; i < tMap_.set_.length; ++i )
+            {
+                if ( tMap_.set_[i].equals(FREE) || tMap_.set_[i].equals(REMOVED) )
+                {
+                    continue;
+                }
+                else
+                {                    
+                    break;
+                }
+            }
+            nextIndex_ = i;
+        }
+        
+        public void remove()
+        {
+            tMap_.removeAt(nextIndex_);
+            --expectedSize_;
+        }
+    }
+
+    /**
+     * a view onto the keys of the map.
+     */
+    protected class KeyView extends MapBackedView<K>
+    {
+        public Iterator<K> iterator()
+        {
+            return new FastHashMapIterator(FastHashMap.this);
+        }
+        
+        public boolean removeElement(K key)
+        {
+            return null != FastHashMap.this.remove(key);
+        }
+
+        public boolean containsElement(K key)
+        {
+            return FastHashMap.this.contains(key);
+        }
+    }
+
+    final class Entry implements Map.Entry<K, V>
+    {
+        private K key;
+        private V val;
+        private final int index;
+
+        Entry(final K key, V value, final int index)
+        {
+            this.key = key;
+            this.val = value;
+            this.index = index;
+        }
+
+        void setKey(K aKey)
+        {
+            this.key = aKey;
+        }
+
+        void setValue0(V aValue)
+        {
+            this.val = aValue;
+        }
+
+        public K getKey()
+        {
+            return key;
+        }
+
+        public V getValue()
+        {
+            return val;
+        }
+
+        public V setValue(V o)
+        {
+            if (values_[index] != val)
+            {
+                throw new ConcurrentModificationException();
+            }
+            values_[index] = o;
+            o = val; // need to return previous value
+            val = o; // update this entry's value, in case
+            // setValue is called again
+            return o;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (o instanceof Map.Entry)
+            {
+                Map.Entry e1 = this;
+                Map.Entry e2 = (Map.Entry) o;
+                return (e1.getKey() == null ? e2.getKey() == null : e1.getKey().equals(e2.getKey()))
+                        && (e1.getValue() == null ? e2.getValue() == null : e1.getValue().equals(e2.getValue()));
+            }
+            return false;
+        }
+
+        public int hashCode()
+        {
+            return (getKey() == null ? 0 : getKey().hashCode()) ^ (getValue() == null ? 0 : getValue().hashCode());
+        }
+    }
+    
+    public static void main(String[] args) throws Throwable
+    {
+        Map<String, String> map = new FastHashMap<String, String>();
+        map.put("Avinash", "Avinash");
+        map.put("Avinash", "Srinivas");
+    }
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/FastLinkedHashMap.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/FastLinkedHashMap.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/FastLinkedHashMap.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/FastLinkedHashMap.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+public class FastLinkedHashMap<K,V> extends FastHashMap<K,V>
+{
+    
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/FastObjectHash.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/FastObjectHash.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/FastObjectHash.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/FastObjectHash.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,306 @@
+package org.apache.cassandra.utils;
+
+import java.util.Arrays;
+
+/**
+ * An open addressed hashing implementation for Object types.
+ * 
+ * @author Avinash Lakshman
+ */
+abstract public class FastObjectHash<T> extends FastHash
+{
+    static final long serialVersionUID = -3461112548087185871L;
+
+    /** the set of Objects */
+    protected transient Object[] set_;
+    protected static final Object REMOVED = new Object(), FREE = new Object();
+
+    /**
+     * Creates a new <code>TObjectHash</code> instance with the default
+     * capacity and load factor.
+     */
+    public FastObjectHash()
+    {
+        super();        
+    }
+
+    /**
+     * Creates a new <code>TObjectHash</code> instance whose capacity is the
+     * next highest prime above <tt>initialCapacity + 1</tt> unless that value
+     * is already prime.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     */
+    public FastObjectHash(int initialCapacity)
+    {
+        super(initialCapacity);        
+    }
+
+    /**
+     * Creates a new <code>TObjectHash</code> instance with a prime value at
+     * or near the specified capacity and load factor.
+     * 
+     * @param initialCapacity
+     *            used to find a prime capacity for the table.
+     * @param loadFactor
+     *            used to calculate the threshold over which rehashing takes
+     *            place.
+     */
+    public FastObjectHash(int initialCapacity, float loadFactor)
+    {
+        super(initialCapacity, loadFactor);        
+    }
+
+    /**
+     * @return a shallow clone of this collection
+     */
+    public FastObjectHash<T> clone()
+    {
+        FastObjectHash<T> h = (FastObjectHash<T>) super.clone();
+        h.set_ = (Object[]) this.set_.clone();
+        return h;
+    }
+    
+    /**
+     * This method is invoked every time a key is
+     * added into the Map.
+     * @param key key that is inserted
+     * @param index index position of the key being 
+     *              inserted
+     */
+    abstract void addEntry(Object key, int index);
+    
+    /**
+     * This method is invoked every time a key is
+     * deleted from the Map.
+     * @param key key being deleted
+     * @param index index position of the key being
+     *              deleted
+     */
+    abstract void removeEntry(Object key, int index);
+
+    protected int capacity()
+    {
+        return set_.length;
+    }
+
+    protected void removeAt(int index)
+    {
+        set_[index] = REMOVED;
+        super.removeAt(index);
+    }
+
+    /**
+     * initializes the Object set of this hash table.
+     * 
+     * @param initialCapacity
+     *            an <code>int</code> value
+     * @return an <code>int</code> value
+     */
+    protected int setUp(int initialCapacity)
+    {
+        int capacity;
+
+        capacity = super.setUp(initialCapacity);
+        set_ = new Object[capacity];
+        Arrays.fill(set_, FREE);
+        return capacity;
+    }
+
+    /**
+     * Searches the set for <tt>obj</tt>
+     * 
+     * @param obj
+     *            an <code>Object</code> value
+     * @return a <code>boolean</code> value
+     */
+    public boolean contains(Object obj)
+    {
+        return index((T) obj) >= 0;
+    }
+
+    /**
+     * Locates the index of <tt>obj</tt>.
+     * 
+     * @param obj
+     *            an <code>Object</code> value
+     * @return the index of <tt>obj</tt> or -1 if it isn't in the set.
+     */
+    protected int index(Object obj)
+    {        
+        final Object[] set = set_;
+        final int length = set.length;
+        final int hash = obj.hashCode() & 0x7fffffff;
+        int index = hash % length;
+        Object cur = set[index];
+
+        if (cur == FREE)
+            return -1;
+
+        // NOTE: here it has to be REMOVED or FULL (some user-given value)
+        if (cur == REMOVED || cur.equals(obj))
+        {
+            // see Knuth, p. 529
+            final int probe = 1 + (hash % (length - 2));
+
+            while (cur != FREE&& (cur == REMOVED || !cur.equals(obj)))
+            {
+                index -= probe;
+                if (index < 0)
+                {
+                    index += length;
+                }
+                cur = set[index];
+            }            
+        }
+
+        return cur == FREE ? -1 : index;
+    }
+
+    /**
+     * Locates the index at which <tt>obj</tt> can be inserted. if there is
+     * already a value equal()ing <tt>obj</tt> in the set, returns that
+     * value's index as <tt>-index - 1</tt>.
+     * 
+     * @param obj
+     *            an <code>Object</code> value
+     * @return the index of a FREE slot at which obj can be inserted or, if obj
+     *         is already stored in the hash, the negative value of that index,
+     *         minus 1: -index -1.
+     */
+    protected int insertionIndex(T obj)
+    {        
+        final Object[] set = set_;
+        final int length = set.length;
+        final int hash = obj.hashCode() & 0x7fffffff;
+        int index = hash % length;
+        Object cur = set[index];
+
+        if (cur == FREE)
+        {
+            return index; // empty, all done
+        }
+        else if (cur != REMOVED && cur.equals(obj))
+        {
+            return -index - 1; // already stored
+        }
+        else
+        { // already FULL or REMOVED, must probe
+            // compute the double hash
+            final int probe = 1 + (hash % (length - 2));
+
+            // if the slot we landed on is FULL (but not removed), probe
+            // until we find an empty slot, a REMOVED slot, or an element
+            // equal to the one we are trying to insert.
+            // finding an empty slot means that the value is not present
+            // and that we should use that slot as the insertion point;
+            // finding a REMOVED slot means that we need to keep searching,
+            // however we want to remember the offset of that REMOVED slot
+            // so we can reuse it in case a "new" insertion (i.e. not an update)
+            // is possible.
+            // finding a matching value means that we've found that our desired
+            // key is already in the table
+            if (cur != REMOVED)
+            {
+                // starting at the natural offset, probe until we find an
+                // offset that isn't full.
+                do
+                {
+                    index -= probe;
+                    if (index < 0)
+                    {
+                        index += length;
+                    }
+                    cur = set[index];
+                }
+                while (cur != FREE && cur != REMOVED
+                        && !cur.equals(obj));
+            }
+
+            // if the index we found was removed: continue probing until we
+            // locate a free location or an element which equal()s the
+            // one we have.
+            if (cur == REMOVED)
+            {
+                int firstRemoved = index;
+                while (cur != FREE
+                        && (cur == REMOVED || !cur.equals(obj)))
+                {
+                    index -= probe;
+                    if (index < 0)
+                    {
+                        index += length;
+                    }
+                    cur = set[index];
+                }
+                // NOTE: cur cannot == REMOVED in this block
+                return (cur != FREE) ? -index - 1 : firstRemoved;
+            }
+            // if it's full, the key is already stored
+            // NOTE: cur cannot equal REMOVE here (would have retuned already
+            // (see above)
+            return (cur != FREE) ? -index - 1 : index;
+        }
+    }
+
+    /**
+     * This is the default implementation of TObjectHashingStrategy: it
+     * delegates hashing to the Object's hashCode method.
+     * 
+     * @param o
+     *            for which the hashcode is to be computed
+     * @return the hashCode
+     * @see Object#hashCode()
+     */
+    public final int computeHashCode(T o)
+    {
+        return o == null ? 0 : o.hashCode();
+    }
+
+    /**
+     * This is the default implementation of TObjectHashingStrategy: it
+     * delegates equality comparisons to the first parameter's equals() method.
+     * 
+     * @param o1
+     *            an <code>Object</code> value
+     * @param o2
+     *            an <code>Object</code> value
+     * @return true if the objects are equal
+     * @see Object#equals(Object)
+     */
+    public final boolean equals(T o1, T o2)
+    {
+        return o1 == null ? o2 == null : o1.equals(o2);
+    }
+
+    /**
+     * Convenience methods for subclasses to use in throwing exceptions about
+     * badly behaved user objects employed as keys. We have to throw an
+     * IllegalArgumentException with a rather verbose message telling the user
+     * that they need to fix their object implementation to conform to the
+     * general contract for java.lang.Object.
+     * 
+     * @param o1
+     *            the first of the equal elements with unequal hash codes.
+     * @param o2
+     *            the second of the equal elements with unequal hash codes.
+     * @exception IllegalArgumentException
+     *                the whole point of this method.
+     */
+    protected final void throwObjectContractViolation(Object o1, Object o2)
+            throws IllegalArgumentException
+    {
+        throw new IllegalArgumentException(
+                "Equal objects must have equal hashcodes. "
+                        + "During rehashing, Trove discovered that "
+                        + "the following two objects claim to be "
+                        + "equal (as in java.lang.Object.equals()) "
+                        + "but their hashCodes (or those calculated by "
+                        + "your TObjectHashingStrategy) are not equal."
+                        + "This violates the general contract of "
+                        + "java.lang.Object.hashCode().  See bullet point two "
+                        + "in that method's documentation. " + "object #1 ="
+                        + o1 + "; object #2 =" + o2);
+    }
+} // TObjectHash

Added: incubator/cassandra/src/org/apache/cassandra/utils/FileUtils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/FileUtils.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/FileUtils.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/FileUtils.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.*;
+import java.text.DecimalFormat;
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.ThreadFactoryImpl;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.log4j.Logger;
+
+
+
+/**
+ * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
+ */
+
+public class FileUtils
+{
+    private static Logger logger_ = Logger.getLogger(FileUtils.class);
+    private static final DecimalFormat df_ = new DecimalFormat("#.##");
+    private static final double kb_ = 1024d;
+    private static final double mb_ = 1024*1024d;
+    private static final double gb_ = 1024*1024*1024d;
+    private static final double tb_ = 1024*1024*1024*1024d;
+
+    private static ExecutorService deleter_ = new DebuggableThreadPoolExecutor( 1,
+            1,
+            Integer.MAX_VALUE,
+            TimeUnit.SECONDS,
+            new LinkedBlockingQueue<Runnable>(),
+            new ThreadFactoryImpl("FILEUTILS-DELETE-POOL")
+            );
+
+    public static void shutdown()
+    {
+    	deleter_.shutdownNow();
+    }
+
+    public static class Deleter implements Runnable
+    {
+    	File file_ = null;
+
+    	public Deleter(File f)
+        {
+    		file_ = f;
+        }
+
+        public void run()
+        {
+        	if(file_ == null)
+        		return;
+        	logger_.info("*** Deleting " + file_.getName() + " ***");
+        	if(!file_.delete())
+        	{
+            	logger_.warn("Warning : Unable to delete file " + file_.getAbsolutePath());
+        	}
+        }
+    }
+
+    public static class FileComparator implements Comparator<File>
+    {
+        public int compare(File f, File f2)
+        {
+            return (int)(f.lastModified() - f2.lastModified());
+        }
+
+        public boolean equals(Object o)
+        {
+            if ( !(o instanceof FileComparator) )
+                return false;
+            return true;
+        }
+    }
+
+    public static void createDirectory(String directory) throws IOException
+    {
+        File file = new File(directory);
+        if ( !file.exists() )
+            file.mkdir();
+    }
+
+    public static void createFile(String directory) throws IOException
+    {
+        File file = new File(directory);
+        if ( !file.exists() )
+            file.createNewFile();
+    }
+
+    public static boolean isExists(String filename) throws IOException
+    {
+        File file = new File(filename);
+        return file.exists();
+    }
+
+    public static boolean delete(String file)
+    {
+        File f = new File(file);
+        return f.delete();
+    }
+
+    public static void deleteAsync(String file) throws IOException
+    {
+        File f = new File(file);
+    	Runnable deleter = new Deleter(f);
+        deleter_.submit(deleter);
+    }
+
+    public static boolean delete(List<String> files) throws IOException
+    {
+        boolean bVal = true;
+        for ( int i = 0; i < files.size(); ++i )
+        {
+            String file = files.get(i);
+            bVal = delete(file);
+            if (bVal)
+            {
+            	logger_.debug("Deleted file " + file);
+                files.remove(i);
+            }
+        }
+        return bVal;
+    }
+
+    public static void delete(File[] files) throws IOException
+    {
+        for ( File file : files )
+        {
+            file.delete();
+        }
+    }
+
+    public static String stringifyFileSize(double value)
+    {
+        double d = 0d;
+        if ( value >= tb_ )
+        {
+            d = value / tb_;
+            String val = df_.format(d);
+            return val + " TB";
+        }
+        else if ( value >= gb_ )
+        {
+            d = value / gb_;
+            String val = df_.format(d);
+            return val + " GB";
+        }
+        else if ( value >= mb_ )
+        {
+            d = value / mb_;
+            String val = df_.format(d);
+            return val + " MB";
+        }
+        else if ( value >= kb_ )
+        {
+            d = value / kb_;
+            String val = df_.format(d);
+            return val + " KB";
+        }
+        else
+        {       
+            String val = df_.format(value);
+            return val + " bytes.";
+        }        
+    }
+    
+    public static double stringToFileSize(String value)
+    {        
+        String[] peices = value.split(" ");
+        double d = Double.valueOf(peices[0]);
+        if ( peices[1].equals("TB") )
+        {
+            d *= tb_;
+        }
+        else if ( peices[1].equals("GB") )
+        {
+            d *= gb_;
+        }
+        else if ( peices[1].equals("MB") )
+        {
+            d *= mb_;
+        }
+        else if ( peices[1].equals("KB") )
+        {
+            d *= kb_;
+        }
+        else
+        {
+            d *= 1;
+        }
+        return d;
+    }
+    
+    public static long getUsedDiskSpace()
+    {
+        long diskSpace = 0L;
+        String[] directories = DatabaseDescriptor.getAllDataFileLocations();        
+        for ( String directory : directories )
+        {
+            File f = new File(directory);
+            File[] files = f.listFiles();
+            for ( File file : files )
+            {
+                diskSpace += file.length();
+            }
+        }
+
+        String value = df_.format(diskSpace);
+        return Long.parseLong(value);
+    }    
+    
+    
+	
+    /**
+     * Deletes all files and subdirectories under "dir".
+     * @param dir Directory to be deleted
+     * @return boolean Returns "true" if all deletions were successful.
+     *                 If a deletion fails, the method stops attempting to
+     *                 delete and returns "false".
+     */
+    public static boolean deleteDir(File dir) {
+
+        if (dir.isDirectory()) {
+            String[] children = dir.list();
+            for (int i=0; i<children.length; i++) {
+                boolean success = deleteDir(new File(dir, children[i]));
+                if (!success) {
+                    return false;
+                }
+            }
+        }
+
+        // The directory is now empty so now it can be smoked
+        return dir.delete();
+    }
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/GuidGenerator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/GuidGenerator.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/GuidGenerator.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/GuidGenerator.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.util.*;
+import java.net.*;
+import java.security.*;
+/**
+ * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
+ */
+
+public class GuidGenerator {
+    private static Random myRand;
+    private static SecureRandom mySecureRand;
+    private static String s_id;
+    private static SafeMessageDigest md5 = null;
+
+    static {
+        if (System.getProperty("java.security.egd") == null) {
+            System.setProperty("java.security.egd", "file:/dev/urandom");
+        }
+        mySecureRand = new SecureRandom();
+        long secureInitializer = mySecureRand.nextLong();
+        myRand = new Random(secureInitializer);
+        try {
+            s_id = InetAddress.getLocalHost().toString();
+        }
+        catch (UnknownHostException e) {
+            LogUtil.getLogger(GuidGenerator.class.getName()).debug(LogUtil.throwableToString(e));
+        }
+
+        try {
+            MessageDigest myMd5 = MessageDigest.getInstance("MD5");
+            md5 = new SafeMessageDigest(myMd5);
+        }
+        catch (NoSuchAlgorithmException e) {
+            LogUtil.getLogger(GuidGenerator.class.getName()).debug(LogUtil.throwableToString(e));
+        }
+    }
+
+
+    public static String guid() {
+        byte[] array = guidAsBytes();
+        
+        StringBuffer sb = new StringBuffer();
+        for (int j = 0; j < array.length; ++j) {
+            int b = array[j] & 0xFF;
+            if (b < 0x10) sb.append('0');
+            sb.append(Integer.toHexString(b));
+        }
+
+        return convertToStandardFormat( sb.toString() );
+    }
+    
+    public static String guidToString(byte[] bytes)
+    {
+        StringBuffer sb = new StringBuffer();
+        for (int j = 0; j < bytes.length; ++j) {
+            int b = bytes[j] & 0xFF;
+            if (b < 0x10) sb.append('0');
+            sb.append(Integer.toHexString(b));
+        }
+
+        return convertToStandardFormat( sb.toString() );
+    }
+    
+    public static byte[] guidAsBytes()
+    {
+        StringBuffer sbValueBeforeMD5 = new StringBuffer();
+        long time = System.currentTimeMillis();
+        long rand = 0;
+        rand = myRand.nextLong();
+        sbValueBeforeMD5.append(s_id);
+        sbValueBeforeMD5.append(":");
+        sbValueBeforeMD5.append(Long.toString(time));
+        sbValueBeforeMD5.append(":");
+        sbValueBeforeMD5.append(Long.toString(rand));
+
+        String valueBeforeMD5 = sbValueBeforeMD5.toString();
+        return md5.digest(valueBeforeMD5.getBytes());
+    }
+
+    /*
+        * Convert to the standard format for GUID
+        * Example: C2FEEEAC-CFCD-11D1-8B05-00600806D9B6
+    */
+
+    private static String convertToStandardFormat(String valueAfterMD5) {
+        String raw = valueAfterMD5.toUpperCase();
+        StringBuffer sb = new StringBuffer();
+        sb.append(raw.substring(0, 8));
+        sb.append("-");
+        sb.append(raw.substring(8, 12));
+        sb.append("-");
+        sb.append(raw.substring(12, 16));
+        sb.append("-");
+        sb.append(raw.substring(16, 20));
+        sb.append("-");
+        sb.append(raw.substring(20));
+        return sb.toString();
+    }
+}
+
+
+
+
+
+

Added: incubator/cassandra/src/org/apache/cassandra/utils/HashingSchemes.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/HashingSchemes.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/HashingSchemes.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/HashingSchemes.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+/**
+ * Created by IntelliJ IDEA.
+ * User: lakshman
+ * Date: Aug 17, 2005
+ * Time: 3:32:42 PM
+ * To change this template use File | Settings | File Templates.
+ */
+
+public final class HashingSchemes
+{
+    public static final String SHA_1 = "SHA-1";
+    public static final String SHA1 = "SHA1";
+    public static final String MD5 = "MD5";
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/ICacheExpungeHook.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/ICacheExpungeHook.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/ICacheExpungeHook.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/ICacheExpungeHook.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+/**
+ * Created by IntelliJ IDEA.
+ * User: lakshman
+ * Date: Aug 16, 2005
+ * Time: 1:08:58 PM
+ * To change this template use File | Settings | File Templates.
+ */
+public interface ICacheExpungeHook<K,V>
+{
+    public void callMe(K key , V value);
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/ICachetable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/ICachetable.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/ICachetable.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/ICachetable.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.util.Enumeration;
+import java.util.Set;
+
+public interface ICachetable<K,V>
+{
+    public void put(K key, V value);
+    public void put(K key, V value, ICacheExpungeHook<K,V> hook);
+	public V get(K key);
+    public V remove(K key);
+    public int size();
+    public boolean containsKey(K key);
+    public boolean containsValue(V value);
+    public boolean isEmpty();    
+    public Set<K> keySet();
+    public void shutdown();
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/JenkinsHash.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/JenkinsHash.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/JenkinsHash.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/JenkinsHash.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,185 @@
+package org.apache.cassandra.utils;
+
+class JenkinsHash
+{
+
+	// max value to limit it to 4 bytes
+	private static final long MAX_VALUE = 0xFFFFFFFFL;
+
+	// internal variables used in the various calculations
+	private long a_;
+	private long b_;
+	private long c_;
+
+	/**
+	 * Convert a byte into a long value without making it negative.
+	 */
+	private long byteToLong(byte b)
+	{
+		long val = b & 0x7F;
+		if ((b & 0x80) != 0)
+		{
+			val += 128;
+		}
+		return val;
+	}
+
+	/**
+	 * Do addition and turn into 4 bytes.
+	 */
+	private long add(long val, long add)
+	{
+		return (val + add) & MAX_VALUE;
+	}
+
+	/**
+	 * Do subtraction and turn into 4 bytes.
+	 */
+	private long subtract(long val, long subtract)
+	{
+		return (val - subtract) & MAX_VALUE;
+	}
+
+	/**
+	 * Left shift val by shift bits and turn in 4 bytes.
+	 */
+	private long xor(long val, long xor)
+	{
+		return (val ^ xor) & MAX_VALUE;
+	}
+
+	/**
+	 * Left shift val by shift bits. Cut down to 4 bytes.
+	 */
+	private long leftShift(long val, int shift)
+	{
+		return (val << shift) & MAX_VALUE;
+	}
+
+	/**
+	 * Convert 4 bytes from the buffer at offset into a long value.
+	 */
+	private long fourByteToLong(byte[] bytes, int offset)
+	{
+		return (byteToLong(bytes[offset + 0])
+				+ (byteToLong(bytes[offset + 1]) << 8)
+				+ (byteToLong(bytes[offset + 2]) << 16) + (byteToLong(bytes[offset + 3]) << 24));
+	}
+
+	/**
+	 * Mix up the values in the hash function.
+	 */
+	private void hashMix()
+	{
+		a_ = subtract(a_, b_);
+		a_ = subtract(a_, c_);
+		a_ = xor(a_, c_ >> 13);
+		b_ = subtract(b_, c_);
+		b_ = subtract(b_, a_);
+		b_ = xor(b_, leftShift(a_, 8));
+		c_ = subtract(c_, a_);
+		c_ = subtract(c_, b_);
+		c_ = xor(c_, (b_ >> 13));
+		a_ = subtract(a_, b_);
+		a_ = subtract(a_, c_);
+		a_ = xor(a_, (c_ >> 12));
+		b_ = subtract(b_, c_);
+		b_ = subtract(b_, a_);
+		b_ = xor(b_, leftShift(a_, 16));
+		c_ = subtract(c_, a_);
+		c_ = subtract(c_, b_);
+		c_ = xor(c_, (b_ >> 5));
+		a_ = subtract(a_, b_);
+		a_ = subtract(a_, c_);
+		a_ = xor(a_, (c_ >> 3));
+		b_ = subtract(b_, c_);
+		b_ = subtract(b_, a_);
+		b_ = xor(b_, leftShift(a_, 10));
+		c_ = subtract(c_, a_);
+		c_ = subtract(c_, b_);
+		c_ = xor(c_, (b_ >> 15));
+	}
+
+	/**
+	 * Hash a variable-length key into a 32-bit value. Every bit of the key
+	 * affects every bit of the return value. Every 1-bit and 2-bit delta
+	 * achieves avalanche. The best hash table sizes are powers of 2.
+	 * 
+	 * @param buffer
+	 *            Byte array that we are hashing on.
+	 * @param initialValue
+	 *            Initial value of the hash if we are continuing from a previous
+	 *            run. 0 if none.
+	 * @return Hash value for the buffer.
+	 */
+	public long hash(byte[] buffer, long initialValue)
+	{
+		int len, pos;
+
+		// set up the internal state
+		// the golden ratio; an arbitrary value
+		a_ = 0x09e3779b9L;
+		// the golden ratio; an arbitrary value
+		b_ = 0x09e3779b9L;
+		// the previous hash value
+		c_ = initialValue;
+
+		// handle most of the key
+		pos = 0;
+		for (len = buffer.length; len >= 12; len -= 12)
+		{
+			a_ = add(a_, fourByteToLong(buffer, pos));
+			b_ = add(b_, fourByteToLong(buffer, pos + 4));
+			c_ = add(c_, fourByteToLong(buffer, pos + 8));
+			hashMix();
+			pos += 12;
+		}
+
+		c_ += buffer.length;
+
+		// all the case statements fall through to the next on purpose
+		switch (len)
+		{
+			case 11:
+				c_ = add(c_, leftShift(byteToLong(buffer[pos + 10]), 24));
+			case 10:
+				c_ = add(c_, leftShift(byteToLong(buffer[pos + 9]), 16));
+			case 9:
+				c_ = add(c_, leftShift(byteToLong(buffer[pos + 8]), 8));
+				// the first byte of c is reserved for the length
+			case 8:
+				b_ = add(b_, leftShift(byteToLong(buffer[pos + 7]), 24));
+			case 7:
+				b_ = add(b_, leftShift(byteToLong(buffer[pos + 6]), 16));
+			case 6:
+				b_ = add(b_, leftShift(byteToLong(buffer[pos + 5]), 8));
+			case 5:
+				b_ = add(b_, byteToLong(buffer[pos + 4]));
+			case 4:
+				a_ = add(a_, leftShift(byteToLong(buffer[pos + 3]), 24));
+			case 3:
+				a_ = add(a_, leftShift(byteToLong(buffer[pos + 2]), 16));
+			case 2:
+				a_ = add(a_, leftShift(byteToLong(buffer[pos + 1]), 8));
+			case 1:
+				a_ = add(a_, byteToLong(buffer[pos + 0]));
+				// case 0: nothing left to add
+		}
+		hashMix();
+
+		return c_;
+	}
+
+	/**
+	 * See hash(byte[] buffer, long initialValue)
+	 * 
+	 * @param buffer
+	 *            Byte array that we are hashing on.
+	 * @return Hash value for the buffer.
+	 */
+	public long hash(byte[] buffer)
+	{
+		return hash(buffer, 0);
+	}
+}
+

Added: incubator/cassandra/src/org/apache/cassandra/utils/Log4jLogger.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/Log4jLogger.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/Log4jLogger.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/Log4jLogger.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+/**
+ * Log4j configurations may change while the application is running, 
+ * potentially invalidating a logger's appender(s).  This is a convinience
+ * class to wrap logger calls so that a logger is always explicitly 
+ * invoked.
+ */
+
+
+public class Log4jLogger {
+    
+    private String name_ = null;
+    
+    public Log4jLogger(String name){
+        name_ = name;
+    }
+    
+    public void debug(Object arg){ 
+        LogUtil.getLogger(name_).debug(LogUtil.getTimestamp() + " - " + arg);
+    }    
+    public void info(Object arg){
+        LogUtil.getLogger(name_).info(LogUtil.getTimestamp() + " - " + arg);
+    }
+    public void warn(Object arg){
+        LogUtil.getLogger(name_).warn(LogUtil.getTimestamp() + " - " + arg);
+    }
+    public void error(Object arg){
+        LogUtil.getLogger(name_).error(LogUtil.getTimestamp() + " - " + arg);
+    }
+    public void fatal(Object arg){
+        LogUtil.getLogger(name_).fatal(LogUtil.getTimestamp() + " - " + arg);
+    } 
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/LogUtil.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/LogUtil.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/LogUtil.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/LogUtil.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.io.*;
+import java.text.*;
+import java.util.*;
+import org.apache.log4j.*;
+import org.apache.log4j.spi.LoggerFactory;
+import org.apache.log4j.xml.DOMConfigurator;
+
+public class LogUtil
+{
+
+    public LogUtil()
+    {
+    }
+
+    public static void init()
+    {
+        //BasicConfigurator.configure();
+        String file = System.getProperty("storage-config");
+        file += System.getProperty("file.separator") + "log4j.properties";
+        PropertyConfigurator.configure(file);
+    }
+
+    public static Logger getLogger(String name)
+    {
+        return Logger.getLogger(name);
+    }
+    
+    public static String stackTrace(Throwable e)
+    {
+        StringWriter sw = new StringWriter();
+        PrintWriter pw = new PrintWriter(sw);
+        e.printStackTrace(pw);
+        return sw.toString();
+    }
+
+    public static String getTimestamp()
+    {
+        Date date = new Date();
+        DateFormat df = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss");
+        return df.format(date);
+    }
+    
+    public static String throwableToString(Throwable e)
+    {
+        StringBuffer sbuf = new StringBuffer("");
+        String trace = stackTrace(e);
+        sbuf.append((new StringBuilder()).append("Exception was generated at : ").append(getTimestamp()).append(" on thread ").append(Thread.currentThread().getName()).toString());
+        sbuf.append(System.getProperty("line.separator"));
+        String message = e.getMessage();
+        if(message != null)
+            sbuf.append(message);
+        sbuf.append(System.getProperty("line.separator"));
+        sbuf.append(trace);
+        return sbuf.toString();
+    }
+
+    public static String getLogMessage(String message)
+    {
+        StringBuffer sbuf = new StringBuffer((new StringBuilder()).append("Log started at : ").append(getTimestamp()).toString());
+        sbuf.append(System.getProperty("line.separator"));
+        sbuf.append(message);
+        return sbuf.toString();
+    }
+
+    public static void setLogLevel(String logger, String level)
+    {        
+        Logger loggerObj = LogManager.getLogger(logger);
+        if(null == loggerObj)
+            return;
+        level = level.toUpperCase();
+        if(level.equals("DEBUG"))
+            loggerObj.setLevel(Level.DEBUG);
+        else
+        if(level.equals("ERROR"))
+            loggerObj.setLevel(Level.ERROR);
+        else
+        if(level.equals("FATAL"))
+            loggerObj.setLevel(Level.FATAL);
+        else
+        if(level.equals("INFO"))
+            loggerObj.setLevel(Level.INFO);
+        else
+        if(level.equals("OFF"))
+            loggerObj.setLevel(Level.OFF);
+        else
+        if(level.equals("WARN"))
+            loggerObj.setLevel(Level.WARN);
+        else
+            loggerObj.setLevel(Level.ALL);
+    }
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/PrimeFinder.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/PrimeFinder.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/PrimeFinder.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/PrimeFinder.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,144 @@
+package org.apache.cassandra.utils;
+
+import java.util.Arrays;
+
+/**
+ * Used to keep hash table capacities prime numbers. Not of interest for users;
+ * only for implementors of hashtables.
+ * 
+ * <p>
+ * Choosing prime numbers as hash table capacities is a good idea to keep them
+ * working fast, particularly under hash table expansions.
+ * 
+ */
+public final class PrimeFinder
+{
+    /**
+     * The largest prime this class can generate; currently equal to
+     * <tt>Integer.MAX_VALUE</tt>.
+     */
+    public static final int largestPrime = Integer.MAX_VALUE; // yes, it is
+                                                                // prime.
+
+    /**
+     * The prime number list consists of 11 chunks.
+     * 
+     * Each chunk contains prime numbers.
+     * 
+     * A chunk starts with a prime P1. The next element is a prime P2. P2 is the
+     * smallest prime for which holds: P2 >= 2*P1.
+     * 
+     * The next element is P3, for which the same holds with respect to P2, and
+     * so on.
+     * 
+     * Chunks are chosen such that for any desired capacity >= 1000 the list
+     * includes a prime number <= desired capacity * 1.11.
+     * 
+     * Therefore, primes can be retrieved which are quite close to any desired
+     * capacity, which in turn avoids wasting memory.
+     * 
+     * For example, the list includes
+     * 1039,1117,1201,1277,1361,1439,1523,1597,1759,1907,2081.
+     * 
+     * So if you need a prime >= 1040, you will find a prime <= 1040*1.11=1154.
+     * 
+     * Chunks are chosen such that they are optimized for a hashtable
+     * growthfactor of 2.0;
+     * 
+     * If your hashtable has such a growthfactor then, after initially "rounding
+     * to a prime" upon hashtable construction, it will later expand to prime
+     * capacities such that there exist no better primes.
+     * 
+     * In total these are about 32*10=320 numbers -> 1 KB of static memory
+     * needed.
+     * 
+     * If you are stingy, then delete every second or fourth chunk.
+     */
+
+    private static final int[] primeCapacities = {
+    // chunk #0
+            largestPrime,
+
+            // chunk #1
+            5, 11, 23, 47, 97, 197, 397, 797, 1597, 3203, 6421, 12853, 25717,
+            51437, 102877, 205759, 411527, 823117, 1646237, 3292489, 6584983,
+            13169977, 26339969, 52679969, 105359939, 210719881, 421439783,
+            842879579, 1685759167,
+
+            // chunk #2
+            433, 877, 1759, 3527, 7057, 14143, 28289, 56591, 113189, 226379,
+            452759, 905551, 1811107, 3622219, 7244441, 14488931, 28977863,
+            57955739, 115911563, 231823147, 463646329, 927292699, 1854585413,
+
+            // chunk #3
+            953, 1907, 3821, 7643, 15287, 30577, 61169, 122347, 244703, 489407,
+            978821, 1957651, 3915341, 7830701, 15661423, 31322867, 62645741,
+            125291483, 250582987, 501165979, 1002331963, 2004663929,
+
+            // chunk #4
+            1039, 2081, 4177, 8363, 16729, 33461, 66923, 133853, 267713,
+            535481, 1070981, 2141977, 4283963, 8567929, 17135863, 34271747,
+            68543509, 137087021, 274174111, 548348231, 1096696463,
+
+            // chunk #5
+            31, 67, 137, 277, 557, 1117, 2237, 4481, 8963, 17929, 35863, 71741,
+            143483, 286973, 573953, 1147921, 2295859, 4591721, 9183457,
+            18366923, 36733847, 73467739, 146935499, 293871013, 587742049,
+            1175484103,
+
+            // chunk #6
+            599, 1201, 2411, 4831, 9677, 19373, 38747, 77509, 155027, 310081,
+            620171, 1240361, 2480729, 4961459, 9922933, 19845871, 39691759,
+            79383533, 158767069, 317534141, 635068283, 1270136683,
+
+            // chunk #7
+            311, 631, 1277, 2557, 5119, 10243, 20507, 41017, 82037, 164089,
+            328213, 656429, 1312867, 2625761, 5251529, 10503061, 21006137,
+            42012281, 84024581, 168049163, 336098327, 672196673, 1344393353,
+
+            // chunk #8
+            3, 7, 17, 37, 79, 163, 331, 673, 1361, 2729, 5471, 10949, 21911,
+            43853, 87719, 175447, 350899, 701819, 1403641, 2807303, 5614657,
+            11229331, 22458671, 44917381, 89834777, 179669557, 359339171,
+            718678369, 1437356741,
+
+            // chunk #9
+            43, 89, 179, 359, 719, 1439, 2879, 5779, 11579, 23159, 46327,
+            92657, 185323, 370661, 741337, 1482707, 2965421, 5930887, 11861791,
+            23723597, 47447201, 94894427, 189788857, 379577741, 759155483,
+            1518310967,
+
+            // chunk #10
+            379, 761, 1523, 3049, 6101, 12203, 24407, 48817, 97649, 195311,
+            390647, 781301, 1562611, 3125257, 6250537, 12501169, 25002389,
+            50004791, 100009607, 200019221, 400038451, 800076929, 1600153859 };
+
+    static
+    { // initializer
+        // The above prime numbers are formatted for human readability.
+        // To find numbers fast, we sort them once and for all.
+
+        Arrays.sort(primeCapacities);
+    }
+
+    /**
+     * Returns a prime number which is <code>&gt;= desiredCapacity</code> and
+     * very close to <code>desiredCapacity</code> (within 11% if
+     * <code>desiredCapacity &gt;= 1000</code>).
+     * 
+     * @param desiredCapacity
+     *            the capacity desired by the user.
+     * @return the capacity which should be used for a hashtable.
+     */
+    public static final int nextPrime(int desiredCapacity)
+    {
+        int i = Arrays.binarySearch(primeCapacities, desiredCapacity);
+        if (i < 0)
+        {
+            // desired capacity not found, choose next prime greater
+            // than desired capacity
+            i = -i - 1; // remember the semantics of binarySearch...
+        }
+        return primeCapacities[i];
+    }
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/SafeMessageDigest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/SafeMessageDigest.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/SafeMessageDigest.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/SafeMessageDigest.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+/**
+ * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
+ */
+
+public class SafeMessageDigest
+{
+    private MessageDigest md_ = null;
+
+    public static SafeMessageDigest digest_;
+    static
+    {
+        try
+        {
+            digest_ = new SafeMessageDigest(MessageDigest.getInstance("SHA-1"));
+        }
+        catch (NoSuchAlgorithmException e)
+        {
+            assert (false);
+        }
+    }
+
+    public SafeMessageDigest(MessageDigest md)
+    {
+        md_ = md;
+    }
+
+    public synchronized void update(byte[] theBytes)
+    {
+        md_.update(theBytes);
+    }
+
+    //NOTE: This should be used instead of seperate update() and then digest()
+    public synchronized byte[] digest(byte[] theBytes)
+    {
+        //this does an implicit update()
+        return md_.digest(theBytes);
+    }
+
+    public synchronized byte[] digest()
+    {
+        return md_.digest();
+    }
+
+    public byte[] unprotectedDigest()
+    {
+        return md_.digest();
+    }
+
+    public void unprotectedUpdate(byte[] theBytes)
+    {
+        md_.update(theBytes);
+    }
+
+    public byte[] unprotectedDigest(byte[] theBytes)
+    {
+        return md_.digest(theBytes);
+    }
+
+    public int getDigestLength()
+    {
+        return md_.getDigestLength();
+    }
+}

Added: incubator/cassandra/src/org/apache/cassandra/utils/XMLUtils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/src/org/apache/cassandra/utils/XMLUtils.java?rev=749204&view=auto
==============================================================================
--- incubator/cassandra/src/org/apache/cassandra/utils/XMLUtils.java (added)
+++ incubator/cassandra/src/org/apache/cassandra/utils/XMLUtils.java Mon Mar  2 06:11:20 2009
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.utils;
+
+import java.util.*;
+import javax.xml.parsers.*;
+import javax.xml.transform.*;
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpression;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+import java.io.*;
+import org.w3c.dom.*;
+import org.xml.sax.*;
+
+/**
+ * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
+ */
+
+public class XMLUtils
+{
+	private Document document_;
+    private XPath xpath_;
+
+    public XMLUtils(String xmlSrc) throws FileNotFoundException, ParserConfigurationException, SAXException, IOException
+    {        
+        DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+        DocumentBuilder db = dbf.newDocumentBuilder();
+        document_ = db.parse(xmlSrc);
+        
+        XPathFactory xpathFactory = XPathFactory.newInstance();
+        xpath_ = xpathFactory.newXPath();
+    }
+
+	public String getNodeValue(String xql) throws XPathExpressionException
+	{        
+        XPathExpression expr = xpath_.compile(xql);
+        String value = expr.evaluate(document_);
+        if ( value != null && value.equals("") )
+            value = null;
+        return value;	
+    }
+        
+	public String[] getNodeValues(String xql) throws XPathExpressionException
+	{
+        XPathExpression expr = xpath_.compile(xql);        
+        NodeList nl = (NodeList)expr.evaluate(document_, XPathConstants.NODESET);
+        int size = nl.getLength();
+        String[] values = new String[size];
+        
+        for ( int i = 0; i < size; ++i )
+        {
+            Node node = nl.item(i);
+            node = node.getFirstChild();
+            values[i] = node.getNodeValue();
+        }
+        return values;       		
+	}
+
+	public NodeList getRequestedNodeList(String xql) throws XPathExpressionException
+	{
+        XPathExpression expr = xpath_.compile(xql);
+        NodeList nodeList = (NodeList)expr.evaluate(document_, XPathConstants.NODESET);		
+		return nodeList;
+	}
+
+	public static String getAttributeValue(Node node, String attrName) throws TransformerException
+	{        
+		String value = null;
+		node = node.getAttributes().getNamedItem(attrName);
+		if ( node != null )
+		{
+		    value = node.getNodeValue();
+		}
+		return value;
+	}
+
+    public static void main(String[] args) throws Throwable
+    {
+        XMLUtils xmlUtils = new XMLUtils("C:\\Engagements\\Cassandra-Golden\\storage-conf.xml");
+        String[] value = xmlUtils.getNodeValues("/Storage/Seeds/Seed");
+        System.out.println(value);
+    }
+}