cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gdusba...@apache.org
Subject svn commit: r934155 [2/2] - in /cassandra/trunk: contrib/pig/src/java/org/apache/cassandra/hadoop/pig/ contrib/word_count/src/ interface/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/cli/ src/java/org/apache/cass...
Date Wed, 14 Apr 2010 19:53:24 GMT
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=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java Wed Apr 14 19:53:23 2010
@@ -166,7 +166,7 @@ public class CliClient 
            colParent = new ColumnParent(columnFamily).setSuper_column(CliCompiler.getColumn(columnFamilySpec,
0).getBytes("UTF-8"));
        }
        
-       int count = thriftClient_.get_count(tableName, key, colParent, ConsistencyLevel.ONE);
+       int count = thriftClient_.get_count(tableName, key.getBytes(), colParent, ConsistencyLevel.ONE);
        css_.out.printf("%d columns\n", count);
     }
     
@@ -227,7 +227,7 @@ public class CliClient 
             columnName = CliCompiler.getColumn(columnFamilySpec, 1).getBytes("UTF-8");
         }
 
-        thriftClient_.remove(tableName, key, new ColumnPath(columnFamily).setSuper_column(superColumnName).setColumn(columnName),
+        thriftClient_.remove(tableName, key.getBytes(), new ColumnPath(columnFamily).setSuper_column(superColumnName).setColumn(columnName),
                              timestampMicros(), ConsistencyLevel.ONE);
         css_.out.println(String.format("%s removed.", (columnSpecCnt == 0) ? "row" : "column"));
     }
@@ -243,7 +243,7 @@ public class CliClient 
             throws InvalidRequestException, UnavailableException, TimedOutException, TException,
UnsupportedEncodingException, IllegalAccessException, NotFoundException, InstantiationException,
ClassNotFoundException
     {
         SliceRange range = new SliceRange(ArrayUtils.EMPTY_BYTE_ARRAY, ArrayUtils.EMPTY_BYTE_ARRAY,
true, 1000000);
-        List<ColumnOrSuperColumn> columns = thriftClient_.get_slice(keyspace, key,
+        List<ColumnOrSuperColumn> columns = thriftClient_.get_slice(keyspace, key.getBytes(),
                                                                     new ColumnParent(columnFamily).setSuper_column(superColumnName),
                                                                     new SlicePredicate().setColumn_names(null).setSlice_range(range),
ConsistencyLevel.ONE);
         int size = columns.size();
@@ -363,7 +363,7 @@ public class CliClient 
         
         // Perform a get(), print out the results.
         ColumnPath path = new ColumnPath(columnFamily).setSuper_column(superColumnName).setColumn(columnName);
-        Column column = thriftClient_.get(tableName, key, path, ConsistencyLevel.ONE).column;
+        Column column = thriftClient_.get(tableName, key.getBytes(), path, ConsistencyLevel.ONE).column;
         css_.out.printf("=> (column=%s, value=%s, timestamp=%d)\n", formatColumnName(tableName,
columnFamily, column),
                         new String(column.value, "UTF-8"), column.timestamp);
     }
@@ -411,7 +411,7 @@ public class CliClient 
         }
         
         // do the insert
