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/12 12:56:47 UTC

git commit: Fix (some) 2i on composite components omissions

Updated Branches:
  refs/heads/cassandra-2.0.0 1c606bb8a -> 703bd684c


Fix (some) 2i on composite components omissions

patch by Aleksey Yeschenko; reviewed by Jonathan Ellis for
CASSANDRA-5851


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

Branch: refs/heads/cassandra-2.0.0
Commit: 703bd684ca79f817392acfed5c26ad0871660735
Parents: 1c606bb
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Sat Aug 3 23:40:02 2013 +0200
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Aug 12 12:55:56 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 bin/cqlsh                                       |  43 ++-
 pylib/cqlshlib/cql3handling.py                  | 281 ++++---------------
 pylib/cqlshlib/test/test_cqlsh_output.py        |   5 +-
 .../cql3/statements/CreateIndexStatement.java   |   2 +-
 .../cql3/statements/SelectStatement.java        | 116 ++++----
 .../CompositesIndexOnClusteringKey.java         |   5 +-
 7 files changed, 135 insertions(+), 318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5bc2745..396459b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,6 +2,7 @@
  * enable vnodes by default (CASSANDRA-5869)
  * fix CAS contention timeout (CASSANDRA-5830)
  * fix HsHa to respect max frame size (CASSANDRA-4573)
+ * Fix (some) 2i on composite components omissions (CASSANDRA-5851)
 Merged from 1.2:
  * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
  * Add KeyCacheHitRate metric to CF metrics (CASSANDRA-5868)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 21d43c6..5343a47 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -1090,14 +1090,8 @@ class Shell(cmd.Cmd):
         layout = self.get_columnfamily_layout(ksname, cfname)
         cfname = self.cql_protect_name(layout.name)
         out.write("CREATE TABLE %s (\n" % cfname)
-        keycol = layout.columns[0]
-        out.write("  %s %s" % (self.cql_protect_name(keycol.name),
-                               keycol.cqltype.cql_parameterized_type()))
-        if len(layout.primary_key_components) == 1:
-            out.write(" PRIMARY KEY")
-
-        indexed_columns = []
-        for col in layout.columns[1:]:
+
+        for col in layout.columns:
             colname = self.cql_protect_name(col.name)
             coltype = col.cqltype
 
@@ -1105,29 +1099,26 @@ class Shell(cmd.Cmd):
             if issubclass(coltype, ReversedType):
                 coltype = coltype.subtypes[0]
 
-            out.write(",\n  %s %s" % (colname, coltype.cql_parameterized_type()))
-            if col.index_name is not None:
-                indexed_columns.append(col)
+            out.write("  %s %s,\n" % (colname, coltype.cql_parameterized_type()))
 
-        if len(layout.primary_key_components) > 1:
-            out.write(",\n  PRIMARY KEY (")
-            partkeynames = self.cql_protect_names(layout.partition_key_components)
-            if len(partkeynames) > 1:
-                partkey = "(%s)" % ', '.join(partkeynames)
-            else:
-                partkey = partkeynames[0]
-            pk_parts = [partkey] + self.cql_protect_names(layout.column_aliases)
-            out.write(', '.join(pk_parts) + ')')
+        out.write("  PRIMARY KEY (")
+        partkeynames = self.cql_protect_names(layout.partition_key_columns)
+        if len(partkeynames) > 1:
+            partkey = "(%s)" % ', '.join(partkeynames)
+        else:
+            partkey = partkeynames[0]
+        pk_parts = [partkey] + self.cql_protect_names(layout.clustering_key_columns)
+        out.write(', '.join(pk_parts) + ')')
 
         out.write("\n)")
         joiner = 'WITH'
 
-        if layout.compact_storage:
+        if layout.is_compact_storage():
             out.write(' WITH COMPACT STORAGE')
             joiner = 'AND'
 
         # check if we need a CLUSTERING ORDER BY clause
-        if layout.column_aliases:
+        if layout.clustering_key_columns:
             # get a list of clustering component types
             if issubclass(layout.comparator, CompositeType):
                 clustering_types = layout.comparator.subtypes
