You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2013/08/21 18:17:14 UTC

[1/2] git commit: Add ability for CQL3 to list partition keys

Updated Branches:
  refs/heads/trunk 43105f7f1 -> 0da21de3e


Add ability for CQL3 to list partition keys

patch by Aleksey Yeschenko and Dan Jatnieks;
reviewed by Sylvain Lebresne for CASSANDRA-4536


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

Branch: refs/heads/trunk
Commit: b362aebd304144235121350913839b9a7ba3a5b0
Parents: bfb0647
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Aug 21 19:09:58 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Aug 21 19:09:58 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 bin/cqlsh                                       |  4 +-
 doc/cql3/CQL.textile                            |  9 ++-
 pylib/cqlshlib/cql3handling.py                  |  2 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  6 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  2 +-
 .../cql3/statements/ModificationStatement.java  |  2 +-
 .../cql3/statements/SelectStatement.java        | 63 ++++++++++++++++----
 8 files changed, 69 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8335423..cdc79c1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,6 +6,7 @@
  * Use a range aware scanner for cleanup (CASSANDRA-2524)
  * Cleanup doesn't need to inspect sstables that contain only local data 
    (CASSANDRA-5722)
+ * Add ability for CQL3 to list partition keys (CASSANDRA-4536)
 
 
 2.0.0

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 0f51c7f..b1764db 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -32,7 +32,7 @@ exit 1
 from __future__ import with_statement
 
 description = "CQL Shell for Apache Cassandra"
-version = "4.0.0"
+version = "4.0.1"
 
 from StringIO import StringIO
 from itertools import groupby
@@ -132,7 +132,7 @@ if os.path.exists(OLD_HISTORY):
 
 DEFAULT_HOST = 'localhost'
 DEFAULT_PORT = 9160
-DEFAULT_CQLVER = '3.1.0'
+DEFAULT_CQLVER = '3.1.1'
 DEFAULT_TRANSPORT_FACTORY = 'cqlshlib.tfactory.regular_transport_factory'
 
 DEFAULT_TIME_FORMAT = '%Y-%m-%d %H:%M:%S%z'

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index c87b9a5..e8f991b 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.1.0
+h1. Cassandra Query Language (CQL) v3.1.1
 
 
  <span id="tableOfContents">
@@ -604,7 +604,7 @@ bc(syntax)..
                   ( LIMIT <integer> )?
                   ( ALLOW FILTERING )?
 
-<select-clause> ::= <selection-list>
+<select-clause> ::= DISTINCT? <selection-list>
                   | COUNT '(' ( '*' | '1' ) ')' (AS <identifier>)?
 
 <selection-list> ::= <selector> (AS <identifier>)? ( ',' <selector> (AS <identifier>)? )*
@@ -1077,12 +1077,17 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @WHERE@        | yes |
 | @WITH@         | yes |
 | @WRITETIME@    | no  |
