You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/11/13 18:35:21 UTC

cassandra git commit: Fix aggregation fn on empty selection, column names

Repository: cassandra
Updated Branches:
  refs/heads/trunk 7c93b04b3 -> 4ff6e7dff


Fix aggregation fn on empty selection, column names

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-8229


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/4ff6e7df
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/4ff6e7df
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/4ff6e7df

Branch: refs/heads/trunk
Commit: 4ff6e7dfff58eaca02c49a13d82058665cfad23b
Parents: 7c93b04
Author: blerer <b_...@hotmail.com>
Authored: Thu Nov 13 11:34:39 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu Nov 13 11:34:39 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 pylib/cqlshlib/cql3handling.py                  |  9 +----
 .../org/apache/cassandra/cql3/ResultSet.java    |  5 +++
 .../selection/AbstractFunctionSelector.java     | 21 +++++-----
 .../cassandra/cql3/selection/FieldSelector.java | 17 ++++----
 .../cassandra/cql3/selection/Selection.java     |  5 +++
 .../cassandra/cql3/selection/Selector.java      | 25 +++++++++++-
 .../cql3/selection/SelectorFactories.java       | 20 ++++++++++
 .../cql3/selection/SimpleSelector.java          | 15 ++++---
 .../cql3/selection/WritetimeOrTTLSelector.java  | 16 ++++----
 .../apache/cassandra/cql3/AggregationTest.java  | 42 +++++++++++++++++++-
 .../org/apache/cassandra/cql3/CQLTester.java    | 18 +++++++++
 12 files changed, 150 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 23e3af5..04b0322 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 3.0
