You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2010/10/22 17:46:50 UTC

svn commit: r1026380 - in /cassandra/trunk: contrib/bmt_example/ src/java/org/apache/cassandra/cli/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/db/marshal/ src/java/org/apache/cassandra/dht/ src/java/org/apache/cassandra/utils/ ...

Author: jbellis
Date: Fri Oct 22 15:46:49 2010
New Revision: 1026380

URL: http://svn.apache.org/viewvc?rev=1026380&view=rev
Log:
update CassandraBulkLoader for Thrift 0.5; replace UnsupportedEncodingException catches w/ Charsets.UTF_8.  patch by jbellis

Modified:
    cassandra/trunk/contrib/bmt_example/CassandraBulkLoader.java
    cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java
    cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    cassandra/trunk/src/java/org/apache/cassandra/db/marshal/AsciiType.java
    cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
    cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
    cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
    cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java

Modified: cassandra/trunk/contrib/bmt_example/CassandraBulkLoader.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/contrib/bmt_example/CassandraBulkLoader.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/contrib/bmt_example/CassandraBulkLoader.java (original)
+++ cassandra/trunk/contrib/bmt_example/CassandraBulkLoader.java Fri Oct 22 15:46:49 2010
@@ -51,6 +51,7 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -58,6 +59,8 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import com.google.common.base.Charsets;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Column;
@@ -70,6 +73,7 @@ import org.apache.cassandra.net.IAsyncRe
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.Path;
@@ -170,7 +174,11 @@ public class CassandraBulkLoader {
                 String ColumnName = fields[2];
                 String ColumnValue = fields[3];
                 int timestamp = 0;
-                columnFamily.addColumn(new QueryPath(cfName, SuperColumnName.getBytes("UTF-8"), ColumnName.getBytes("UTF-8")), ColumnValue.getBytes(), timestamp);
+                columnFamily.addColumn(new QueryPath(cfName,
+                                                     ByteBuffer.wrap(SuperColumnName.getBytes(Charsets.UTF_8)),
+                                                     ByteBuffer.wrap(ColumnName.getBytes(Charsets.UTF_8))), 
+                                       ByteBuffer.wrap(ColumnValue.getBytes()),
+                                       timestamp);
             }
 
             columnFamilies.add(columnFamily);