-        thriftClient_.insert(tableName, key, new ColumnPath(columnFamily).setSuper_column(superColumnName).setColumn(columnName),
+        thriftClient_.insert(tableName, key.getBytes(), new ColumnPath(columnFamily).setSuper_column(superColumnName).setColumn(columnName),
                              value.getBytes(), timestampMicros(), ConsistencyLevel.ONE);
         
         css_.out.println("Value inserted.");

Modified: cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java?rev=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java Wed
Apr 14 19:53:23 2010
@@ -60,7 +60,7 @@ import org.apache.thrift.transport.TTran
  *
  * The default split size is 64k rows.
  */
-public class ColumnFamilyInputFormat extends InputFormat<String, SortedMap<byte[],
IColumn>>
+public class ColumnFamilyInputFormat extends InputFormat<byte[], SortedMap<byte[],
IColumn>>
 {
 
     private static final Logger logger = LoggerFactory.getLogger(StorageService.class);
@@ -218,7 +218,7 @@ public class ColumnFamilyInputFormat ext
     }
 
     @Override
-    public RecordReader<String, SortedMap<byte[], IColumn>> createRecordReader(InputSplit
inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
+    public RecordReader<byte[], SortedMap<byte[], IColumn>> createRecordReader(InputSplit
inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
     {
         return new ColumnFamilyRecordReader();
     }

Modified: cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java?rev=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java Wed
Apr 14 19:53:23 2010
@@ -46,11 +46,11 @@ import org.apache.thrift.protocol.TBinar
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransportException;
 
-public class ColumnFamilyRecordReader extends RecordReader<String, SortedMap<byte[],
IColumn>>
+public class ColumnFamilyRecordReader extends RecordReader<byte[], SortedMap<byte[],
IColumn>>
 {
     private ColumnFamilySplit split;
     private RowIterator iter;
-    private Pair<String, SortedMap<byte[], IColumn>> currentRow;
+    private Pair<byte[], SortedMap<byte[], IColumn>> currentRow;
     private SlicePredicate predicate;
     private int totalRowCount; // total number of rows to fetch
     private int batchRowCount; // fetch this many per batch
@@ -59,7 +59,7 @@ public class ColumnFamilyRecordReader ex
 
     public void close() {}
     
-    public String getCurrentKey()
+    public byte[] getCurrentKey()
     {
         return currentRow.left;
     }
@@ -95,7 +95,7 @@ public class ColumnFamilyRecordReader ex
         return true;
     }
 
-    private class RowIterator extends AbstractIterator<Pair<String, SortedMap<byte[],
IColumn>>>
+    private class RowIterator extends AbstractIterator<Pair<byte[], SortedMap<byte[],
IColumn>>>
     {
 
         private List<KeySlice> rows;
@@ -159,8 +159,7 @@ public class ColumnFamilyRecordReader ex
                 // prepare for the next slice to be read
                 KeySlice lastRow = rows.get(rows.size() - 1);
                 IPartitioner p = DatabaseDescriptor.getPartitioner();
-                // FIXME: thrift strings
-                byte[] rowkey = lastRow.getKey().getBytes(UTF8);
+                byte[] rowkey = lastRow.getKey();
                 startToken = p.getTokenFactory().toString(p.getToken(rowkey));
             }
             catch (Exception e)
@@ -213,7 +212,7 @@ public class ColumnFamilyRecordReader ex
         }
 
         @Override
-        protected Pair<String, SortedMap<byte[], IColumn>> computeNext()
+        protected Pair<byte[], SortedMap<byte[], IColumn>> computeNext()
         {
             maybeInit();
             if (rows == null)
@@ -227,7 +226,7 @@ public class ColumnFamilyRecordReader ex
                 IColumn column = unthriftify(cosc);
                 map.put(column.name(), column);
             }
-            return new Pair<String, SortedMap<byte[], IColumn>>(ks.key, map);
+            return new Pair<byte[], SortedMap<byte[], IColumn>>(ks.key, map);
         }
     }
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java?rev=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java Wed Apr 14 19:53:23
2010
@@ -186,18 +186,17 @@ public class CassandraServer implements 
         return thriftSuperColumns;
     }
 
-    private Map<String, List<ColumnOrSuperColumn>> getSlice(List<ReadCommand>
commands, ConsistencyLevel consistency_level)
+    private Map<byte[], List<ColumnOrSuperColumn>> getSlice(List<ReadCommand>
commands, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         Map<byte[], ColumnFamily> columnFamilies = readColumnFamily(commands, consistency_level);
-        Map<String, List<ColumnOrSuperColumn>> columnFamiliesMap = new HashMap<String,
List<ColumnOrSuperColumn>>();
+        Map<byte[], List<ColumnOrSuperColumn>> columnFamiliesMap = new HashMap<byte[],
List<ColumnOrSuperColumn>>();
         for (ReadCommand command: commands)
         {
             ColumnFamily cf = columnFamilies.get(command.key);
             boolean reverseOrder = command instanceof SliceFromReadCommand && ((SliceFromReadCommand)command).reversed;
             List<ColumnOrSuperColumn> thriftifiedColumns = thriftifyColumnFamily(cf,
command.queryPath.superColumnName != null, reverseOrder);
-            // FIXME: string keys
-            columnFamiliesMap.put(new String(command.key, UTF8), thriftifiedColumns);
+            columnFamiliesMap.put(command.key, thriftifiedColumns);
         }
 
         return columnFamiliesMap;
@@ -222,7 +221,7 @@ public class CassandraServer implements 
             return thriftifyColumns(cf.getSortedColumns(), reverseOrder);
     }
 
