You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2014/05/29 12:55:00 UTC

[1/4] git commit: Fix bug with some IN queries missing results

Repository: cassandra
Updated Branches:
  refs/heads/trunk 56c2f0ab1 -> af593090b


Fix bug with some IN queries missing results

patch by slebresne; reviewed by thobbs for CASSANDRA-7105


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

Branch: refs/heads/trunk
Commit: 7bbeb5aa2d1064281589b98306f950550bd47e60
Parents: a374821
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu May 29 12:48:06 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu May 29 12:48:42 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                                      | 1 +
 .../org/apache/cassandra/cql3/statements/SelectStatement.java    | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7bbeb5aa/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e7d7028..619c219 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * raise streaming phi convict threshold level (CASSANDRA-7063)
  * reduce garbage creation in calculatePendingRanges (CASSANDRA-7191)
  * exit CQLSH with error status code if script fails (CASSANDRA-6344)
+ * Fix bug with some IN queries missig results (CASSANDRA-7105)
 
 
 1.2.16

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7bbeb5aa/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 e058cff..02833e7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -626,7 +626,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                             throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
                         ColumnNameBuilder copy = builder.copy().add(val);
                         // See below for why this
-                        s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
+                        s.add((eocBound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
                     }
                     return new ArrayList<ByteBuffer>(s);
                 }
@@ -652,7 +652,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
         // case using the eoc would be bad, since for the random partitioner we have no guarantee that
         // builder.buildAsEndOfRange() will sort after builder.build() (see #5240).
