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/01/30 00:43:19 UTC

svn commit: r904676 - in /incubator/cassandra/branches/cassandra-0.5: CHANGES.txt src/java/org/apache/cassandra/service/CassandraServer.java test/system/test_server.py test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java

Author: jbellis
Date: Fri Jan 29 23:43:19 2010
New Revision: 904676

URL: http://svn.apache.org/viewvc?rev=904676&view=rev
Log:
don't omit live subcolumns of deleted supercolumns in thrift result.  patch by jbellis; tested by Vijay Parthasarathy for CASSANDRA-703

Modified:
    incubator/cassandra/branches/cassandra-0.5/CHANGES.txt
    incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/service/CassandraServer.java
    incubator/cassandra/branches/cassandra-0.5/test/system/test_server.py
    incubator/cassandra/branches/cassandra-0.5/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java

Modified: incubator/cassandra/branches/cassandra-0.5/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/CHANGES.txt?rev=904676&r1=904675&r2=904676&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/CHANGES.txt (original)
+++ incubator/cassandra/branches/cassandra-0.5/CHANGES.txt Fri Jan 29 23:43:19 2010
@@ -1,6 +1,9 @@
 0.5.1
  * ensure all files for an sstable are streamed to the same directory.
    (CASSANDRA-716)
+ * don't omit live subcolumns of deleted supercolumns in thrift result.
+   (CASSANDRA-703)
+
 
 0.5.0 final
  * avoid attempting to delete temporary bootstrap files twice (CASSANDRA-681)

Modified: incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/service/CassandraServer.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/service/CassandraServer.java?rev=904676&r1=904675&r2=904676&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/service/CassandraServer.java (original)
+++ incubator/cassandra/branches/cassandra-0.5/src/java/org/apache/cassandra/service/CassandraServer.java Fri Jan 29 23:43:19 2010
@@ -258,41 +258,6 @@
         return column;
     }
 