@@ -178,7 +186,7 @@ public class CassandraBulkLoader {
             /* Get serialized message to send to cluster */
             message = createMessage(keyspace, key.getBytes(), cfName, columnFamilies);
             List<IAsyncResult> results = new ArrayList<IAsyncResult>();
-            for (InetAddress endpoint: StorageService.instance.getNaturalEndpoints(keyspace, key.getBytes()))
+            for (InetAddress endpoint: StorageService.instance.getNaturalEndpoints(keyspace, ByteBuffer.wrap(key.getBytes())))
             {
                 /* Send message to end point */
                 results.add(MessagingService.instance.sendRR(message, endpoint));
@@ -239,7 +247,7 @@ public class CassandraBulkLoader {
         }
     }
 
-    public static Message createMessage(String Keyspace, byte[] Key, String CFName, List<ColumnFamily> ColumnFamiles)
+    public static Message createMessage(String keyspace, byte[] key, String columnFamily, List<ColumnFamily> columnFamilies)
     {
         ColumnFamily baseColumnFamily;
         DataOutputBuffer bufOut = new DataOutputBuffer();
@@ -249,20 +257,20 @@ public class CassandraBulkLoader {
 
         /* Get the first column family from list, this is just to get past validation */
         baseColumnFamily = new ColumnFamily(ColumnFamilyType.Standard,
-                                            DatabaseDescriptor.getComparator(Keyspace, CFName),
-                                            DatabaseDescriptor.getSubComparator(Keyspace, CFName),
-                                            CFMetaData.getId(Keyspace, CFName));
+                                            DatabaseDescriptor.getComparator(keyspace, columnFamily),
+                                            DatabaseDescriptor.getSubComparator(keyspace, columnFamily),
+                                            CFMetaData.getId(keyspace, columnFamily));
         
-        for(ColumnFamily cf : ColumnFamiles) {
+        for(ColumnFamily cf : columnFamilies) {
             bufOut.reset();
             ColumnFamily.serializer().serializeWithIndexes(cf, bufOut);
             byte[] data = new byte[bufOut.getLength()];
             System.arraycopy(bufOut.getData(), 0, data, 0, bufOut.getLength());
 
-            column = new Column(FBUtilities.toByteBuffer(cf.id()), data, 0);
+            column = new Column(FBUtilities.toByteBuffer(cf.id()), ByteBuffer.wrap(data), 0);
             baseColumnFamily.addColumn(column);
         }
-        rm = new RowMutation(Keyspace, Key);
+        rm = new RowMutation(keyspace, ByteBuffer.wrap(key));
         rm.add(baseColumnFamily);
 
         try

Modified: cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java Fri Oct 22 15:46:49 2010
@@ -17,11 +17,12 @@
  */
 package org.apache.cassandra.cli;
 
-import java.io.UnsupportedEncodingException;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.base.Charsets;
+
 import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.Tree;
 import org.apache.cassandra.auth.SimpleAuthenticator;
@@ -151,83 +152,76 @@ public class CliClient 
     {
         CommonTree ast = CliCompiler.compileQuery(stmt);
 
-        try
+        switch (ast.getType())
         {
-            switch (ast.getType())
-            {
-                case CliParser.NODE_EXIT:
-                    cleanupAndExit();
-                    break;
-                case CliParser.NODE_THRIFT_GET:
-                    executeGet(ast);
-                    break;
-                case CliParser.NODE_THRIFT_GET_WITH_CONDITIONS:
-                    executeGetWithConditions(ast);
-                    break;       
-                case CliParser.NODE_HELP:
-                	printCmdHelp(ast);
-                    break;
-                case CliParser.NODE_THRIFT_SET:
-                    executeSet(ast);
-                    break;
-                case CliParser.NODE_THRIFT_DEL:
-                    executeDelete(ast);
-                    break;
-                case CliParser.NODE_THRIFT_COUNT:
-                    executeCount(ast);
-                    break;
-                case CliParser.NODE_ADD_KEYSPACE:
-                    executeAddKeyspace(ast.getChild(0));
-                    break;
-                case CliParser.NODE_ADD_COLUMN_FAMILY:
-                    executeAddColumnFamily(ast.getChild(0));
-                    break;
-                case CliParser.NODE_UPDATE_KEYSPACE:
-                    executeUpdateKeyspace(ast.getChild(0));
-                    break;
-                case CliParser.NODE_UPDATE_COLUMN_FAMILY:
-                    executeUpdateColumnFamily(ast.getChild(0));
-                    break;
-                case CliParser.NODE_DEL_COLUMN_FAMILY:
-                    executeDelColumnFamily(ast);
-                    break;
-                case CliParser.NODE_DEL_KEYSPACE:
-                    executeDelKeyspace(ast);
-                    break;
-                case CliParser.NODE_SHOW_CLUSTER_NAME:
-                    executeShowClusterName();
-                    break;
-                case CliParser.NODE_SHOW_VERSION:
-                    executeShowVersion();
-                    break;
-                case CliParser.NODE_SHOW_TABLES:
-                    executeShowTables();
-                    break;
-                case CliParser.NODE_DESCRIBE_TABLE:
-                    executeDescribeTable(ast);
-                    break;
-                case CliParser.NODE_USE_TABLE:
-                	executeUseTable(ast);
-                	break;
-                case CliParser.NODE_CONNECT:
-                    executeConnect(ast);
-                    break;
-                case CliParser.NODE_LIST:
-                	executeList(ast);
-                    break;
-                case CliParser.NODE_NO_OP:
-                    // comment lines come here; they are treated as no ops.
-                    break;
-                default:
-                    css_.err.println("Invalid Statement (Type: " + ast.getType() + ")");
-                    if (css_.batch)
-                        System.exit(2);
-                    break;
-            }
-        }
-        catch (UnsupportedEncodingException e)
-        {
-            throw new RuntimeException("Unable to encode string as UTF-8", e);
+            case CliParser.NODE_EXIT:
+                cleanupAndExit();
+                break;
+            case CliParser.NODE_THRIFT_GET:
+                executeGet(ast);
+                break;
+            case CliParser.NODE_THRIFT_GET_WITH_CONDITIONS:
+                executeGetWithConditions(ast);
+                break;
+            case CliParser.NODE_HELP:
+                printCmdHelp(ast);
+                break;
+            case CliParser.NODE_THRIFT_SET:
+                executeSet(ast);
+                break;
+            case CliParser.NODE_THRIFT_DEL:
+                executeDelete(ast);
+                break;
+            case CliParser.NODE_THRIFT_COUNT:
+                executeCount(ast);
+                break;
+            case CliParser.NODE_ADD_KEYSPACE:
+                executeAddKeyspace(ast.getChild(0));
+                break;
+            case CliParser.NODE_ADD_COLUMN_FAMILY:
+                executeAddColumnFamily(ast.getChild(0));
+                break;
+            case CliParser.NODE_UPDATE_KEYSPACE:
+                executeUpdateKeyspace(ast.getChild(0));
+                break;
+            case CliParser.NODE_UPDATE_COLUMN_FAMILY:
+                executeUpdateColumnFamily(ast.getChild(0));
+                break;
+            case CliParser.NODE_DEL_COLUMN_FAMILY:
+                executeDelColumnFamily(ast);
+                break;
+            case CliParser.NODE_DEL_KEYSPACE:
+                executeDelKeyspace(ast);
+                break;
+            case CliParser.NODE_SHOW_CLUSTER_NAME:
+                executeShowClusterName();
+                break;
+            case CliParser.NODE_SHOW_VERSION:
+                executeShowVersion();
+                break;
+            case CliParser.NODE_SHOW_TABLES:
+                executeShowTables();
+                break;
+            case CliParser.NODE_DESCRIBE_TABLE:
+                executeDescribeTable(ast);
+                break;
+            case CliParser.NODE_USE_TABLE:
+                executeUseTable(ast);
+                break;
+            case CliParser.NODE_CONNECT:
+                executeConnect(ast);
+                break;
+            case CliParser.NODE_LIST:
+                executeList(ast);
+                break;
+            case CliParser.NODE_NO_OP:
+                // comment lines come here; they are treated as no ops.
+                break;
+            default:
+                css_.err.println("Invalid Statement (Type: " + ast.getType() + ")");
+                if (css_.batch)
+                    System.exit(2);
+                break;
         }
     }
 
@@ -508,7 +502,7 @@ public class CliClient 
         return keyspacesMap.get(keyspace);
     }
     
-    private void executeCount(CommonTree ast) throws TException, InvalidRequestException, UnavailableException, TimedOutException, UnsupportedEncodingException
+    private void executeCount(CommonTree ast) throws TException, InvalidRequestException, UnavailableException, TimedOutException
     {
        if (!CliMain.isConnected() || !hasKeySpace())
            return;
@@ -532,17 +526,17 @@ public class CliClient 
        else
        {
            assert (columnSpecCnt == 1);
-           colParent = new ColumnParent(columnFamily).setSuper_column(CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8"));
+           colParent = new ColumnParent(columnFamily).setSuper_column(CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8));
        }
 
        SliceRange range = new SliceRange(FBUtilities.EMPTY_BYTE_BUFFER, FBUtilities.EMPTY_BYTE_BUFFER, false, Integer.MAX_VALUE);
        SlicePredicate predicate = new SlicePredicate().setColumn_names(null).setSlice_range(range);
        
-       int count = thriftClient_.get_count(ByteBuffer.wrap(key.getBytes("UTF-8")), colParent, predicate, ConsistencyLevel.ONE);
+       int count = thriftClient_.get_count(ByteBuffer.wrap(key.getBytes(Charsets.UTF_8)), colParent, predicate, ConsistencyLevel.ONE);
        css_.out.printf("%d columns\n", count);
     }
     
-    private void executeDelete(CommonTree ast) throws TException, InvalidRequestException, UnavailableException, TimedOutException, UnsupportedEncodingException
+    private void executeDelete(CommonTree ast) throws TException, InvalidRequestException, UnavailableException, TimedOutException
     {
         if (!CliMain.isConnected() || !hasKeySpace())
             return;
@@ -585,15 +579,15 @@ public class CliClient 
         {
             // table.cf['key']['column']
             if (isSuper)
-                superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8");
+                superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8);
             else
-                columnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8");
+                columnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8);
         }
         else if (columnSpecCnt == 2)
         {
             // table.cf['key']['column']['column']
-            superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8");
-            columnName = CliCompiler.getColumn(columnFamilySpec, 1).getBytes("UTF-8");
+            superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8);
+            columnName = CliCompiler.getColumn(columnFamilySpec, 1).getBytes(Charsets.UTF_8);
         }
 
         ColumnPath path = new ColumnPath(columnFamily);