-    public List<ColumnOrSuperColumn> get_slice(String keyspace, String key, ColumnParent
column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
+    public List<ColumnOrSuperColumn> get_slice(String keyspace, byte[] key, ColumnParent
column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -232,7 +231,7 @@ public class CassandraServer implements 
         return multigetSliceInternal(keyspace, Arrays.asList(key), column_parent, predicate,
consistency_level).get(key);
     }
     
-    public Map<String, List<ColumnOrSuperColumn>> multiget_slice(String keyspace,
List<String> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel
consistency_level)
+    public Map<byte[], List<ColumnOrSuperColumn>> multiget_slice(String keyspace,
List<byte[]> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel
consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -243,7 +242,7 @@ public class CassandraServer implements 
         return multigetSliceInternal(keyspace, keys, column_parent, predicate, consistency_level);
     }
 
-    private Map<String, List<ColumnOrSuperColumn>> multigetSliceInternal(String
keyspace, List<String> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel
consistency_level)
+    private Map<byte[], List<ColumnOrSuperColumn>> multigetSliceInternal(String
keyspace, List<byte[]> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel
consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         ThriftValidation.validateColumnParent(keyspace, column_parent);
@@ -252,28 +251,26 @@ public class CassandraServer implements 
         List<ReadCommand> commands = new ArrayList<ReadCommand>();
         if (predicate.column_names != null)
         {
-            for (String key: keys)
+            for (byte[] key: keys)
             {
                 ThriftValidation.validateKey(key);
-                // FIXME: string keys
-                commands.add(new SliceByNamesReadCommand(keyspace, key.getBytes(UTF8), column_parent,
predicate.column_names));
+                commands.add(new SliceByNamesReadCommand(keyspace, key, column_parent, predicate.column_names));
             }
         }
         else
         {
             SliceRange range = predicate.slice_range;
-            for (String key: keys)
+            for (byte[] key: keys)
             {
                 ThriftValidation.validateKey(key);
-                // FIXME: string keys
-                commands.add(new SliceFromReadCommand(keyspace, key.getBytes(UTF8), column_parent,
range.start, range.finish, range.reversed, range.count));
+                commands.add(new SliceFromReadCommand(keyspace, key, column_parent, range.start,
range.finish, range.reversed, range.count));
             }
         }
 
         return getSlice(commands, consistency_level);
     }
 
-    public ColumnOrSuperColumn get(String table, String key, ColumnPath column_path, ConsistencyLevel
consistency_level)
+    public ColumnOrSuperColumn get(String table, byte[] key, ColumnPath column_path, ConsistencyLevel
consistency_level)
     throws InvalidRequestException, NotFoundException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -290,7 +287,7 @@ public class CassandraServer implements 
     }
 
     /** always returns a ColumnOrSuperColumn for each key, even if there is no data for it
*/
-    public Map<String, ColumnOrSuperColumn> multiget(String table, List<String>
keys, ColumnPath column_path, ConsistencyLevel consistency_level)
+    public Map<byte[], ColumnOrSuperColumn> multiget(String table, List<byte[]>
keys, ColumnPath column_path, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -301,7 +298,7 @@ public class CassandraServer implements 
         return multigetInternal(table, keys, column_path, consistency_level);
     }
 