@@ -1136,13 +1127,13 @@ class Shell(cmd.Cmd):
 
             # only write CLUSTERING ORDER clause of we have >= 1 DESC item
             if any(issubclass(t, ReversedType) for t in clustering_types):
-                if layout.compact_storage:
+                if layout.is_compact_storage():
                     out.write(' AND\n ')
                 else:
                     out.write(' WITH')
                 out.write(' CLUSTERING ORDER BY (')
 
-                clustering_names = self.cql_protect_names(layout.column_aliases)
+                clustering_names = self.cql_protect_names(layout.clustering_key_columns)
 
                 inner = []
                 for colname, coltype in zip(clustering_names, clustering_types):
@@ -1195,7 +1186,7 @@ class Shell(cmd.Cmd):
                 joiner = 'AND'
         out.write(";\n")
 
-        for col in indexed_columns:
+        for col in [ c for c in layout.columns if c.index_name is not None ]:
             out.write('\n')
             if col.index_type != 'CUSTOM':
                 out.write('CREATE INDEX %s ON %s (%s);\n'
@@ -1454,7 +1445,7 @@ class Shell(cmd.Cmd):
                     rowmap[name] = self.cql_protect_value(value)
                 else:
                     rowmap[name] = value
-            elif name in layout.column_aliases and not type.empty_binary_ok:
+            elif name in layout.clustering_key_columns and not type.empty_binary_ok:
                 rowmap[name] = 'blobAs%s(0x)' % cqltype.title()
             else:
                 rowmap[name] = 'null'

http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 3c63d09..5a117d5 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -589,7 +589,7 @@ def select_order_column_completer(ctxt, cass):
         if not keyname:
             return [Hint("Can't ORDER BY here: need to specify partition key in WHERE clause")]
     layout = get_cf_layout(ctxt, cass)
-    order_by_candidates = layout.column_aliases[:]
+    order_by_candidates = layout.clustering_key_columns[:]
     if len(order_by_candidates) > len(prev_order_cols):
         return [maybe_escape_name(order_by_candidates[len(prev_order_cols)])]
     return [Hint('No more orderable columns here.')]
@@ -601,21 +601,21 @@ def relation_token_word_completer(ctxt, cass):
 @completer_for('relation', 'rel_tokname')
 def relation_token_subject_completer(ctxt, cass):
     layout = get_cf_layout(ctxt, cass)
-    return [layout.partition_key_components[0]]
+    return [layout.partition_key_columns[0]]
 
 @completer_for('relation', 'rel_lhs')
 def select_relation_lhs_completer(ctxt, cass):
     layout = get_cf_layout(ctxt, cass)
-    filterable = set((layout.partition_key_components[0], layout.column_aliases[0]))
+    filterable = set((layout.partition_key_columns[0], layout.clustering_key_columns[0]))
     already_filtered_on = map(dequote_name, ctxt.get_binding('rel_lhs'))
-    for num in range(1, len(layout.partition_key_components)):
-        if layout.partition_key_components[num - 1] in already_filtered_on:
-            filterable.add(layout.partition_key_components[num])
+    for num in range(1, len(layout.partition_key_columns)):
+        if layout.partition_key_columns[num - 1] in already_filtered_on:
+            filterable.add(layout.partition_key_columns[num])
         else:
             break
-    for num in range(1, len(layout.column_aliases)):
-        if layout.column_aliases[num - 1] in already_filtered_on:
-            filterable.add(layout.column_aliases[num])
+    for num in range(1, len(layout.clustering_key_columns)):
+        if layout.clustering_key_columns[num - 1] in already_filtered_on:
+            filterable.add(layout.clustering_key_columns[num])
         else:
             break
     for cd in layout.columns:
@@ -647,11 +647,11 @@ syntax_rules += r'''
 def insert_colname_completer(ctxt, cass):
     layout = get_cf_layout(ctxt, cass)
     colnames = set(map(dequote_name, ctxt.get_binding('colname', ())))
-    keycols = layout.primary_key_components
+    keycols = layout.primary_key_columns
     for k in keycols:
         if k not in colnames:
             return [maybe_escape_name(k)]
-    normalcols = set([c.name for c in layout.columns]) - set(keycols) - colnames
+    normalcols = set(layout.regular_columns) - colnames
     return map(maybe_escape_name, normalcols)
 
 @completer_for('insertStatement', 'newval')
@@ -714,9 +714,7 @@ def insert_option_completer(ctxt, cass):
 @completer_for('assignment', 'updatecol')
 def update_col_completer(ctxt, cass):
     layout = get_cf_layout(ctxt, cass)
-    normals = set([cm.name for cm in layout.columns]) \
-            - set(layout.primary_key_components)
-    return map(maybe_escape_name, normals)
+    return map(maybe_escape_name, layout.regular_columns)
 
 @completer_for('assignment', 'update_rhs')
 def update_countername_completer(ctxt, cass):
@@ -790,10 +788,8 @@ def delete_opt_completer(ctxt, cass):
 
 @completer_for('deleteSelector', 'delcol')
 def delete_delcol_completer(ctxt, cass):
-        layout = get_cf_layout(ctxt, cass)
-        cols = set([c.name for c in layout.columns
-                    if c not in layout.primary_key_components])
-        return map(maybe_escape_name, cols)
+    layout = get_cf_layout(ctxt, cass)
+    return map(maybe_escape_name, layout.regular_columns)
 
 syntax_rules += r'''
 <batchStatement> ::= "BEGIN" ( "UNLOGGED" | "COUNTER" )? "BATCH"
@@ -1123,72 +1119,11 @@ def username_name_completer(ctxt, cass):
 
 CqlRuleSet.append_rules(syntax_rules)
 
-
-
-# current assumption is that all valid CQL tables match the rules in the
-# following table.
-#
-#                        non-empty     non-empty      multiple    composite
-#                       value_alias  column_aliases  key_aliases  comparator
-# ---------------------+----------------------------------------------------
-# A: single-column PK, |
-# compact storage      |   either         no            no           no
-# ---------------------+----------------------------------------------------
-# B: single-column PK, |
-# dynamic storage      |    no            no            no           yes
-# ---------------------+----------------------------------------------------
-# C: compound PK,      |
-# plain part. key,     |    yes[1]        yes           no          either
-# compact storage      |
-# ---------------------+----------------------------------------------------
-# D: compound PK,      |
-# plain part. key,     |    no            yes           no           yes
-# dynamic storage      |
-# ---------------------+----------------------------------------------------
-# E: compound PK,      |
-# multipart part. key, |
-# all key components   |   either         no            yes          no
-# go in part. key,     |
-# compact storage      |
-# ---------------------+----------------------------------------------------
-# F: compound PK,      |
-# multipart part. key, |
-# all key components   |    no            no            yes          yes
-# go in part. key,     |
-# dynamic storage      |
-# ---------------------+----------------------------------------------------
-# G: compound PK,      |
-# multipart part. key, |
-# some key components  |    yes[1]        yes           yes         either
-# not in part. key,    |
-# compact storage      |
-# ---------------------+----------------------------------------------------
-# H: compound PK,      |
-# multipart part. key, |
-# some key components  |    no            yes           yes          yes
-# not in part. key,    |
-# dynamic storage      |
-# ---------------------+----------------------------------------------------
-#
-# [1] the value_alias may be blank, but not null.
-
-# for compact storage:
-#
-# if no column aliases:
-#     comparator will be UTF8Type
-# elif one column alias:
-#     comparator will be type of that column
-# else:
-#     comparator will be composite of types of all column_aliases
-#
-# for dynamic storage:
-#
-# comparator is composite of types of column_aliases, followed by UTF8Type,
-# followed by one CTCT if there are collections.
-
 class CqlColumnDef:
     index_name = None
     index_type = None
+    component_type = 'regular'
+    component_index = None
     index_options = {}
 
     def __init__(self, name, cqltype):
@@ -1198,11 +1133,11 @@ class CqlColumnDef:
 
     @classmethod
     def from_layout(cls, layout):
-        try:
-            colname = layout[u'column_name']
-        except KeyError:
-            colname = layout[u'column']
-        c = cls(colname, lookup_casstype(layout[u'validator']))
+        c = cls(layout[u'column_name'], lookup_casstype(layout[u'validator']))
+        c.component_type = layout[u'type']
+        idx = layout[u'component_index'] # can be None
+        if idx:
+            c.component_index = int(idx)
         c.index_name = layout[u'index_name']
         c.index_type = layout[u'index_type']
         if c.index_type == 'CUSTOM':
@@ -1215,24 +1150,19 @@ class CqlColumnDef:
     __repr__ = __str__
 
 class CqlTableDef:
-    json_attrs = ('column_aliases', 'compaction_strategy_options', 'compression_parameters',
-                  'key_aliases')
-    colname_type = UTF8Type
-    column_class = CqlColumnDef
+    """Names of all columns which are grouped into the partition key"""
+    partition_key_columns = ()
 
-    """True if this CF has compact storage (isn't a CQL3 table)"""
-    compact_storage = False
+    """Names of all columns which are part of the primary key, but not grouped
+       into the partition key"""
+    clustering_key_columns = ()
 
     """Names of all columns which are part of the primary key, whether or not
        they are grouped into the partition key"""
-    primary_key_components = ()
+    primary_key_columns = ()
 
-    """Names of all columns which are grouped into the partition key"""
-    partition_key_components = ()
-
-    """Names of all columns which are part of the primary key, but not grouped
-       into the partition key"""
-    column_aliases = ()
+    """Names of all columns which aren't part of the primary key"""
+    regular_columns = ()
 
     """CqlColumnDef objects for all columns. Use .get_column() to access one
        by name."""
@@ -1248,40 +1178,33 @@ class CqlTableDef:
         of system.schema_columnfamilies, and a sequence of similar dictionaries
         from corresponding rows in system.schema_columns.
         """
-        try:
-            cfname = layout[u'columnfamily_name']
-            ksname = layout[u'keyspace_name']
-        except KeyError:
-            cfname = layout[u'columnfamily']
-            ksname = layout[u'keyspace']
-        cf = cls(name=cfname)
+        cf = cls(name=layout[u'columnfamily_name'])
+        cf.keyspace = layout[u'keyspace_name']
         for attr, val in layout.items():
             setattr(cf, attr.encode('ascii'), val)
-        cf.keyspace = ksname
-        for attr in cls.json_attrs:
-            try:
-                val = getattr(cf, attr)
-                # cfs created in 1.1 may not have key_aliases defined
-                if attr == 'key_aliases' and val is None:
-                    val = '[]'
-                setattr(cf, attr, json.loads(val))
-            except AttributeError:
-                pass
-        cf.partition_key_validator = lookup_casstype(cf.key_validator)
         cf.comparator = lookup_casstype(cf.comparator)
-        cf.default_validator = lookup_casstype(cf.default_validator)
-        cf.coldefs = cf.filter_regular_coldefs(coldefs)
-        cf.compact_storage = cf.is_compact_storage()
-        cf.key_aliases = cf.get_key_aliases()
-        cf.partition_key_components = cf.key_aliases
-        cf.column_aliases = cf.get_column_aliases()
-        cf.primary_key_components = cf.key_aliases + list(cf.column_aliases)
-        cf.columns = cf.get_columns()
-        return cf
+        for attr in ('compaction_strategy_options', 'compression_parameters'):
+            setattr(cf, attr, json.loads(getattr(cf, attr)))
+
+        # deal with columns
+        columns = map(CqlColumnDef.from_layout, coldefs)
+
+        partition_key_cols = filter(lambda c: c.component_type == u'partition_key', columns)
+        partition_key_cols.sort(key=lambda c: c.component_index)
+        cf.partition_key_columns = map(lambda c: c.name, partition_key_cols)
+
+        clustering_key_cols = filter(lambda c: c.component_type == u'clustering_key', columns)
+        clustering_key_cols.sort(key=lambda c: c.component_index)
+        cf.clustering_key_columns = map(lambda c: c.name, clustering_key_cols)
 
-    def filter_regular_coldefs(self, cols):
-        return [ c for c in cols if c.get('type', 'regular') == 'regular' ]
+        cf.primary_key_columns = cf.partition_key_columns + cf.clustering_key_columns
 
+        regular_cols = list(set(columns) - set(partition_key_cols) - set(clustering_key_cols))
+        regular_cols.sort(key=lambda c: c.name)
+        cf.regular_columns = map(lambda c: c.name, regular_cols)
+
+        cf.columns = partition_key_cols + clustering_key_cols + regular_cols
+        return cf
 
     # not perfect, but good enough; please read CFDefinition constructor comments
     # returns False if we are dealing with a CQL3 table, True otherwise.
@@ -1292,111 +1215,11 @@ class CqlTableDef:
         for subtype in self.comparator.subtypes:
             if issubclass(subtype, ColumnToCollectionType):
                 return False
-        if len(self.column_aliases) == len(self.comparator.subtypes) - 1:
+        if len(self.clustering_key_columns) == len(self.comparator.subtypes) - 1:
             if self.comparator.subtypes[-1] is UTF8Type:
                 return False
         return True
 
-    def get_key_aliases(self):
-        if not issubclass(self.partition_key_validator, CompositeType):
-            return self.key_aliases or (self.key_alias and [self.key_alias]) or [u'key']
-        expected = len(self.partition_key_validator.subtypes)
-        # key, key2, key3, ..., keyN
-        aliases = [u'key'] + [ u'key' + str(i) for i in range(2, expected + 1) ]
-        # append the missing (non-renamed) aliases (if any)
-        return self.key_aliases + aliases[len(self.key_aliases):]
-
-    def get_column_aliases(self):
-        # CQL3 table
-        if not self.compact_storage:
-            return self.column_aliases
-        if not issubclass(self.comparator, CompositeType):
-            # static cf
-            if self.coldefs:
-                return []
-            else:
-                return self.column_aliases or [u'column1']
-        expected = len(self.comparator.subtypes)
-        # column1, column2, column3, ..., columnN
-        aliases = [ u'column' + str(i) for i in range(1, expected + 1) ]
-        # append the missing (non-renamed) aliases (if any)
-        return self.column_aliases + aliases[len(self.column_aliases):]
-
-    def get_columns(self):
-        if self.compact_storage:
-            return self.get_columns_compact()
-        else:
-            return self.get_columns_cql3()
-
-    # dense composite or dynamic cf or static cf (technically not compact).
-    def get_columns_compact(self):
-        if issubclass(self.partition_key_validator, CompositeType):
-            partkey_types = self.partition_key_validator.subtypes
-        else:
-            partkey_types = [self.partition_key_validator]
-        partkey_cols = map(self.column_class, self.partition_key_components, partkey_types)
-
-        if len(self.column_aliases) == 0:
-            if self.comparator is not UTF8Type:
-                warn(UnexpectedTableStructure("Compact storage CF %s has no column aliases,"
-                                              " but comparator is not UTF8Type." % (self.name,)))
-            colalias_types = []
-        elif issubclass(self.comparator, CompositeType):
-            colalias_types = self.comparator.subtypes
-        else:
-            colalias_types = [self.comparator]
-        if len(colalias_types) != len(self.column_aliases):
-            warn(UnexpectedTableStructure("Compact storage CF comparator-types %r is not"
-                                          " the same length as its column_aliases %r"
-                                          % (colalias_types, self.column_aliases)))
-        colalias_cols = map(self.column_class, self.column_aliases, colalias_types)
-
-        if self.value_alias is not None:
-            if self.coldefs:
-                warn(UnexpectedTableStructure("Compact storage CF has both a value_alias"
-                                              " (%r) and entries in system.schema_columns"
-                                              % (self.value_alias,)))
-            if self.value_alias == '':
-                value_cols = []
-            else:
-                value_cols = [self.column_class(self.value_alias, self.default_validator)]
-        elif self.value_alias is None and not self.coldefs:
-            value_cols = [self.column_class("value", self.default_validator)]
-        else:
-            value_cols = map(self.column_class.from_layout, self.coldefs)
-            value_cols.sort(key=lambda c: c.name)
-
-        return partkey_cols + colalias_cols + value_cols
-
-    # sparse composite (CQL3 table).
-    def get_columns_cql3(self):
-        if issubclass(self.partition_key_validator, CompositeType):
-            partkey_types = self.partition_key_validator.subtypes
-        else:
-            partkey_types = [self.partition_key_validator]
-        partkey_cols = map(self.column_class, self.partition_key_components, partkey_types)
-
-        for subtype in self.comparator.subtypes[:-1]:
-            if issubclass(subtype, ColumnToCollectionType):
-                warn(UnexpectedTableStructure("ColumnToCollectionType found, but not in "
-                                              "last position inside composite comparator"))
-        coltypes = list(self.comparator.subtypes)
-        if issubclass(coltypes[-1], ColumnToCollectionType):
-            # all this information should be available in schema_columns
-            coltypes.pop(-1)
-        if len(coltypes) != len(self.column_aliases) + 1 or coltypes[-1] is not UTF8Type:
-            warn(UnexpectedTableStructure("CQL3 CF does not have UTF8Type"
-                                          " added to comparator"))
-        colalias_cols = map(self.column_class, self.column_aliases, coltypes[:-1])
-
-        if self.value_alias is not None:
-            warn(UnexpectedTableStructure("CQL3 CF has a value_alias (%r)"
-                                          % (self.value_alias,)))
-        value_cols = map(self.column_class.from_layout, self.coldefs)
-        value_cols.sort(key=lambda c: c.name)
-
-        return partkey_cols + colalias_cols + value_cols
-
     def is_counter_col(self, colname):
         try:
             return bool(self.get_column(colname).cqltype is CounterColumnType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/pylib/cqlshlib/test/test_cqlsh_output.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index 8aef154..11ad949 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -638,7 +638,7 @@ class TestCqlshOutput(BaseTestCase):
         table_desc3 = dedent("""
 
             CREATE TABLE has_all_types (
-              num int PRIMARY KEY,
+              num int,
               asciicol ascii,
               bigintcol bigint,
               blobcol blob,
@@ -651,7 +651,8 @@ class TestCqlshOutput(BaseTestCase):
               timestampcol timestamp,
               uuidcol uuid,
               varcharcol text,
-              varintcol varint
+              varintcol varint,
+              PRIMARY KEY (num)
             ) WITH
               bloom_filter_fp_chance=0.010000 AND
               caching='KEYS_ONLY' AND

http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 0ecb252..9de7106 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -90,7 +90,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
         if (cd.getValidator().isCollection() && !isCustom)
             throw new InvalidRequestException("Indexes on collections are no yet supported");
 
-        if (cd.type == ColumnDefinition.Type.PARTITION_KEY && (cd.componentIndex == null || cd.componentIndex == 0))
+        if (cd.type == ColumnDefinition.Type.PARTITION_KEY && cd.componentIndex == null)
             throw new InvalidRequestException(String.format("Cannot add secondary index to already primarily indexed column %s", columnName));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/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 fbabf2f..5aced28 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -347,6 +347,7 @@ public class SelectStatement implements CQLStatement
 
             RowPosition startKey = RowPosition.forKey(startKeyBytes, p);
             RowPosition finishKey = RowPosition.forKey(finishKeyBytes, p);
+
             if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum(p))
                 return null;
 
