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

svn commit: r749218 [34/34] - in /incubator/cassandra: branches/ dist/ nightly/ site/ tags/ trunk/ trunk/lib/ trunk/src/ trunk/src/org/ trunk/src/org/apache/ trunk/src/org/apache/cassandra/ trunk/src/org/apache/cassandra/analytics/ trunk/src/org/apache...

Added: incubator/cassandra/trunk/src/org/apache/cassandra/utils/GuidGenerator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/GuidGenerator.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/GuidGenerator.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/GuidGenerator.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/HashingSchemes.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/HashingSchemes.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/HashingSchemes.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/HashingSchemes.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/ICacheExpungeHook.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/ICacheExpungeHook.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/ICacheExpungeHook.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/ICacheExpungeHook.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/ICachetable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/ICachetable.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/ICachetable.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/ICachetable.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/JenkinsHash.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/JenkinsHash.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/JenkinsHash.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/JenkinsHash.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/Log4jLogger.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/Log4jLogger.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/Log4jLogger.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/Log4jLogger.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/LogUtil.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/LogUtil.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/LogUtil.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/LogUtil.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/PrimeFinder.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/PrimeFinder.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/PrimeFinder.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/PrimeFinder.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/SafeMessageDigest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/SafeMessageDigest.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/SafeMessageDigest.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/SafeMessageDigest.java Mon Mar  2 07:57:22 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/trunk/src/org/apache/cassandra/utils/XMLUtils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/XMLUtils.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/XMLUtils.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/XMLUtils.java Mon Mar  2 07:57:22 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);
+    }
+}