+ * Fix aggregate fn results on empty selection, result column name,
+   and cqlsh parsing (CASSANDRA-8229)
  * Mark sstables as repaired after full repair (CASSANDRA-7586) 
  * Extend Descriptor to include a format value and refactor reader/writer apis (CASSANDRA-7443)
  * Integrate JMH for microbenchmarks (CASSANDRA-8151)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 5a4907d..261161c 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -39,7 +39,7 @@ NONALTERBALE_KEYSPACES = ('system')
 class Cql3ParsingRuleSet(CqlParsingRuleSet):
     keywords = set((
         'select', 'from', 'where', 'and', 'key', 'insert', 'update', 'with',
-        'limit', 'using', 'use', 'count', 'set',
+        'limit', 'using', 'use', 'set',
         'begin', 'apply', 'batch', 'truncate', 'delete', 'in', 'create',
         'function', 'keyspace', 'schema', 'columnfamily', 'table', 'index', 'on', 'drop',
         'primary', 'into', 'values', 'timestamp', 'ttl', 'alter', 'add', 'type',
@@ -292,7 +292,6 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
 <unreservedKeyword> ::= nocomplete=
                         ( <K_KEY>
                         | <K_CLUSTERING>
-                        # | <K_COUNT>  -- to get count(*) completion, treat count as reserved
                         | <K_TTL>
                         | <K_COMPACT>
                         | <K_STORAGE>
@@ -620,7 +619,6 @@ syntax_rules += r'''
              ;
 <selectClause> ::= "DISTINCT"? <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*
                  | "*"
-                 | "COUNT" "(" star=( "*" | "1" ) ")" ("AS" <cident>)?
                  ;
 <udtSubfieldSelection> ::= <identifier> "." <identifier>
                          ;
@@ -628,6 +626,7 @@ syntax_rules += r'''
              | <udtSubfieldSelection>
              | "WRITETIME" "(" [colname]=<cident> ")"
              | "TTL" "(" [colname]=<cident> ")"
+             | "COUNT" "(" star=( "*" | "1" ) ")"
              | <functionName> <selectionFunctionArguments>
              ;
 <selectionFunctionArguments> ::= "(" ( <selector> ( "," <selector> )* )? ")"
@@ -679,10 +678,6 @@ def select_relation_lhs_completer(ctxt, cass):
             filterable.add(cd.name)
     return map(maybe_escape_name, filterable)
 
-@completer_for('selectClause', 'star')
-def select_count_star_completer(ctxt, cass):
-    return ['*']
-
 explain_completion('selector', 'colname')
 
 syntax_rules += r'''

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 9762f01..c8c3869 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -58,6 +58,11 @@ public class ResultSet
         return rows.size();
     }
 
+    public boolean isEmpty()
+    {
+        return size() == 0;
+    }
+
     public void addRow(List<ByteBuffer> row)
     {
         assert row.size() == metadata.valueCount();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index b04f346..4660e1d 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -21,13 +21,11 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.commons.lang3.text.StrBuilder;
+
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.commons.lang3.text.StrBuilder;
 
 abstract class AbstractFunctionSelector<T extends Function> extends Selector
 {
@@ -56,12 +54,17 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
 
         return new Factory()
         {
-            public ColumnSpecification getColumnSpecification(CFMetaData cfm)
+            protected String getColumnName()
+            {
+                return new StrBuilder(fun.name().toString()).append('(')
+                                                            .appendWithSeparators(factories.getColumnNames(), ", ")
+                                                            .append(')')
+                                                            .toString();
+            }
+
+            protected AbstractType<?> getReturnType()
             {
-                return new ColumnSpecification(cfm.ksName,
-                                               cfm.cfName,
-                                               new ColumnIdentifier(fun.toString(), true),
-                                               fun.returnType());
+                return fun.returnType();
             }
 
             public Selector newInstance()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index 0bba56c..7e14486 100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@ -19,9 +19,6 @@ package org.apache.cassandra.cql3.selection;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -38,14 +35,16 @@ final class FieldSelector extends Selector
     {
         return new Factory()
         {
-            public ColumnSpecification getColumnSpecification(CFMetaData cfm)
+            protected String getColumnName()
             {
-                ColumnIdentifier identifier =
-                        new ColumnIdentifier(String.format("%s.%s",
-                                                           factory.getColumnSpecification(cfm).name,
-                                                           UTF8Type.instance.getString(type.fieldName(field))), true);
+                return String.format("%s.%s",
+                                     factory.getColumnName(),
+                                     UTF8Type.instance.getString(type.fieldName(field)));
+            }
 
-                return new ColumnSpecification(cfm.ksName, cfm.cfName, identifier, type.fieldType(field));
+            protected AbstractType<?> getReturnType()
+            {
+                return type.fieldType(field);
             }
 
             public Selector newInstance()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 17e2e92..7c3d34c 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -237,6 +237,11 @@ public abstract class Selection
                 selectors.reset();
                 current = null;
             }
+
+            if (resultSet.isEmpty() && selectors.isAggregate())
+            {
+                resultSet.addRow(selectors.getOutputRow());
+            }
             return resultSet;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selector.java b/src/java/org/apache/cassandra/cql3/selection/Selector.java
index 310e175..889da70 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -46,7 +47,13 @@ public abstract class Selector implements AssignmentTestable
          * @param cfm the column family meta data
          * @return a column specification
          */
-        public abstract ColumnSpecification getColumnSpecification(CFMetaData cfm);
+        public final ColumnSpecification getColumnSpecification(CFMetaData cfm)
+        {
+            return new ColumnSpecification(cfm.ksName,
+                                           cfm.cfName,
+                                           new ColumnIdentifier(getColumnName(), true),
+                                           getReturnType());
+        }
 
         /**
          * Creates a new <code>Selector</code> instance.
@@ -87,6 +94,22 @@ public abstract class Selector implements AssignmentTestable
         {
             return false;
         }
+
+        /**
+         * Returns the name of the column corresponding to the output value of the selector instances created by
+         * this factory.
+         *
+         * @return a column name
+         */
+        protected abstract String getColumnName();
+
+        /**
+         * Returns the type of the values returned by the selector instances created by this factory.
+         *
+         * @return the selector output type
+         */
+        protected abstract AbstractType<?> getReturnType();
+
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
index e033556..6922994 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@ -21,6 +21,9 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.selection.Selector.Factory;
@@ -148,4 +151,21 @@ final class SelectorFactories implements Iterable<Selector.Factory>
     {
         return factories.iterator();
     }
+
+    /**
+     * Returns the names of the columns corresponding to the output values of the selector instances created by
+     * these factories.
+     *
+     * @return a list of column names
+     */
+    public List<String> getColumnNames()
+    {
+        return Lists.transform(factories, new Function<Selector.Factory, String>()
+        {
+            public String apply(Selector.Factory factory)
+            {
+                return factory.getColumnName();
+            }
+        });
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
index 37459d2..cdd91b2 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
@@ -19,9 +19,6 @@ package org.apache.cassandra.cql3.selection;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -37,12 +34,14 @@ public final class SimpleSelector extends Selector
     {
         return new Factory()
         {
-            public ColumnSpecification getColumnSpecification(CFMetaData cfm)
+            protected String getColumnName()
             {
-                return new ColumnSpecification(cfm.ksName,
-                                               cfm.cfName,
-                                               new ColumnIdentifier(columnName, true),
-                                               type);
+                return columnName;
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
             }
 
             public Selector newInstance()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
index a57a3ca..2494334 100644
--- a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
@@ -19,9 +19,6 @@ package org.apache.cassandra.cql3.selection;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.Int32Type;
@@ -39,13 +36,14 @@ final class WritetimeOrTTLSelector extends Selector
     {
         return new Factory()
         {
-            public ColumnSpecification getColumnSpecification(CFMetaData cfm)
+            protected String getColumnName()
             {
-                String text = String.format("%s(%s)", isWritetime ? "writetime" : "ttl", columnName);
-                return new ColumnSpecification(cfm.ksName,
-                                               cfm.cfName,
-                                               new ColumnIdentifier(text, true),
-                                               isWritetime ? LongType.instance : Int32Type.instance);
+                return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", columnName);
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return isWritetime ? LongType.instance : Int32Type.instance;
             }
 
             public Selector newInstance()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/test/unit/org/apache/cassandra/cql3/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
index 87b7ca7..99db62a 100644
--- a/test/unit/org/apache/cassandra/cql3/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
@@ -32,6 +32,15 @@ public class AggregationTest extends CQLTester
     public void testFunctions() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c double, d decimal, primary key (a, b))");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT COUNT(*) FROM %s"), "count");
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(0L));
+        assertColumnNames(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
+                          "max(b)", "min(b)", "sum(b)", "avg(b)" , "max(c)", "sum(c)", "avg(c)", "sum(d)", "avg(d)");
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d) FROM %s"),
+                   row(null, null, 0, 0, null, 0.0, 0.0, new BigDecimal("0"), new BigDecimal("0")));
+
         execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 11.5, 11.5)");
         execute("INSERT INTO %s (a, b, c, d) VALUES (1, 2, 9.5, 1.5)");
         execute("INSERT INTO %s (a, b, c, d) VALUES (1, 3, 9.0, 2.0)");
@@ -46,6 +55,26 @@ public class AggregationTest extends CQLTester
     }
 
     @Test
+    public void testFunctionsWithCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int , b int, c double, primary key(a, b) ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 11.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 9.0)");
+
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s"),
+                   row(3, 1, 6, 2, 11.5, 30.0, 10.0));
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(3L));
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(3L));
+        assertRows(execute("SELECT COUNT(*) FROM %s WHERE a = 1 AND b > 1"), row(2L));
+        assertRows(execute("SELECT COUNT(1) FROM %s WHERE a = 1 AND b > 1"), row(2L));
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s WHERE a = 1 AND b > 1"),
+                   row(3, 2, 5, 2, 9.5, 18.5, 9.25));
+    }
+
+    @Test
     public void testInvalidCalls() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b))");
@@ -65,6 +94,17 @@ public class AggregationTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int primary key, b timeuuid, c double, d double)");
 
+        execute("CREATE OR REPLACE FUNCTION copySign(magnitude double, sign double) RETURNS double LANGUAGE JAVA\n" +
+                "AS 'return Double.valueOf(Math.copySign(magnitude.doubleValue(), sign.doubleValue()));';");
+
+        assertColumnNames(execute("SELECT max(a), max(unixTimestampOf(b)) FROM %s"), "max(a)", "max(unixtimestampof(b))");
+        assertRows(execute("SELECT max(a), max(unixTimestampOf(b)) FROM %s"), row(null, null));
+        assertColumnNames(execute("SELECT max(a), unixTimestampOf(max(b)) FROM %s"), "max(a)", "unixtimestampof(max(b))");
+        assertRows(execute("SELECT max(a), unixTimestampOf(max(b)) FROM %s"), row(null, null));
+
+        assertColumnNames(execute("SELECT max(copySign(c, d)) FROM %s"), "max(copysign(c, d))");
+        assertRows(execute("SELECT max(copySign(c, d)) FROM %s"), row((Object) null));
+
         execute("INSERT INTO %s (a, b, c, d) VALUES (1, maxTimeuuid('2011-02-03 04:05:00+0000'), -1.2, 2.1)");
         execute("INSERT INTO %s (a, b, c, d) VALUES (2, maxTimeuuid('2011-02-03 04:06:00+0000'), 1.3, -3.4)");
         execute("INSERT INTO %s (a, b, c, d) VALUES (3, maxTimeuuid('2011-02-03 04:10:00+0000'), 1.4, 1.2)");
@@ -76,8 +116,6 @@ public class AggregationTest extends CQLTester
 
         assertRows(execute("SELECT max(a), max(unixTimestampOf(b)) FROM %s"), row(3, date.getTime()));
         assertRows(execute("SELECT max(a), unixTimestampOf(max(b)) FROM %s"), row(3, date.getTime()));
-        execute("CREATE OR REPLACE FUNCTION copySign(magnitude double, sign double) RETURNS double LANGUAGE JAVA\n" +
-                "AS 'return Double.valueOf(Math.copySign(magnitude.doubleValue(), sign.doubleValue()));';");
 
         assertRows(execute("SELECT copySign(max(c), min(c)) FROM %s"), row(-1.4));
         assertRows(execute("SELECT copySign(c, d) FROM %s"), row(1.2), row(-1.3), row(1.4));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ff6e7df/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 4ada0c5..ca02237 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -349,6 +349,24 @@ public abstract class CQLTester
         Assert.assertTrue(String.format("Got %s rows than expected. Expected %d but got %d", rows.length>i ? "less" : "more", rows.length, i), i == rows.length);
     }
 
+    protected void assertColumnNames(UntypedResultSet result, String... expectedColumnNames)
+    {
+        if (result == null)
+        {
+            Assert.fail("No rows returned by query.");
+            return;
+        }
+
+        List<ColumnSpecification> metadata = result.metadata();
+        Assert.assertEquals("Got less columns than expected.", expectedColumnNames.length, metadata.size());
+
+        for (int i = 0, m = metadata.size(); i < m; i++)
+        {
+            ColumnSpecification columnSpec = metadata.get(i);
+            Assert.assertEquals(expectedColumnNames[i], columnSpec.name.toString());
+        }
+    }
+
     protected void assertAllRows(Object[]... rows) throws Throwable
     {
         assertRows(execute("SELECT * FROM %s"), rows);