-    /** no values will be mapped to keys with no data */
-    private Map<String, Collection<IColumn>> multigetColumns(List<ReadCommand> commands, int consistency_level)
-    throws InvalidRequestException, UnavailableException, TimedOutException
-    {
-        Map<String, ColumnFamily> cfamilies = readColumnFamily(commands, consistency_level);
-        Map<String, Collection<IColumn>> columnFamiliesMap = new HashMap<String, Collection<IColumn>>();
-
-        for (ReadCommand command: commands)
-        {
-            ColumnFamily cfamily = cfamilies.get(command.key);
-            if (cfamily == null)
-                continue;
-
-            Collection<IColumn> columns = null;
-            if (command.queryPath.superColumnName != null)
-            {
-                IColumn column = cfamily.getColumn(command.queryPath.superColumnName);
-                if (column != null)
-                {
-                    columns = column.getSubColumns();
-                }
-            }
-            else
-            {
-                columns = cfamily.getSortedColumns();
-            }
-
-            if (columns != null && columns.size() != 0)
-            {
-                columnFamiliesMap.put(command.key, columns);
-            }
-        }
-        return columnFamiliesMap;
-    }
-
     /** 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, int consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
@@ -317,36 +282,20 @@
         }
 
         Map<String, ColumnOrSuperColumn> columnFamiliesMap = new HashMap<String, ColumnOrSuperColumn>();
-        Map<String, Collection<IColumn>> columnsMap = multigetColumns(commands, consistency_level);
+        Map<String, ColumnFamily> cfamilies = readColumnFamily(commands, consistency_level);
 
         for (ReadCommand command: commands)
         {
-            ColumnOrSuperColumn columnorsupercolumn;
-
-            Collection<IColumn> columns = columnsMap.get(command.key);
-            if (columns == null)
+            ColumnFamily cf = cfamilies.get(command.key);
+            if (cf == null)
             {
-               columnorsupercolumn = new ColumnOrSuperColumn();
+                columnFamiliesMap.put(command.key, new ColumnOrSuperColumn());
             }
             else
             {
-                assert columns.size() == 1;
-                IColumn column = columns.iterator().next();
-
-
-                if (column.isMarkedForDelete())
-                {
-                    columnorsupercolumn = new ColumnOrSuperColumn();
-                }
-                else
-                {
-                    columnorsupercolumn = column instanceof org.apache.cassandra.db.Column
-                                          ? new ColumnOrSuperColumn(new Column(column.name(), column.value(), column.timestamp()), null)
-                                          : new ColumnOrSuperColumn(null, new SuperColumn(column.name(), thriftifySubColumns(column.getSubColumns())));
-                }
-
+                List<ColumnOrSuperColumn> tcolumns = thriftifyColumnFamily(cf, command.queryPath.superColumnName != null, false);
+                columnFamiliesMap.put(command.key, tcolumns.size() > 0 ? tcolumns.iterator().next() : new ColumnOrSuperColumn());
             }
-            columnFamiliesMap.put(command.key, columnorsupercolumn);
         }
 
         return columnFamiliesMap;
@@ -357,44 +306,10 @@
     {
         if (logger.isDebugEnabled())
             logger.debug("get_count");
-        return multigetCountInternal(table, Arrays.asList(key), column_parent, consistency_level).get(key);
-    }
-
-    private Map<String, Integer> multigetCountInternal(String table, List<String> keys, ColumnParent column_parent, int consistency_level)
-    throws InvalidRequestException, UnavailableException, TimedOutException
-    {
-        // validateColumnParent assumes we require simple columns; g_c_c is the only
-        // one of the columnParent-taking apis that can also work at the SC level.
-        // so we roll a one-off validator here.
-        String cfType = ThriftValidation.validateColumnFamily(table, column_parent.column_family);
-        if (cfType.equals("Standard") && column_parent.super_column != null)
-        {
-            throw new InvalidRequestException("columnfamily alone is required for standard CF " + column_parent.column_family);
-        }
-
-        List<ReadCommand> commands = new ArrayList<ReadCommand>();
-        for (String key: keys)
-        {
-            ThriftValidation.validateKey(key);
-            commands.add(new SliceFromReadCommand(table, key, column_parent, ArrayUtils.EMPTY_BYTE_ARRAY, ArrayUtils.EMPTY_BYTE_ARRAY, true, Integer.MAX_VALUE));
-        }
-
-        Map<String, Integer> columnFamiliesMap = new HashMap<String, Integer>();
-        Map<String, Collection<IColumn>> columnsMap = multigetColumns(commands, consistency_level);
 
-        for (ReadCommand command: commands)
-        {
-            Collection<IColumn> columns = columnsMap.get(command.key);
-            if(columns == null)
-            {
-               columnFamiliesMap.put(command.key, 0);
-            }
-            else
-            {
-               columnFamiliesMap.put(command.key, columns.size());
-            }
-        }
-        return columnFamiliesMap;
+        SliceRange range = new SliceRange(ArrayUtils.EMPTY_BYTE_ARRAY, ArrayUtils.EMPTY_BYTE_ARRAY, false, Integer.MAX_VALUE);
+        SlicePredicate predicate = new SlicePredicate().setSlice_range(range);
+        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, int consistency_level)

Modified: incubator/cassandra/branches/cassandra-0.5/test/system/test_server.py
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/test/system/test_server.py?rev=904676&r1=904675&r2=904676&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/test/system/test_server.py (original)
+++ incubator/cassandra/branches/cassandra-0.5/test/system/test_server.py Fri Jan 29 23:43:19 2010
@@ -472,6 +472,18 @@
         assert columns == [Column(_i64(5), 'value5', 6)], columns
 
 
+    def test_super_cf_resurrect_subcolumn(self):
+        key = 'vijay'
+        client.insert('Keyspace1', key, ColumnPath('Super1', 'sc1', _i64(4)), 'value4', 0, ConsistencyLevel.ONE)
+
+        client.remove('Keyspace1', key, ColumnPath('Super1', 'sc1'), 1, ConsistencyLevel.ONE)
+
+        client.insert('Keyspace1', key, ColumnPath('Super1', 'sc1', _i64(4)), 'value4', 2, ConsistencyLevel.ONE)
+
+        result = client.get('Keyspace1', key, ColumnPath('Super1', 'sc1'), ConsistencyLevel.ONE)
+        assert result.super_column.columns is not None, result.super_column
+
+
     def test_empty_range(self):
         assert client.get_key_range('Keyspace1', 'Standard1', '', '', 1000, ConsistencyLevel.ONE) == []
         _insert_simple()

Modified: incubator/cassandra/branches/cassandra-0.5/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.5/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java?rev=904676&r1=904675&r2=904676&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.5/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java (original)
+++ incubator/cassandra/branches/cassandra-0.5/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java Fri Jan 29 23:43:19 2010
@@ -159,4 +159,33 @@
         assert subColumns.iterator().next().timestamp() == 2;
     }
 
+    @Test
+    public void testRemoveSuperColumnResurrection() throws IOException, ExecutionException, InterruptedException
+    {
+        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super2");
+        RowMutation rm;
+        String key = "keyC";
+
+        // add data
+        rm = new RowMutation("Keyspace1", key);
+        addMutation(rm, "Super2", "SC1", 1, "val1", 0);
+        rm.apply();
+
+        // remove
+        rm = new RowMutation("Keyspace1", key);
+        rm.delete(new QueryPath("Super2", "SC1".getBytes()), 1);
+        rm.apply();
+        assertNull(store.getColumnFamily(new NamesQueryFilter(key, new QueryPath("Super2"), "SC1".getBytes()), Integer.MAX_VALUE));
+
+        // resurrect
+        rm = new RowMutation("Keyspace1", key);
+        addMutation(rm, "Super2", "SC1", 1, "val2", 2);
+        rm.apply();
+
+        // validate
+        ColumnFamily resolved = store.getColumnFamily(new NamesQueryFilter(key, new QueryPath("Super2"), "SC1".getBytes()), Integer.MAX_VALUE);
+        Collection<IColumn> subColumns = resolved.getSortedColumns().iterator().next().getSubColumns();
+        assert subColumns.size() == 1;
+        assert subColumns.iterator().next().timestamp() == 2;
+    }
 }