@@ -603,13 +597,13 @@ public class CliClient 
         if(columnName != null)
             path.setColumn(columnName);
         
-        thriftClient_.remove(ByteBuffer.wrap(key.getBytes("UTF-8")), path,
+        thriftClient_.remove(ByteBuffer.wrap(key.getBytes(Charsets.UTF_8)), path,
                              FBUtilities.timestampMicros(), ConsistencyLevel.ONE);
         css_.out.println(String.format("%s removed.", (columnSpecCnt == 0) ? "row" : "column"));
     }
 
     private void doSlice(String keyspace, String key, String columnFamily, byte[] superColumnName)
-            throws InvalidRequestException, UnavailableException, TimedOutException, TException, UnsupportedEncodingException, IllegalAccessException, NotFoundException, InstantiationException, NoSuchFieldException
+            throws InvalidRequestException, UnavailableException, TimedOutException, TException, IllegalAccessException, NotFoundException, InstantiationException, NoSuchFieldException
     {
         
         ColumnParent parent = new ColumnParent(columnFamily);
@@ -617,7 +611,7 @@ public class CliClient 
             parent.setSuper_column(superColumnName);
                 
         SliceRange range = new SliceRange(FBUtilities.EMPTY_BYTE_BUFFER, FBUtilities.EMPTY_BYTE_BUFFER, true, 1000000);
-        List<ColumnOrSuperColumn> columns = thriftClient_.get_slice(ByteBuffer.wrap(key.getBytes("UTF-8")),parent,
+        List<ColumnOrSuperColumn> columns = thriftClient_.get_slice(ByteBuffer.wrap(key.getBytes(Charsets.UTF_8)),parent,
                                                                     new SlicePredicate().setColumn_names(null).setSlice_range(range), ConsistencyLevel.ONE);
         int size = columns.size();
 
@@ -693,7 +687,7 @@ public class CliClient 
     }
 
     // Execute GET statement
-    private void executeGet(CommonTree ast) throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException, UnsupportedEncodingException, IllegalAccessException, InstantiationException, ClassNotFoundException, NoSuchFieldException
+    private void executeGet(CommonTree ast) throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException, IllegalAccessException, InstantiationException, ClassNotFoundException, NoSuchFieldException
     {
         if (!CliMain.isConnected() || !hasKeySpace())
             return;
@@ -721,7 +715,7 @@ public class CliClient 
         {
             if (isSuper)
             {
-                superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8");
+                superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8);
                 doSlice(keySpace, key, columnFamily, superColumnName);
                 return;
             }
@@ -733,7 +727,7 @@ public class CliClient 
         // table.cf['key']['column']['column'] -- get of a sub-column
         else if (columnSpecCnt == 2)
         {
-            superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8");
+            superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8);
             columnName = CliCompiler.getColumn(columnFamilySpec, 1);
         }
         // The parser groks an arbitrary number of these so it is possible to get here.
@@ -750,7 +744,7 @@ public class CliClient 
         ColumnPath path = new ColumnPath(columnFamily);
         if(superColumnName != null) path.setSuper_column(superColumnName);
         if(columnNameInBytes != null) path.setColumn(columnNameInBytes);
-        Column column = thriftClient_.get(ByteBuffer.wrap(key.getBytes("UTF-8")), path, ConsistencyLevel.ONE).column;
+        Column column = thriftClient_.get(ByteBuffer.wrap(key.getBytes(Charsets.UTF_8)), path, ConsistencyLevel.ONE).column;
 
         byte[] columnValue = column.getValue();       
         String valueAsString;
@@ -774,7 +768,7 @@ public class CliClient 
         }
         else
         {
-            valueAsString = (validator == null) ? new String(columnValue, "UTF-8") : validator.getString(ByteBuffer.wrap(columnValue));
+            valueAsString = (validator == null) ? new String(columnValue, Charsets.UTF_8) : validator.getString(ByteBuffer.wrap(columnValue));
         }
 
         // print results