@@ -488,6 +489,12 @@ public class SelectStatement implements CQLStatement
 
     private ByteBuffer getKeyBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
+        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the first
+        // component of a composite partition key).
+        for (int i = 0; i < keyRestrictions.length; i++)
+            if (keyRestrictions[i] == null)
+                return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
         // We deal with IN queries for keys in other places, so we know buildBound will return only one result
         return buildBound(b, cfDef.keys.values(), keyRestrictions, false, cfDef.getKeyNameBuilder(), variables).get(0);
     }
@@ -744,6 +751,7 @@ public class SelectStatement implements CQLStatement
                     // We don't allow restricting a VALUE_ALIAS for now in prepare.
                     throw new AssertionError();
             }
+
             if (restriction.isEquality())
             {
                 assert restriction.eqValues.size() == 1; // IN is not supported for indexed columns.
@@ -1073,6 +1081,7 @@ public class SelectStatement implements CQLStatement
              *   - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value in CQL so far)
              */
             boolean hasQueriableIndex = false;
+            boolean hasQueriableClusteringColumnIndex = false;
             for (Relation rel : whereClause)
             {
                 CFDefinition.Name name = cfDef.get(rel.getEntity());
@@ -1086,10 +1095,11 @@ public class SelectStatement implements CQLStatement
 
                 ColumnDefinition def = cfDef.cfm.getColumnDefinition(name.name.key);
                 stmt.restrictedNames.add(name);
-                if (def.isIndexed())
+                if (def.isIndexed() && rel.operator() == Relation.Type.EQ)
                 {
-                    if (rel.operator() == Relation.Type.EQ)
-                        hasQueriableIndex = true;
+                    hasQueriableIndex = true;
+                    if (name.kind == CFDefinition.Name.Kind.COLUMN_ALIAS)
+                        hasQueriableClusteringColumnIndex = true;
                 }
 
                 switch (name.kind)
@@ -1116,13 +1126,12 @@ public class SelectStatement implements CQLStatement
             // But we still need to know 2 things:
             //   - If we don't have a queriable index, is the query ok
             //   - Is it queriable without 2ndary index, which is always more efficient
-            // If a component of the partition key is restricted by a non-EQ relation, all preceding
-            // components must have a EQ, and all following must have no restriction
-            boolean shouldBeDone = false;
+            // If a component of the partition key is restricted by a relation, all preceding
+            // components must have a EQ. Only the last partition key component can be in IN relation.
+            boolean canRestrictFurtherComponents = true;
             CFDefinition.Name previous = null;
             stmt.keyIsInRelation = false;
             Iterator<CFDefinition.Name> iter = cfDef.keys.values().iterator();
-            int lastRestrictedPartitionKey = stmt.keyRestrictions.length - 1;
             for (int i = 0; i < stmt.keyRestrictions.length; i++)
             {
                 CFDefinition.Name cname = iter.next();
@@ -1130,27 +1139,25 @@ public class SelectStatement implements CQLStatement
 
                 if (restriction == null)
                 {
-                    if (!shouldBeDone)
-                        lastRestrictedPartitionKey = i - 1;
-
                     if (stmt.onToken)
                         throw new InvalidRequestException("The token() function must be applied to all partition key components or none of them");
 
-                    // The only time not restricting a key part is allowed is if none are restricted
-                    if (i > 0 && stmt.keyRestrictions[i-1] != null)
+                    // The only time not restricting a key part is allowed is if none are restricted or an index is used.
+                    if (i > 0 && stmt.keyRestrictions[i - 1] != null)
                     {
                         if (hasQueriableIndex)
                         {
                             stmt.usesSecondaryIndexing = true;
+                            stmt.isKeyRange = true;
                             break;
                         }
                         throw new InvalidRequestException(String.format("Partition key part %s must be restricted since preceding part is", cname));
                     }
 
                     stmt.isKeyRange = true;
-                    shouldBeDone = true;
+                    canRestrictFurtherComponents = false;
                 }
-                else if (shouldBeDone)
+                else if (!canRestrictFurtherComponents)
                 {
                     if (hasQueriableIndex)
                     {
@@ -1188,13 +1195,17 @@ public class SelectStatement implements CQLStatement
                 previous = cname;
             }
 
-            // If a cluster key column is restricted by a non-EQ relation, all preceding
+            // All (or none) of the partition key columns have been specified;
+            // hence there is no need to turn these restrictions into index expressions.
+            if (!stmt.usesSecondaryIndexing)
+                stmt.restrictedNames.removeAll(cfDef.keys.values());
+
+            // If a clustering key column is restricted by a non-EQ relation, all preceding
             // columns must have a EQ, and all following must have no restriction. Unless
             // the column is indexed that is.
-            shouldBeDone = false;
+            canRestrictFurtherComponents = true;
             previous = null;
             iter = cfDef.columns.values().iterator();
-            int lastRestrictedClusteringKey = stmt.columnRestrictions.length - 1;
             for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
                 CFDefinition.Name cname = iter.next();
@@ -1202,67 +1213,56 @@ public class SelectStatement implements CQLStatement
 
                 if (restriction == null)
                 {
-                    if (!shouldBeDone)
-                        lastRestrictedClusteringKey = i - 1;
-                    shouldBeDone = true;
+                    canRestrictFurtherComponents = false;
                 }
-                else
+                else if (!canRestrictFurtherComponents)
                 {
-                    if (shouldBeDone)
-                    {
-                        if (hasQueriableIndex)
-                        {
-                            stmt.usesSecondaryIndexing = true;
-                            break;
-                        }
-                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
-                    }
-                    else if (!restriction.isEquality())
+                    if (hasQueriableIndex)
                     {
-                        lastRestrictedClusteringKey = i;
-                        shouldBeDone = true;
-                        // For non-composite slices, we don't support internally the difference between exclusive and
-                        // inclusive bounds, so we deal with it manually.
-                        if (!cfDef.isComposite && (!restriction.isInclusive(Bound.START) || !restriction.isInclusive(Bound.END)))
-                            stmt.sliceRestriction = restriction;
+                        stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+                        break;
                     }
+                    throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
+                }
+                else if (!restriction.isEquality())
+                {
+                    canRestrictFurtherComponents = false;
+                    // For non-composite slices, we don't support internally the difference between exclusive and
+                    // inclusive bounds, so we deal with it manually.
+                    if (!cfDef.isComposite && (!restriction.isInclusive(Bound.START) || !restriction.isInclusive(Bound.END)))
+                        stmt.sliceRestriction = restriction;
+                }
+                else if (restriction.isINRestriction())
+                {
                     // We only support IN for the last name and for compact storage so far
                     // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
-                    else if (restriction.isINRestriction())
-                    {
-                        if (i != stmt.columnRestrictions.length - 1)
-                            throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cname));
-                        else if (stmt.selectACollection())
-                            throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cname));
-                    }
+                    if (i != stmt.columnRestrictions.length - 1)
+                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cname));
+                    else if (stmt.selectACollection())
+                        throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cname));
                 }
 
                 previous = cname;
             }
 