+| @DISTINCT@     | no  |
 
 
 h2(#changes). Changes
 
 The following describes the addition/changes brought for each version of CQL.
 
+h3. 3.1.1
+
+* @SELECT@ statement now allows listing the partition keys (using the @DISTINCT@ modifier). See "CASSANDRA-4536":https://issues.apache.org/jira/browse/CASSANDRA-4536.
+
 h3. 3.1.0
 
 * "ALTER TABLE":#alterTableStmt @DROP@ option has been reenabled for CQL3 tables and has new semantics now: the space formerly used by dropped columns will now be eventually reclaimed (post-compaction). You should not readd previously dropped columns unless you use timestamps with microsecond precision (see "CASSANDRA-3919":https://issues.apache.org/jira/browse/CASSANDRA-3919 for more details).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index eacc433..0b38d04 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -568,7 +568,7 @@ syntax_rules += r'''
                              ")" ("=" | "<" | ">" | "<=" | ">=") <tokenDefinition>
              | [rel_lhs]=<cident> "IN" "(" <term> ( "," <term> )* ")"
              ;
-<selectClause> ::= <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*
+<selectClause> ::= "DISTINCT"? <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*
                  | "*"
                  | "COUNT" "(" star=( "*" | "1" ) ")" ("AS" <cident>)?
                  ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 0035c41..a46927f 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -212,13 +212,14 @@ useStatement returns [UseStatement stmt]
  */
 selectStatement returns [SelectStatement.RawStatement expr]
     @init {
+        boolean isDistinct = false;
         boolean isCount = false;
         ColumnIdentifier countAlias = null;
         Term.Raw limit = null;
         Map<ColumnIdentifier, Boolean> orderings = new LinkedHashMap<ColumnIdentifier, Boolean>();
         boolean allowFiltering = false;
     }
-    : K_SELECT ( sclause=selectClause
+    : K_SELECT ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause
                | (K_COUNT '(' sclause=selectCountClause ')' { isCount = true; } (K_AS c=cident { countAlias = c; })?) )
       K_FROM cf=columnFamilyName
       ( K_WHERE wclause=whereClause )?
@@ -227,6 +228,7 @@ selectStatement returns [SelectStatement.RawStatement expr]
       ( K_ALLOW K_FILTERING  { allowFiltering = true; } )?
       {
           SelectStatement.Parameters params = new SelectStatement.Parameters(orderings,
+                                                                             isDistinct,
                                                                              isCount,
                                                                              countAlias,
                                                                              allowFiltering);
@@ -939,6 +941,7 @@ unreserved_function_keyword returns [String str]
         | K_EXISTS
         | K_CUSTOM
         | K_TRIGGER
+        | K_DISTINCT
         ) { $str = $k.text; }
     | t=native_type { $str = t.toString(); }
     ;
@@ -957,6 +960,7 @@ K_WITH:        W I T H;
 K_LIMIT:       L I M I T;
 K_USING:       U S I N G;
 K_USE:         U S E;
+K_DISTINCT:    D I S T I N C T;
 K_COUNT:       C O U N T;
 K_SET:         S E T;
 K_BEGIN:       B E G I N;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index e593289..6d48d2d 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 
 public class QueryProcessor
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.0");
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.1");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index e9141f9..2fce619 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -438,7 +438,7 @@ public abstract class ModificationStatement implements CQLStatement
 
         long now = System.currentTimeMillis();
         Selection.ResultSetBuilder builder = selection.resultSetBuilder(now);
-        SelectStatement.forSelection(cfDef, selection).processColumnFamily(key, cf, Collections.<ByteBuffer>emptyList(), Integer.MAX_VALUE, now, builder);
+        SelectStatement.forSelection(cfDef, selection).processColumnFamily(key, cf, Collections.<ByteBuffer>emptyList(), now, builder);
 
         return builder.build();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b362aebd/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 2ac94df..8e09fe5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
@@ -80,7 +81,7 @@ public class SelectStatement implements CQLStatement
     private Map<CFDefinition.Name, Integer> orderingIndexes;
 
     // Used by forSelection below
-    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, null, false);
+    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, false, null, false);
 
     private static enum Bound
     {
@@ -306,7 +307,7 @@ public class SelectStatement implements CQLStatement
         AbstractBounds<RowPosition> keyBounds = getKeyBounds(variables);
         return keyBounds == null
              ? null
-             : new RangeSliceCommand(keyspace(), columnFamily(), now,  filter, keyBounds, expressions, limit, true, false);
+             : new RangeSliceCommand(keyspace(), columnFamily(), now,  filter, keyBounds, expressions, limit, !parameters.isDistinct, false);
     }
 
     private AbstractBounds<RowPosition> getKeyBounds(List<ByteBuffer> variables) throws InvalidRequestException
@@ -369,7 +370,11 @@ public class SelectStatement implements CQLStatement
     private IDiskAtomFilter makeFilter(List<ByteBuffer> variables, int limit)
     throws InvalidRequestException
     {
-        if (isColumnRange())
+        if (parameters.isDistinct)
+        {
+            return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, -1);
+        }
+        else if (isColumnRange())
         {
             // For sparse, we used to ask for 'defined columns' * 'asked limit' (where defined columns includes the row marker)
             // to account for the grouping of columns.
@@ -826,7 +831,7 @@ public class SelectStatement implements CQLStatement
             if (row.cf == null)
                 continue;
 
-            processColumnFamily(row.key.key, row.cf, variables, limit, now, result);
+            processColumnFamily(row.key.key, row.cf, variables, now, result);
         }
 
         ResultSet cqlRows = result.build();
@@ -843,13 +848,24 @@ public class SelectStatement implements CQLStatement
     }
 
     // Used by ModificationStatement for CAS operations