-        return Collections.singletonList((bound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
+        return Collections.singletonList((eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
     }
 
     private List<ByteBuffer> getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException


[3/4] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by sl...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java


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

Branch: refs/heads/trunk
Commit: ef41567fa0bd0bc762a865ac44cb318b532da38f
Parents: 0932ed6 975145c
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu May 29 12:54:33 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu May 29 12:54:33 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                                      | 1 +
 .../org/apache/cassandra/cql3/statements/SelectStatement.java    | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef41567f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2cb2f5d,d83b5af..8448ea6
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -82,124 -50,26 +82,125 @@@ Merged from 1.2
   * Add Cloudstack snitch (CASSANDRA-7147)
   * Update system.peers correctly when relocating tokens (CASSANDRA-7126)
   * Add Google Compute Engine snitch (CASSANDRA-7132)
 - * Fix nodetool display with vnodes (CASSANDRA-7082)
 - * Fix schema concurrency exceptions (CASSANDRA-6841)
 - * Fix BatchlogManager#deleteBatch() use of millisecond timsestamps
 -   (CASSANDRA-6822)
 - * Fix batchlog to account for CF truncation records (CASSANDRA-6999)
 - * Fix CQLSH parsing of functions and BLOB literals (CASSANDRA-7018)
 - * Require nodetool rebuild_index to specify index names (CASSANDRA-7038)
 - * Ensure that batchlog and hint timeouts do not produce hints (CASSANDRA-7058)
 - * Always clean up references in SerializingCache (CASSANDRA-6994)
 - * Don't shut MessagingService down when replacing a node (CASSANDRA-6476)
 - * fix npe when doing -Dcassandra.fd_initial_value_ms (CASSANDRA-6751)
 - * Preserves CQL metadata when updating table from thrift (CASSANDRA-6831)
   * remove duplicate query for local tokens (CASSANDRA-7182)
 - * raise streaming phi convict threshold level (CASSANDRA-7063)
 - * reduce garbage creation in calculatePendingRanges (CASSANDRA-7191)
   * exit CQLSH with error status code if script fails (CASSANDRA-6344)
+  * Fix bug with some IN queries missig results (CASSANDRA-7105)
  
  
 -2.0.7
 +2.1.0-beta2
 + * Increase default CL space to 8GB (CASSANDRA-7031)
 + * Add range tombstones to read repair digests (CASSANDRA-6863)
 + * Fix BTree.clear for large updates (CASSANDRA-6943)
 + * Fail write instead of logging a warning when unable to append to CL
 +   (CASSANDRA-6764)
 + * Eliminate possibility of CL segment appearing twice in active list 
 +   (CASSANDRA-6557)
 + * Apply DONTNEED fadvise to commitlog segments (CASSANDRA-6759)
 + * Switch CRC component to Adler and include it for compressed sstables 
 +   (CASSANDRA-4165)
 + * Allow cassandra-stress to set compaction strategy options (CASSANDRA-6451)
 + * Add broadcast_rpc_address option to cassandra.yaml (CASSANDRA-5899)
 + * Auto reload GossipingPropertyFileSnitch config (CASSANDRA-5897)
 + * Fix overflow of memtable_total_space_in_mb (CASSANDRA-6573)
 + * Fix ABTC NPE and apply update function correctly (CASSANDRA-6692)
 + * Allow nodetool to use a file or prompt for password (CASSANDRA-6660)
 + * Fix AIOOBE when concurrently accessing ABSC (CASSANDRA-6742)
 + * Fix assertion error in ALTER TYPE RENAME (CASSANDRA-6705)
 + * Scrub should not always clear out repaired status (CASSANDRA-5351)
 + * Improve handling of range tombstone for wide partitions (CASSANDRA-6446)
 + * Fix ClassCastException for compact table with composites (CASSANDRA-6738)
 + * Fix potentially repairing with wrong nodes (CASSANDRA-6808)
 + * Change caching option syntax (CASSANDRA-6745)
 + * Fix stress to do proper counter reads (CASSANDRA-6835)
 + * Fix help message for stress counter_write (CASSANDRA-6824)
 + * Fix stress smart Thrift client to pick servers correctly (CASSANDRA-6848)
 + * Add logging levels (minimal, normal or verbose) to stress tool (CASSANDRA-6849)
 + * Fix race condition in Batch CLE (CASSANDRA-6860)
 + * Improve cleanup/scrub/upgradesstables failure handling (CASSANDRA-6774)
 + * ByteBuffer write() methods for serializing sstables (CASSANDRA-6781)
 + * Proper compare function for CollectionType (CASSANDRA-6783)
 + * Update native server to Netty 4 (CASSANDRA-6236)
 + * Fix off-by-one error in stress (CASSANDRA-6883)
 + * Make OpOrder AutoCloseable (CASSANDRA-6901)
 + * Remove sync repair JMX interface (CASSANDRA-6900)
 + * Add multiple memory allocation options for memtables (CASSANDRA-6689, 6694)
 + * Remove adjusted op rate from stress output (CASSANDRA-6921)
 + * Add optimized CF.hasColumns() implementations (CASSANDRA-6941)
 + * Serialize batchlog mutations with the version of the target node
 +   (CASSANDRA-6931)
 + * Optimize CounterColumn#reconcile() (CASSANDRA-6953)
 + * Properly remove 1.2 sstable support in 2.1 (CASSANDRA-6869)
 + * Lock counter cells, not partitions (CASSANDRA-6880)
 + * Track presence of legacy counter shards in sstables (CASSANDRA-6888)
 + * Ensure safe resource cleanup when replacing sstables (CASSANDRA-6912)
 + * Add failure handler to async callback (CASSANDRA-6747)
 + * Fix AE when closing SSTable without releasing reference (CASSANDRA-7000)
 + * Clean up IndexInfo on keyspace/table drops (CASSANDRA-6924)
 + * Only snapshot relative SSTables when sequential repair (CASSANDRA-7024)
 + * Require nodetool rebuild_index to specify index names (CASSANDRA-7038)
 + * fix cassandra stress errors on reads with native protocol (CASSANDRA-7033)
 + * Use OpOrder to guard sstable references for reads (CASSANDRA-6919)
 + * Preemptive opening of compaction result (CASSANDRA-6916)
 + * Multi-threaded scrub/cleanup/upgradesstables (CASSANDRA-5547)
 + * Optimize cellname comparison (CASSANDRA-6934)
 + * Native protocol v3 (CASSANDRA-6855)
 + * Optimize Cell liveness checks and clean up Cell (CASSANDRA-7119)
 + * Support consistent range movements (CASSANDRA-2434)
 +Merged from 2.0:
 + * Avoid race-prone second "scrub" of system keyspace (CASSANDRA-6797)
 + * Pool CqlRecordWriter clients by inetaddress rather than Range
 +   (CASSANDRA-6665)
 + * Fix compaction_history timestamps (CASSANDRA-6784)
 + * Compare scores of full replica ordering in DES (CASSANDRA-6683)
 + * fix CME in SessionInfo updateProgress affecting netstats (CASSANDRA-6577)
 + * Allow repairing between specific replicas (CASSANDRA-6440)
 + * Allow per-dc enabling of hints (CASSANDRA-6157)
 + * Add compatibility for Hadoop 0.2.x (CASSANDRA-5201)
 + * Fix EstimatedHistogram races (CASSANDRA-6682)
 + * Failure detector correctly converts initial value to nanos (CASSANDRA-6658)
 + * Add nodetool taketoken to relocate vnodes (CASSANDRA-4445)
 + * Fix upgradesstables NPE for non-CF-based indexes (CASSANDRA-6645)
 + * Expose bulk loading progress over JMX (CASSANDRA-4757)
 + * Correctly handle null with IF conditions and TTL (CASSANDRA-6623)
 + * Account for range/row tombstones in tombstone drop
 +   time histogram (CASSANDRA-6522)
 + * Stop CommitLogSegment.close() from calling sync() (CASSANDRA-6652)
 + * Make commitlog failure handling configurable (CASSANDRA-6364)
 + * Avoid overlaps in LCS (CASSANDRA-6688)
 + * Improve support for paginating over composites (CASSANDRA-4851)
 + * Fix count(*) queries in a mixed cluster (CASSANDRA-6707)
 + * Improve repair tasks(snapshot, differencing) concurrency (CASSANDRA-6566)
 + * Fix replaying pre-2.0 commit logs (CASSANDRA-6714)
 + * Add static columns to CQL3 (CASSANDRA-6561)
 + * Optimize single partition batch statements (CASSANDRA-6737)
 + * Disallow post-query re-ordering when paging (CASSANDRA-6722)
 + * Fix potential paging bug with deleted columns (CASSANDRA-6748)
 + * Fix NPE on BulkLoader caused by losing StreamEvent (CASSANDRA-6636)
 + * Fix truncating compression metadata (CASSANDRA-6791)
 + * Add CMSClassUnloadingEnabled JVM option (CASSANDRA-6541)
 + * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)
 + * Fix upgradesstables NPE for non-CF-based indexes (CASSANDRA-6645)
 + * Fix UPDATE updating PRIMARY KEY columns implicitly (CASSANDRA-6782)
 + * Fix IllegalArgumentException when updating from 1.2 with SuperColumns
 +   (CASSANDRA-6733)
 + * FBUtilities.singleton() should use the CF comparator (CASSANDRA-6778)
 + * Fix CQLSStableWriter.addRow(Map<String, Object>) (CASSANDRA-6526)
 + * Fix HSHA server introducing corrupt data (CASSANDRA-6285)
 + * Fix CAS conditions for COMPACT STORAGE tables (CASSANDRA-6813)
 + * Fix saving triggers to schema (CASSANDRA-6789)
 + * Fix trigger mutations when base mutation list is immutable (CASSANDRA-6790)
 + * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
 + * Fix static counter columns (CASSANDRA-6827)
 + * Restore expiring->deleted (cell) compaction optimization (CASSANDRA-6844)
 + * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 + * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
 + * Read message id as string from earlier versions (CASSANDRA-6840)
 + * Properly use the Paxos consistency for (non-protocol) batch (CASSANDRA-6837)
 + * Starting threads in OutboundTcpConnectionPool constructor causes race conditions (CASSANDRA-7177)
 + * Allow overriding cassandra-rackdc.properties file (CASSANDRA-7072)
 + * Set JMX RMI port to 7199 (CASSANDRA-7087)
 + * Use LOCAL_QUORUM for data reads at LOCAL_SERIAL (CASSANDRA-6939)
 + * Log a warning for large batches (CASSANDRA-6487)
   * Put nodes in hibernate when join_ring is false (CASSANDRA-6961)
   * Avoid early loading of non-system keyspaces before compaction-leftovers 
     cleanup at startup (CASSANDRA-6913)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ef41567f/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 9d63389,2710f78..4426bd4
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -842,10 -859,10 +842,10 @@@ public class SelectStatement implement
                      for (ByteBuffer val : values)
                      {
                          if (val == null)
 -                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
 -                        ColumnNameBuilder copy = builder.copy().add(val);
 +                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
 +                        Composite prefix = builder.buildWith(val);
                          // See below for why this
-                         s.add((b == Bound.END && builder.remainingCount() > 0) ? prefix.end() : prefix);
 -                        s.add((eocBound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
++                        s.add((eocBound == Bound.END && builder.remainingCount() > 0) ? prefix.end() : prefix);
                      }
                      return new ArrayList<>(s);
                  }
@@@ -861,36 -878,16 +861,36 @@@
          // it would be harmless to do it. However, we use this method got the partition key too. And when a query
          // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
          // case using the eoc would be bad, since for the random partitioner we have no guarantee that
 -        // builder.buildAsEndOfRange() will sort after builder.build() (see #5240).
 -        return Collections.singletonList((eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
 +        // prefix.end() will sort after prefix (see #5240).
 +        Composite prefix = builder.build();
-         return Collections.singletonList(bound == Bound.END && builder.remainingCount() > 0 ? prefix.end() : prefix);
++        return Collections.singletonList(eocBound == Bound.END && builder.remainingCount() > 0 ? prefix.end() : prefix);
      }
  
 -    private List<ByteBuffer> buildMultiColumnSliceBound(Bound bound,
 -                                                        Collection<CFDefinition.Name> names,
 -                                                        MultiColumnRestriction.Slice slice,
 -                                                        boolean isReversed,
 -                                                        ColumnNameBuilder builder,
 -                                                        List<ByteBuffer> variables) throws InvalidRequestException
 +    private static Composite.EOC eocForRelation(Relation.Type op)
 +    {
 +        switch (op)
 +        {
 +            case LT:
 +                // < X => using startOf(X) as finish bound
 +                return Composite.EOC.START;
 +            case GT:
 +            case LTE:
 +                // > X => using endOf(X) as start bound
 +                // <= X => using endOf(X) as finish bound
 +                return Composite.EOC.END;
 +            default:
 +                // >= X => using X as start bound (could use START_OF too)
 +                // = X => using X
 +                return Composite.EOC.NONE;
 +        }
 +    }
 +
 +    private static List<Composite> buildMultiColumnSliceBound(Bound bound,
 +                                                              List<ColumnDefinition> defs,
 +                                                              MultiColumnRestriction.Slice slice,
 +                                                              boolean isReversed,
 +                                                              CBuilder builder,
 +                                                              QueryOptions options) throws InvalidRequestException
      {
          Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
  


[4/4] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: af593090b9695eb1c1f5c8ab337b48c85ae548ca
Parents: 56c2f0a ef41567
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu May 29 12:54:48 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu May 29 12:54:48 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                                      | 1 +
 .../org/apache/cassandra/cql3/statements/SelectStatement.java    | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


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


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

Posted by sl...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0


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

Branch: refs/heads/trunk
Commit: 975145c2779a5d99c9f5f192f0693bfb40a9001c
Parents: 50599d2 7bbeb5a
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu May 29 12:49:19 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu May 29 12:49:19 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                                      | 1 +
 .../org/apache/cassandra/cql3/statements/SelectStatement.java    | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/975145c2/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 523dda3,619c219..d83b5af
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -66,61 -24,10 +66,62 @@@ Merged from 1.2
   * raise streaming phi convict threshold level (CASSANDRA-7063)
   * reduce garbage creation in calculatePendingRanges (CASSANDRA-7191)
   * exit CQLSH with error status code if script fails (CASSANDRA-6344)
+  * Fix bug with some IN queries missig results (CASSANDRA-7105)
  
  
 -1.2.16
 +2.0.7
 + * Put nodes in hibernate when join_ring is false (CASSANDRA-6961)
 + * Avoid early loading of non-system keyspaces before compaction-leftovers 
 +   cleanup at startup (CASSANDRA-6913)
 + * Restrict Windows to parallel repairs (CASSANDRA-6907)
 + * (Hadoop) Allow manually specifying start/end tokens in CFIF (CASSANDRA-6436)
 + * Fix NPE in MeteredFlusher (CASSANDRA-6820)
 + * Fix race processing range scan responses (CASSANDRA-6820)
 + * Allow deleting snapshots from dropped keyspaces (CASSANDRA-6821)
 + * Add uuid() function (CASSANDRA-6473)
 + * Omit tombstones from schema digests (CASSANDRA-6862)
 + * Include correct consistencyLevel in LWT timeout (CASSANDRA-6884)
 + * Lower chances for losing new SSTables during nodetool refresh and
 +   ColumnFamilyStore.loadNewSSTables (CASSANDRA-6514)
 + * Add support for DELETE ... IF EXISTS to CQL3 (CASSANDRA-5708)
 + * Update hadoop_cql3_word_count example (CASSANDRA-6793)
 + * Fix handling of RejectedExecution in sync Thrift server (CASSANDRA-6788)
 + * Log more information when exceeding tombstone_warn_threshold (CASSANDRA-6865)
 + * Fix truncate to not abort due to unreachable fat clients (CASSANDRA-6864)
 + * Fix schema concurrency exceptions (CASSANDRA-6841)
 + * Fix leaking validator FH in StreamWriter (CASSANDRA-6832)
 + * Fix saving triggers to schema (CASSANDRA-6789)
 + * Fix trigger mutations when base mutation list is immutable (CASSANDRA-6790)
 + * Fix accounting in FileCacheService to allow re-using RAR (CASSANDRA-6838)
 + * Fix static counter columns (CASSANDRA-6827)
 + * Restore expiring->deleted (cell) compaction optimization (CASSANDRA-6844)
 + * Fix CompactionManager.needsCleanup (CASSANDRA-6845)
 + * Correctly compare BooleanType values other than 0 and 1 (CASSANDRA-6779)
 + * Read message id as string from earlier versions (CASSANDRA-6840)
 + * Properly use the Paxos consistency for (non-protocol) batch (CASSANDRA-6837)
 + * Add paranoid disk failure option (CASSANDRA-6646)
 + * Improve PerRowSecondaryIndex performance (CASSANDRA-6876)
 + * Extend triggers to support CAS updates (CASSANDRA-6882)
 + * Static columns with IF NOT EXISTS don't always work as expected (CASSANDRA-6873)
 + * Fix paging with SELECT DISTINCT (CASSANDRA-6857)
 + * Fix UnsupportedOperationException on CAS timeout (CASSANDRA-6923)
 + * Improve MeteredFlusher handling of MF-unaffected column families
 +   (CASSANDRA-6867)
 + * Add CqlRecordReader using native pagination (CASSANDRA-6311)
 + * Add QueryHandler interface (CASSANDRA-6659)
 + * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945)
 + * Make sure upgradesstables keeps sstable level (CASSANDRA-6958)
 + * Fix LIMIT with static columns (CASSANDRA-6956)
 + * Fix clash with CQL column name in thrift validation (CASSANDRA-6892)
 + * Fix error with super columns in mixed 1.2-2.0 clusters (CASSANDRA-6966)
 + * Fix bad skip of sstables on slice query with composite start/finish (CASSANDRA-6825)
 + * Fix unintended update with conditional statement (CASSANDRA-6893)
 + * Fix map element access in IF (CASSANDRA-6914)
 + * Avoid costly range calculations for range queries on system keyspaces
 +   (CASSANDRA-6906)
 + * Fix SSTable not released if stream session fails (CASSANDRA-6818)
 + * Avoid build failure due to ANTLR timeout (CASSANDRA-6991)
 +Merged from 1.2:
   * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816)
   * add extra SSL cipher suites (CASSANDRA-6613)
   * fix nodetool getsstables for blob PK (CASSANDRA-6803)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/975145c2/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 8f061d2,02833e7..2710f78
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -862,12 -626,12 +862,12 @@@ public class SelectStatement implement
                              throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
                          ColumnNameBuilder copy = builder.copy().add(val);
                          // See below for why this
-                         s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
+                         s.add((eocBound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
                      }
 -                    return new ArrayList<ByteBuffer>(s);
 +                    return new ArrayList<>(s);
                  }
  
 -                ByteBuffer val = r.eqValues.get(0).bindAndGet(variables);
 +                ByteBuffer val = values.get(0);
                  if (val == null)
                      throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
                  builder.add(val);
@@@ -879,98 -652,9 +879,98 @@@
          // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
          // case using the eoc would be bad, since for the random partitioner we have no guarantee that
          // builder.buildAsEndOfRange() will sort after builder.build() (see #5240).
-         return Collections.singletonList((bound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
+         return Collections.singletonList((eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
      }
  
 +    private List<ByteBuffer> buildMultiColumnSliceBound(Bound bound,
 +                                                        Collection<CFDefinition.Name> names,
 +                                                        MultiColumnRestriction.Slice slice,
 +                                                        boolean isReversed,
 +                                                        ColumnNameBuilder builder,
 +                                                        List<ByteBuffer> variables) throws InvalidRequestException
 +    {
 +        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
 +
 +        Iterator<CFDefinition.Name> iter = names.iterator();
 +        CFDefinition.Name firstName = iter.next();
 +        // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
 +        // and DESCENDING orders.  This stores the bound for the first component; we will re-use it for all following
 +        // components, even if they don't match the first component's reversal/non-reversal.  Note that this does *not*
 +        // guarantee correct query results, it just preserves the previous behavior.
 +        Bound firstComponentBound = isReversed == isReversedType(firstName) ? bound : Bound.reverse(bound);
 +
 +        if (!slice.hasBound(firstComponentBound))
 +            return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
 +                    ? builder.buildAsEndOfRange()
 +                    : builder.build());
 +
 +        List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, variables);
 +        builder.add(vals.get(firstName.position));
 +
 +        while(iter.hasNext())
 +        {
 +            CFDefinition.Name name = iter.next();
 +            if (name.position >= vals.size())
 +                break;
 +
 +            builder.add(vals.get(name.position));
 +        }
 +        Relation.Type relType = slice.getRelation(eocBound, firstComponentBound);
 +        return Collections.singletonList(builder.buildForRelation(relType));
 +    }
 +
 +    private List<ByteBuffer> buildMultiColumnInBound(Bound bound,
 +                                                     Collection<CFDefinition.Name> names,
 +                                                     MultiColumnRestriction.IN restriction,
 +                                                     boolean isReversed,
 +                                                     ColumnNameBuilder builder,
 +                                                     List<ByteBuffer> variables) throws InvalidRequestException
 +    {
 +        List<List<ByteBuffer>> splitInValues = restriction.splitValues(variables);
 +
 +        // The IN query might not have listed the values in comparator order, so we need to re-sort
 +        // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
 +        TreeSet<ByteBuffer> inValues = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
 +        Iterator<CFDefinition.Name> iter = names.iterator();
 +        for (List<ByteBuffer> components : splitInValues)
 +        {
 +            ColumnNameBuilder nameBuilder = builder.copy();
 +            for (ByteBuffer component : components)
 +                nameBuilder.add(component);
 +
 +            Bound b = isReversed == isReversedType(iter.next()) ? bound : Bound.reverse(bound);
 +            inValues.add((bound == Bound.END && nameBuilder.remainingCount() > 0) ? nameBuilder.buildAsEndOfRange() : nameBuilder.build());
 +        }
 +        return new ArrayList<>(inValues);
 +    }
 +
 +    private List<ByteBuffer> buildMultiColumnEQBound(Bound bound, MultiColumnRestriction.EQ restriction, boolean isReversed, ColumnNameBuilder builder, List<ByteBuffer> variables) throws InvalidRequestException
 +    {
 +        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
 +        for (ByteBuffer component : restriction.values(variables))
 +            builder.add(component);
 +
 +        ByteBuffer result = builder.componentCount() > 0 && eocBound == Bound.END
 +                ? builder.buildAsEndOfRange()
 +                : builder.build();
 +        return Collections.singletonList(result);
 +    }
 +
 +    private static boolean isNullRestriction(Restriction r, Bound b)
 +    {
 +        return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
 +    }
 +
 +    private static ByteBuffer getSliceValue(Restriction r, Bound b, List<ByteBuffer> variables) throws InvalidRequestException
 +    {
 +        Restriction.Slice slice = (Restriction.Slice)r;
 +        assert slice.hasBound(b);
 +        ByteBuffer val = slice.bound(b, variables);
 +        if (val == null)
 +            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
 +        return val;
 +    }
 +
      private List<ByteBuffer> getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
      {
          assert isColumnRange();