-    private Map<String, ColumnOrSuperColumn> multigetInternal(String table, List<String>
keys, ColumnPath column_path, ConsistencyLevel consistency_level)
+    private Map<byte[], ColumnOrSuperColumn> multigetInternal(String table, List<byte[]>
keys, ColumnPath column_path, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         ThriftValidation.validateColumnPath(table, column_path);
@@ -309,37 +306,34 @@ public class CassandraServer implements 
         QueryPath path = new QueryPath(column_path.column_family, column_path.column == null
? null : column_path.super_column);
         List<byte[]> nameAsList = Arrays.asList(column_path.column == null ? column_path.super_column
: column_path.column);
         List<ReadCommand> commands = new ArrayList<ReadCommand>();
-        for (String key: keys)
+        for (byte[] key: keys)
         {
             ThriftValidation.validateKey(key);
-            // FIXME: string keys
-            commands.add(new SliceByNamesReadCommand(table, key.getBytes(UTF8), path, nameAsList));
+            commands.add(new SliceByNamesReadCommand(table, key, path, nameAsList));
         }
 
-        Map<String, ColumnOrSuperColumn> columnFamiliesMap = new HashMap<String,
ColumnOrSuperColumn>();
+        Map<byte[], ColumnOrSuperColumn> columnFamiliesMap = new HashMap<byte[],
ColumnOrSuperColumn>();
         Map<byte[], ColumnFamily> cfamilies = readColumnFamily(commands, consistency_level);
 
 
         for (ReadCommand command: commands)
         {
             ColumnFamily cf = cfamilies.get(command.key);
-            // FIXME: string keys
-            String skey = new String(command.key, UTF8);
             if (cf == null)
             {
-                columnFamiliesMap.put(skey, new ColumnOrSuperColumn());
+                columnFamiliesMap.put(command.key, new ColumnOrSuperColumn());
             }
             else
             {
                 List<ColumnOrSuperColumn> tcolumns = thriftifyColumnFamily(cf, command.queryPath.superColumnName
!= null, false);
-                columnFamiliesMap.put(skey, tcolumns.size() > 0 ? tcolumns.iterator().next()
: new ColumnOrSuperColumn());
+                columnFamiliesMap.put(command.key, tcolumns.size() > 0 ? tcolumns.iterator().next()
: new ColumnOrSuperColumn());
             }
         }
 
         return columnFamiliesMap;
     }
 
-    public int get_count(String table, String key, ColumnParent column_parent, ConsistencyLevel
consistency_level)
+    public int get_count(String table, byte[] key, ColumnParent column_parent, ConsistencyLevel
consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -352,7 +346,7 @@ public class CassandraServer implements 
         return get_slice(table, key, column_parent, predicate, consistency_level).size();
     }
 
-    public void insert(String table, String key, ColumnPath column_path, byte[] value, long
timestamp, ConsistencyLevel consistency_level)
+    public void insert(String table, byte[] key, ColumnPath column_path, byte[] value, long
timestamp, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -363,8 +357,7 @@ public class CassandraServer implements 
         ThriftValidation.validateKey(key);
         ThriftValidation.validateColumnPath(table, column_path);
 
-        // FIXME: string keys
-        RowMutation rm = new RowMutation(table, key.getBytes(UTF8));
+        RowMutation rm = new RowMutation(table, key);
         try
         {
             rm.add(new QueryPath(column_path), value, timestamp);
@@ -376,7 +369,7 @@ public class CassandraServer implements 
         doInsert(consistency_level, rm);
     }
     
-    public void batch_insert(String keyspace, String key, Map<String, List<ColumnOrSuperColumn>>
cfmap, ConsistencyLevel consistency_level)
+    public void batch_insert(String keyspace, byte[] key, Map<String, List<ColumnOrSuperColumn>>
cfmap, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -394,11 +387,10 @@ public class CassandraServer implements 
             }
         }
 
-        // FIXME: string keys
-        doInsert(consistency_level, RowMutation.getRowMutation(keyspace, key.getBytes(UTF8),
cfmap));
+        doInsert(consistency_level, RowMutation.getRowMutation(keyspace, key, cfmap));
     }
 