+            // Covers indexes on the first clustering column (among others).
+            if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
+                stmt.usesSecondaryIndexing = true;
+
+            if (!stmt.usesSecondaryIndexing)
+                stmt.restrictedNames.removeAll(cfDef.columns.values());
+
             // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
             // there is restrictions not covered by the PK.
             if (!stmt.metadataRestrictions.isEmpty())
             {
                 if (!hasQueriableIndex)
                     throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
-
                 stmt.usesSecondaryIndexing = true;
             }
 
-            if (stmt.usesSecondaryIndexing)
-            {
-                if (stmt.keyIsInRelation)
-                    throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
-            }
-
-            iter = cfDef.keys.values().iterator();
-            for (int i = 0; i < lastRestrictedPartitionKey + 1; i++)
-                stmt.restrictedNames.remove(iter.next());
-
-            iter = cfDef.columns.values().iterator();
-            for (int i = 0; i < lastRestrictedClusteringKey + 1; i++)
-                stmt.restrictedNames.remove(iter.next());
+            if (stmt.usesSecondaryIndexing && stmt.keyIsInRelation)
+                throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
 
             if (!stmt.parameters.orderings.isEmpty())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/703bd684/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
index b767d6c..bdb544c 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
@@ -76,9 +76,10 @@ public class CompositesIndexOnClusteringKey extends CompositesIndex
         ByteBuffer[] components = baseComparator.split(columnName);
         CompositeType.Builder builder = getIndexComparator().builder();
         builder.add(rowKey);
-        for (int i = 0; i < columnDef.componentIndex; i++)
+
+        for (int i = 0; i < Math.min(components.length, columnDef.componentIndex); i++)
             builder.add(components[i]);
-        for (int i = columnDef.componentIndex + 1; i < ckCount; i++)
+        for (int i = columnDef.componentIndex + 1; i < Math.min(components.length, ckCount); i++)
             builder.add(components[i]);
         return builder;
     }