-    void processColumnFamily(ByteBuffer key, ColumnFamily cf, List<ByteBuffer> variables, int limit, long now, Selection.ResultSetBuilder result) throws InvalidRequestException
+    void processColumnFamily(ByteBuffer key, ColumnFamily cf, List<ByteBuffer> variables, long now, Selection.ResultSetBuilder result)
+    throws InvalidRequestException
     {
         ByteBuffer[] keyComponents = cfDef.hasCompositeKey
                                    ? ((CompositeType)cfDef.cfm.getKeyValidator()).split(key)
                                    : new ByteBuffer[]{ key };
 
-        if (cfDef.isCompact)
+        if (parameters.isDistinct)
+        {
+            if (!cf.hasOnlyTombstones(now))
+            {
+                result.newRow();
+                // selection.getColumnsList() will contain only the partition key components - all of them.
+                for (CFDefinition.Name name : selection.getColumnsList())
+                    result.add(keyComponents[name.position]);
+            }
+        }
+        else if (cfDef.isCompact)
         {
             // One cqlRow per column
             for (Column c : columnsInOrder(cf, variables))
@@ -1055,6 +1071,9 @@ public class SelectStatement implements CQLStatement
                                 ? Selection.wildcard(cfDef)
                                 : Selection.fromSelectors(cfDef, selectClause);
 
+            if (parameters.isDistinct)
+                validateDistinctSelection(selection.getColumnsList(), cfDef.keys.values());
+
             Term prepLimit = null;
             if (limit != null)
             {
@@ -1370,6 +1389,18 @@ public class SelectStatement implements CQLStatement
             return new ParsedStatement.Prepared(stmt, Arrays.<ColumnSpecification>asList(names));
         }
 
+        private void validateDistinctSelection(Collection<CFDefinition.Name> requestedColumns, Collection<CFDefinition.Name> partitionKey)
+        throws InvalidRequestException
+        {
+            for (CFDefinition.Name name : requestedColumns)
+                if (!partitionKey.contains(name))
+                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", name));
+
+            for (CFDefinition.Name name : partitionKey)
+                if (!requestedColumns.contains(name))
+                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+        }
+
         private boolean containsAlias(final ColumnIdentifier name)
         {
             return Iterables.any(selectClause, new Predicate<RawSelector>()
@@ -1443,11 +1474,13 @@ public class SelectStatement implements CQLStatement
         @Override
         public String toString()
         {
-            return String.format("SelectRawStatement[name=%s, selectClause=%s, whereClause=%s, isCount=%s]",
-                    cfName,
-                    selectClause,
-                    whereClause,
-                    parameters.isCount);
+            return Objects.toStringHelper(this)
+                          .add("name", cfName)
+                          .add("selectClause", selectClause)
+                          .add("whereClause", whereClause)
+                          .add("isDistinct", parameters.isDistinct)
+                          .add("isCount", parameters.isCount)
+                          .toString();
         }
     }
 
@@ -1592,13 +1625,19 @@ public class SelectStatement implements CQLStatement
     public static class Parameters
     {
         private final Map<ColumnIdentifier, Boolean> orderings;
+        private final boolean isDistinct;
         private final boolean isCount;
         private final ColumnIdentifier countAlias;
         private final boolean allowFiltering;
 
-        public Parameters(Map<ColumnIdentifier, Boolean> orderings, boolean isCount, ColumnIdentifier countAlias, boolean allowFiltering)
+        public Parameters(Map<ColumnIdentifier, Boolean> orderings,
+                          boolean isDistinct,
+                          boolean isCount,
+                          ColumnIdentifier countAlias,
+                          boolean allowFiltering)
         {
             this.orderings = orderings;
+            this.isDistinct = isDistinct;
             this.isCount = isCount;
             this.countAlias = countAlias;
             this.allowFiltering = allowFiltering;


[2/2] git commit: Merge branch 'cassandra-2.0' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.0' into trunk


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

Branch: refs/heads/trunk
Commit: 0da21de3ef67e40a42015d7f50728a3d9b5cd2fa
Parents: 43105f7 b362aeb
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Aug 21 19:17:01 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Aug 21 19:17:01 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 bin/cqlsh                                       |  4 +-
 doc/cql3/CQL.textile                            |  9 ++-
 pylib/cqlshlib/cql3handling.py                  |  2 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  6 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  2 +-
 .../cql3/statements/ModificationStatement.java  |  2 +-
 .../cql3/statements/SelectStatement.java        | 63 ++++++++++++++++----
 8 files changed, 69 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0da21de3/CHANGES.txt
----------------------------------------------------------------------