@@ -878,7 +872,7 @@ public class CliClient 
 
     // Execute SET statement
     private void executeSet(CommonTree ast)
-    throws TException, InvalidRequestException, UnavailableException, TimedOutException, UnsupportedEncodingException, NoSuchFieldException, InstantiationException, IllegalAccessException
+    throws TException, InvalidRequestException, UnavailableException, TimedOutException, NoSuchFieldException, InstantiationException, IllegalAccessException
     {
         if (!CliMain.isConnected() || !hasKeySpace())
             return;
@@ -913,7 +907,7 @@ public class CliClient 
             assert (columnSpecCnt == 2) : "serious parsing error (this is a bug).";
             
             // get the super column and column names
-            superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes("UTF-8");
+            superColumnName = CliCompiler.getColumn(columnFamilySpec, 0).getBytes(Charsets.UTF_8);
             columnName = CliCompiler.getColumn(columnFamilySpec, 1);
         }
 
@@ -935,7 +929,7 @@ public class CliClient 
             parent.setSuper_column(superColumnName);
         
         // do the insert
-        thriftClient_.insert(ByteBuffer.wrap(key.getBytes("UTF-8")), parent,
+        thriftClient_.insert(ByteBuffer.wrap(key.getBytes(Charsets.UTF_8)), parent,
                              new Column(columnNameInBytes, columnValueInBytes, FBUtilities.timestampMicros()), ConsistencyLevel.ONE);
         
         css_.out.println("Value inserted.");
@@ -1231,7 +1225,7 @@ public class CliClient 
     }
 
     private void executeList(CommonTree ast)