-    public void batch_mutate(String keyspace, Map<String,Map<String,List<Mutation>>>
mutation_map, ConsistencyLevel consistency_level)
+    public void batch_mutate(String keyspace, Map<byte[],Map<String,List<Mutation>>>
mutation_map, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -425,9 +417,9 @@ public class CassandraServer implements 
         checkLoginAuthorized(needed);
 
         List<RowMutation> rowMutations = new ArrayList<RowMutation>();
-        for (Map.Entry<String, Map<String, List<Mutation>>> mutationEntry:
mutation_map.entrySet())
+        for (Map.Entry<byte[], Map<String, List<Mutation>>> mutationEntry:
mutation_map.entrySet())
         {
-            String key = mutationEntry.getKey();
+            byte[] key = mutationEntry.getKey();
 
             ThriftValidation.validateKey(key);
             Map<String, List<Mutation>> columnFamilyToMutations = mutationEntry.getValue();
@@ -440,8 +432,7 @@ public class CassandraServer implements 
                     ThriftValidation.validateMutation(keyspace, cfName, mutation);
                 }
             }
-            // FIXME: string keys
-            rowMutations.add(RowMutation.getRowMutationFromMutations(keyspace, key.getBytes(UTF8),
columnFamilyToMutations));
+            rowMutations.add(RowMutation.getRowMutationFromMutations(keyspace, key, columnFamilyToMutations));
         }
         if (consistency_level == ConsistencyLevel.ZERO)
         {
@@ -460,7 +451,7 @@ public class CassandraServer implements 
         }
     }
 
-    public void remove(String table, String key, ColumnPath column_path, long timestamp,
ConsistencyLevel consistency_level)
+    public void remove(String table, byte[] key, ColumnPath column_path, long timestamp,
ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -471,8 +462,7 @@ public class CassandraServer implements 
         ThriftValidation.validateKey(key);
         ThriftValidation.validateColumnPathOrParent(table, column_path);
         
-        // FIXME: string keys
-        RowMutation rm = new RowMutation(table, key.getBytes(UTF8));
+        RowMutation rm = new RowMutation(table, key);
         rm.delete(new QueryPath(column_path), timestamp);
 
         doInsert(consistency_level, rm);
@@ -523,7 +513,7 @@ public class CassandraServer implements 
         return columnFamiliesMap;
     }
 
-    public List<KeySlice> get_range_slice(String keyspace, ColumnParent column_parent,
SlicePredicate predicate, String start_key, String finish_key, int maxRows, ConsistencyLevel
consistency_level)
+    public List<KeySlice> get_range_slice(String keyspace, ColumnParent column_parent,
SlicePredicate predicate, byte[] start_key, byte[] finish_key, int maxRows, ConsistencyLevel
consistency_level)
     throws InvalidRequestException, UnavailableException, TException, TimedOutException
     {
         if (logger.isDebugEnabled())
@@ -565,8 +555,7 @@ public class CassandraServer implements 
             }
             else
             {
-                // FIXME: string keys
-                bounds = new Bounds(p.getToken(range.start_key.getBytes(UTF8)), p.getToken(range.end_key.getBytes(UTF8)));
+                bounds = new Bounds(p.getToken(range.start_key), p.getToken(range.end_key));
             }
             rows = StorageProxy.getRangeSlice(new RangeSliceCommand(keyspace, column_parent,
predicate, bounds, range.count), consistency_level);
             assert rows != null;
@@ -585,8 +574,7 @@ public class CassandraServer implements 
         for (Row row : rows)
         {
             List<ColumnOrSuperColumn> thriftifiedColumns = thriftifyColumnFamily(row.cf,
column_parent.super_column != null, reversed);
-            // FIXME: string keys
-            keySlices.add(new KeySlice(new String(row.key.key, UTF8), thriftifiedColumns));
+            keySlices.add(new KeySlice(row.key.key, thriftifiedColumns));
         }
 
         return keySlices;