-    throws TException, InvalidRequestException, NotFoundException, IllegalAccessException, InstantiationException, NoSuchFieldException, UnavailableException, TimedOutException, UnsupportedEncodingException
+    throws TException, InvalidRequestException, NotFoundException, IllegalAccessException, InstantiationException, NoSuchFieldException, UnavailableException, TimedOutException
     {
         if (!CliMain.isConnected())
             return;
@@ -1608,14 +1602,7 @@ public class CliClient 
 
                 if (metaKey.equals("column_name"))
                 {
-                    try
-                    {
-                        columnDefinition.setName(metaVal.getBytes("UTF-8"));
-                    }
-                    catch (UnsupportedEncodingException e)
-                    {
-                        throw new RuntimeException(e.getMessage(), e);
-                    }
+                    columnDefinition.setName(metaVal.getBytes(Charsets.UTF_8));
                 }
                 else if (metaKey.equals("validation_class"))
                 {
@@ -1678,9 +1665,8 @@ public class CliClient 
      * @param object - object to covert into byte array
      * @param comparator - comparator used to convert object
      * @return byte[] - object in the byte array representation
-     * @throws UnsupportedEncodingException - raised but String.getBytes(encoding)
      */
-    private ByteBuffer getBytesAccordingToType(String object, AbstractType comparator) throws UnsupportedEncodingException
+    private ByteBuffer getBytesAccordingToType(String object, AbstractType comparator)
     {
         if (comparator instanceof LongType)
         {
@@ -1722,11 +1708,11 @@ public class CliClient 
         }
         else if (comparator instanceof AsciiType)
         {
-            return ByteBuffer.wrap(object.getBytes("US-ASCII"));
+            return ByteBuffer.wrap(object.getBytes(Charsets.US_ASCII));
         }
         else
         {
-            return ByteBuffer.wrap(object.getBytes("UTF-8"));
+            return ByteBuffer.wrap(object.getBytes(Charsets.UTF_8));
         }
     }
     
@@ -1738,9 +1724,8 @@ public class CliClient 
      * @throws NoSuchFieldException - raised from getFormatTypeForColumn call
      * @throws InstantiationException - raised from getFormatTypeForColumn call
      * @throws IllegalAccessException - raised from getFormatTypeForColumn call
-     * @throws UnsupportedEncodingException - raised from getBytes() calls
      */
-    private ByteBuffer columnNameAsBytes(String column, String columnFamily) throws NoSuchFieldException, InstantiationException, IllegalAccessException, UnsupportedEncodingException
+    private ByteBuffer columnNameAsBytes(String column, String columnFamily) throws NoSuchFieldException, InstantiationException, IllegalAccessException
     {
         CfDef columnFamilyDef   = getCfDef(columnFamily);
         String comparatorClass  = columnFamilyDef.comparator_type;
@@ -1949,7 +1934,6 @@ public class CliClient 
      * Prints out KeySlice list
      * @param columnFamilyDef - column family definition
      * @param slices - list of the KeySlice's to print out
-     * @throws UnsupportedEncodingException -  when trying to covert key
      * @throws NotFoundException - column not found
      * @throws TException - transfer is broken
      * @throws IllegalAccessException - can't do operation
@@ -1957,7 +1941,7 @@ public class CliClient 
      * @throws NoSuchFieldException - column not found
      */
     private void printSliceList(CfDef columnFamilyDef, List<KeySlice> slices)
-            throws UnsupportedEncodingException, NotFoundException, TException, IllegalAccessException, InstantiationException, NoSuchFieldException
+    throws NotFoundException, TException, IllegalAccessException, InstantiationException, NoSuchFieldException
     {
         AbstractType validator;
         String columnFamilyName = columnFamilyDef.getName();
@@ -1965,7 +1949,7 @@ public class CliClient 
         for (KeySlice ks : slices)
         {
             css_.out.printf("-------------------\n");
-            css_.out.printf("RowKey: %s\n", new String(ks.key.array(),ks.key.position(),ks.key.remaining(), "UTF-8"));
+            css_.out.printf("RowKey: %s\n", new String(ks.key.array(),ks.key.position(),ks.key.remaining(), Charsets.UTF_8));
 
             Iterator<ColumnOrSuperColumn> iterator = ks.getColumnsIterator();
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java Fri Oct 22 15:46:49 2010
@@ -18,27 +18,17 @@
 
 package org.apache.cassandra.config;
 
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOError;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.UnsupportedEncodingException;
+import java.io.*;
 import java.net.InetAddress;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
+import java.util.*;
+
+import com.google.common.base.Charsets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.AllowAllAuthenticator;
 import org.apache.cassandra.auth.AllowAllAuthority;
@@ -54,18 +44,12 @@ import org.apache.cassandra.db.migration
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.locator.DynamicEndpointSnitch;
-import org.apache.cassandra.locator.EndpointSnitchInfo;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.locator.*;
 import org.apache.cassandra.scheduler.IRequestScheduler;
 import org.apache.cassandra.scheduler.NoScheduler;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Loader;
 import org.yaml.snakeyaml.TypeDescription;
 import org.yaml.snakeyaml.Yaml;
@@ -595,15 +579,8 @@ public class    DatabaseDescriptor
 
                 for (RawColumnDefinition rcd : cf.column_metadata)
                 {
-                    try
-                    {
-                        ByteBuffer columnName = ByteBuffer.wrap(rcd.name.getBytes("UTF-8"));
-                        metadata.put(columnName, new ColumnDefinition(columnName, rcd.validator_class, rcd.index_type, rcd.index_name));
-                    }
-                    catch (UnsupportedEncodingException e)
-                    {
-                        throw new AssertionError(e);
-                    }
+                    ByteBuffer columnName = ByteBuffer.wrap(rcd.name.getBytes(Charsets.UTF_8));
+                    metadata.put(columnName, new ColumnDefinition(columnName, rcd.validator_class, rcd.index_type, rcd.index_name));
                 }
 
                 cfDefs[j++] = new CFMetaData(keyspace.name, 

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/marshal/AsciiType.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/marshal/AsciiType.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/marshal/AsciiType.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/marshal/AsciiType.java Fri Oct 22 15:46:49 2010
@@ -21,7 +21,6 @@ package org.apache.cassandra.db.marshal;
  */
 
 
-import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
 
 import com.google.common.base.Charsets;

Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java Fri Oct 22 15:46:49 2010
@@ -19,12 +19,12 @@
 package org.apache.cassandra.dht;
 
  import java.io.IOException;
- import java.io.UnsupportedEncodingException;
  import java.net.InetAddress;
  import java.util.*;
  import java.util.concurrent.locks.Condition;
  import java.util.concurrent.CountDownLatch;
 
+ import com.google.common.base.Charsets;
  import com.google.common.collect.ArrayListMultimap;
  import com.google.common.collect.HashMultimap;
  import com.google.common.collect.Multimap;
@@ -253,15 +253,7 @@ public class BootStrapper
         {
             StorageService ss = StorageService.instance;
             String tokenString = StorageService.getPartitioner().getTokenFactory().toString(ss.getBootstrapToken());
-            Message response;
-            try
-            {
-                response = message.getReply(FBUtilities.getLocalAddress(), tokenString.getBytes("UTF-8"));
-            }
-            catch (UnsupportedEncodingException e)
-            {
-                throw new AssertionError();
-            }
+            Message response = message.getReply(FBUtilities.getLocalAddress(), tokenString.getBytes(Charsets.UTF_8));
             MessagingService.instance.sendOneWay(response, message.getFrom());
         }
     }
@@ -286,14 +278,7 @@ public class BootStrapper
 
         public void response(Message msg)
         {
-            try
-            {
-                token = StorageService.getPartitioner().getTokenFactory().fromString(new String(msg.getMessageBody(), "UTF-8"));
-            }
-            catch (UnsupportedEncodingException e)
-            {
-                throw new AssertionError();
-            }
+            token = StorageService.getPartitioner().getTokenFactory().fromString(new String(msg.getMessageBody(), Charsets.UTF_8));
             condition.signalAll();
         }
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java Fri Oct 22 15:46:49 2010
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.dht;
 
-import java.io.UnsupportedEncodingException;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;

Modified: cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/utils/ByteBufferUtil.java Fri Oct 22 15:46:49 2010
@@ -18,7 +18,6 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 

Modified: cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java Fri Oct 22 15:46:49 2010
@@ -27,6 +27,8 @@ import java.util.Arrays;
 import java.util.Random;
 import java.util.UUID;
 
+import com.google.common.base.Charsets;
+
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.junit.Test;
@@ -60,15 +62,15 @@ public class TypeCompareTest
     }
 
     @Test
-    public void testUTF8() throws UnsupportedEncodingException
+    public void testUTF8()
     {
         UTF8Type comparator = new UTF8Type();
         assert comparator.compare(FBUtilities.EMPTY_BYTE_BUFFER, ByteBuffer.wrap("asdf".getBytes())) < 0;
         assert comparator.compare(ByteBuffer.wrap("asdf".getBytes()), FBUtilities.EMPTY_BYTE_BUFFER) > 0;
         assert comparator.compare(FBUtilities.EMPTY_BYTE_BUFFER, FBUtilities.EMPTY_BYTE_BUFFER) == 0;
-        assert comparator.compare(ByteBuffer.wrap("z".getBytes("UTF-8")), ByteBuffer.wrap("a".getBytes("UTF-8"))) > 0;
-        assert comparator.compare(ByteBuffer.wrap("z".getBytes("UTF-8")), ByteBuffer.wrap("z".getBytes("UTF-8"))) == 0;
-        assert comparator.compare(ByteBuffer.wrap("a".getBytes("UTF-8")), ByteBuffer.wrap("z".getBytes("UTF-8"))) < 0;
+        assert comparator.compare(ByteBuffer.wrap("z".getBytes(Charsets.UTF_8)), ByteBuffer.wrap("a".getBytes(Charsets.UTF_8))) > 0;
+        assert comparator.compare(ByteBuffer.wrap("z".getBytes(Charsets.UTF_8)), ByteBuffer.wrap("z".getBytes(Charsets.UTF_8))) == 0;
+        assert comparator.compare(ByteBuffer.wrap("a".getBytes(Charsets.UTF_8)), ByteBuffer.wrap("z".getBytes(Charsets.UTF_8))) < 0;
     }
 
     @Test

Modified: cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java?rev=1026380&r1=1026379&r2=1026380&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java Fri Oct 22 15:46:49 2010
@@ -19,9 +19,9 @@
 package org.apache.cassandra.service;
 
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
 
+import com.google.common.base.Charsets;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -84,8 +84,7 @@ public class EmbeddedCassandraServiceTes
 
     @Test
     public void testEmbeddedCassandraService() throws AuthenticationException, AuthorizationException,
-    UnsupportedEncodingException, InvalidRequestException,
-            UnavailableException, TimedOutException, TException, NotFoundException
+    InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException
     {
         Cassandra.Client client = getClient();
         client.set_keyspace("Keyspace1");
@@ -95,18 +94,18 @@ public class EmbeddedCassandraServiceTes
         long timestamp = System.currentTimeMillis();
         ColumnPath cp = new ColumnPath("Standard1");
         ColumnParent par = new ColumnParent("Standard1");
-        cp.column = ByteBuffer.wrap("name".getBytes("utf-8"));
+        cp.column = ByteBuffer.wrap("name".getBytes(Charsets.UTF_8));
 
         // insert
-        client.insert(key_user_id, par, new Column(ByteBuffer.wrap("name".getBytes("utf-8")),
-                ByteBuffer.wrap( "Ran".getBytes("UTF-8")), timestamp), ConsistencyLevel.ONE);
+        client.insert(key_user_id, par, new Column(ByteBuffer.wrap("name".getBytes(Charsets.UTF_8)),
+                ByteBuffer.wrap( "Ran".getBytes(Charsets.UTF_8)), timestamp), ConsistencyLevel.ONE);
 
         // read
         ColumnOrSuperColumn got = client.get(key_user_id, cp, ConsistencyLevel.ONE);
 
         // assert
         assertNotNull("Got a null ColumnOrSuperColumn", got);
-        assertEquals("Ran", new String(got.getColumn().getValue(), "utf-8"));
+        assertEquals("Ran", new String(got.getColumn().getValue(), Charsets.UTF_8));
     }
 
     /**