Modified: cassandra/trunk/src/java/org/apache/cassandra/thrift/ThriftValidation.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/ThriftValidation.java?rev=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/ThriftValidation.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/ThriftValidation.java Wed Apr 14
19:53:23 2010
@@ -38,16 +38,18 @@ import org.apache.cassandra.utils.FBUtil
 
 public class ThriftValidation
 {
-    static void validateKey(String key) throws InvalidRequestException
+    static void validateKey(byte[] key) throws InvalidRequestException
     {
-        if (key.isEmpty())
+        if (key == null || key.length == 0)
         {
             throw new InvalidRequestException("Key may not be empty");
         }
-        // check that writeUTF will be able to handle it -- encoded length must fit in 2
bytes
-        int utflen = FBUtilities.encodedUTF8Length(key);
-        if (utflen > 65535)
-            throw new InvalidRequestException("Encoded key length of " + utflen + " is longer
than maximum of 65535");
+        // check that key can be handled by FBUtilities.writeShortByteArray
+        if (key.length > FBUtilities.MAX_UNSIGNED_SHORT)
+        {
+            throw new InvalidRequestException("Key length of " + key.length +
+                    " is longer than maximum of " + FBUtilities.MAX_UNSIGNED_SHORT);
+        }
     }
 
     private static void validateTable(String tablename) throws KeyspaceNotDefinedException
@@ -303,9 +305,8 @@ public class ThriftValidation
         if (range.start_key != null)
         {
             IPartitioner p = StorageService.getPartitioner();
-            // FIXME: string keys
-            Token startToken = p.getToken(range.start_key.getBytes(FBUtilities.UTF8));
-            Token endToken = p.getToken(range.end_key.getBytes(FBUtilities.UTF8));
+            Token startToken = p.getToken(range.start_key);
+            Token endToken = p.getToken(range.end_key);
             if (startToken.compareTo(endToken) > 0 && !endToken.equals(p.getMinimumToken()))
             {
                 if (p instanceof RandomPartitioner)

Modified: cassandra/trunk/test/unit/org/apache/cassandra/client/TestRingCache.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/client/TestRingCache.java?rev=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/client/TestRingCache.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/client/TestRingCache.java Wed Apr 14 19:53:23
2010
@@ -87,20 +87,20 @@ public class TestRingCache
 
         for (int nRows = minRow; nRows < maxRow; nRows++)
         {
-            String row = rowPrefix + nRows;
+            byte[] row = (rowPrefix + nRows).getBytes();
             ColumnPath col = new ColumnPath("Standard1").setSuper_column(null).setColumn("col1".getBytes());
 
-            List<InetAddress> endPoints = tester.ringCache.getEndPoint(row.getBytes());
+            List<InetAddress> endPoints = tester.ringCache.getEndPoint(row);
             String hosts="";
             for (int i = 0; i < endPoints.size(); i++)
                 hosts = hosts + ((i > 0) ? "," : "") + endPoints.get(i);
-            System.out.println("hosts with key " + row + " : " + hosts + "; choose " + endPoints.get(0));
+            System.out.println("hosts with key " + new String(row) + " : " + hosts + "; choose
" + endPoints.get(0));
 
             // now, read the row back directly from the host owning the row locally
             tester.setup(endPoints.get(0).getHostAddress(), DatabaseDescriptor.getRpcPort());
             tester.thriftClient.insert(keyspace, row, col, "val1".getBytes(), 1, ConsistencyLevel.ONE);
             Column column = tester.thriftClient.get(keyspace, row, col, ConsistencyLevel.ONE).column;
-            System.out.println("read row " + row + " " + new String(column.name) + ":" +
new String(column.value) + ":" + column.timestamp);
+            System.out.println("read row " + new String(row) + " " + new String(column.name)
+ ":" + new String(column.value) + ":" + column.timestamp);
         }
 
         System.exit(1);

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=934155&r1=934154&r2=934155&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
(original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
Wed Apr 14 19:53:23 2010
@@ -89,7 +89,7 @@ public class EmbeddedCassandraServiceTes
     {
         Cassandra.Client client = getClient();
 
-        String key_user_id = "1";
+        byte[] key_user_id = "1".getBytes();
 
         long timestamp = System.currentTimeMillis();
         ColumnPath cp = new ColumnPath("Standard1");



Mime
View raw message