You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2013/08/08 22:52:23 UTC

[01/16] git commit: Correctly validate sparse composite cells patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855

Updated Branches:
  refs/heads/cassandra-1.2 ccb32a8c8 -> 9e0efa3db
  refs/heads/cassandra-2.0 5d9a33a46 -> e911b767e
  refs/heads/cassandra-2.0.0 7a8cbb3ea -> 5411e5ff8
  refs/heads/trunk 28b827a85 -> a1212c597


Correctly validate sparse composite cells
patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855


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

Branch: refs/heads/cassandra-2.0
Commit: eb884a582b709d67fdd075aedf19facf6803bf93
Parents: 238139c
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:33:39 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:43:43 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  1 +
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb884a58/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 75b349f..8790518 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 1.1.next
  * Backport compaction exception handling from 1.2
+ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 
 
 1.1.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb884a58/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Column.java b/src/java/org/apache/cassandra/db/Column.java
index 97aece0..ea83146 100644
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.utils.Allocator;
@@ -277,7 +278,23 @@ public class Column implements IColumn
     public void validateFields(CFMetaData metadata) throws MarshalException
     {
         validateName(metadata);
-        AbstractType<?> valueValidator = metadata.getValueValidator(name());
+        CFDefinition cfdef = metadata.getCfDef();
+
+        // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+        // (Note that COMPACT composites are handled by validateName, above.)
+        ByteBuffer internalName;
+        if (cfdef.isComposite && !cfdef.isCompact)
+        {
+            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
+            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+            internalName = components.get(components.size() - 1).value;
+        }
+        else
+        {
+            internalName = name;
+        }
+
+        AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
         if (valueValidator != null)
             valueValidator.validate(value());
     }


[06/16] git commit: merge from 1.1

Posted by jb...@apache.org.
merge from 1.1


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

Branch: refs/heads/cassandra-2.0.0
Commit: 1143df1aa4397777618bff6005ca1cee34cb066a
Parents: 23d7bd8 eb884a5
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:46:52 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:46:52 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  2 ++
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1143df1a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 859e7f9,8790518..30ce976
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,214 -1,19 +1,216 @@@
 -1.1.next
 - * Backport compaction exception handling from 1.2
 +1.2.9
 + * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
 + * fix bulk-loading compressed sstables (CASSANDRA-5820)
 + * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
 +   (CASSANDRA-5824)
 + * update default LCS sstable size to 160MB (CASSANDRA-5727)
 + * Allow compacting 2Is via nodetool (CASSANDRA-5670)
 + * Hex-encode non-String keys in OPP (CASSANDRA-5793)
 + * nodetool history logging (CASSANDRA-5823)
 + * (Hadoop) fix support for Thrift tables in CqlPagingRecordReader 
 +   (CASSANDRA-5752)
 + * add "all time blocked" to StatusLogger output (CASSANDRA-5825)
 + * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
 + * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
 +   (CASSANDRA-5718)
++Merged from 1.1:
+  * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
  
  
 -1.1.12
 +1.2.8
 + * Fix reading DeletionTime from 1.1-format sstables (CASSANDRA-5814)
 + * cqlsh: add collections support to COPY (CASSANDRA-5698)
 + * retry important messages for any IOException (CASSANDRA-5804)
 + * Allow empty IN relations in SELECT/UPDATE/DELETE statements (CASSANDRA-5626)
 + * cqlsh: fix crashing on Windows due to libedit detection (CASSANDRA-5812)
 +
 +
 +1.2.7
 + * if no seeds can be a reached a node won't start in a ring by itself (CASSANDRA-5768)
 + * add cassandra.unsafesystem property (CASSANDRA-5704)
 + * (Hadoop) quote identifiers in CqlPagingRecordReader (CASSANDRA-5763)
 + * Add replace_node functionality for vnodes (CASSANDRA-5337)
 + * Add timeout events to query traces (CASSANDRA-5520)
 + * make starting native protocol server idempotent (CASSANDRA-5728)
 + * Fix loading key cache when a saved entry is no longer valid (CASSANDRA-5706)
 + * Fix serialization of the LEFT gossip value (CASSANDRA-5696)
 + * Pig: support for cql3 tables (CASSANDRA-5234)
 + * cqlsh: Don't show 'null' in place of empty values (CASSANDRA-5675)
 + * Race condition in detecting version on a mixed 1.1/1.2 cluster
 +   (CASSANDRA-5692)
 + * Fix skipping range tombstones with reverse queries (CASSANDRA-5712)
 + * Expire entries out of ThriftSessionManager (CASSANRDA-5719)
 + * Don't keep ancestor information in memory (CASSANDRA-5342)
 + * cqlsh: fix handling of semicolons inside BATCH queries (CASSANDRA-5697)
 + * Expose native protocol server status in nodetool info (CASSANDRA-5735)
 + * Fix pathetic performance of range tombstones (CASSANDRA-5677)
 + * Fix querying with an empty (impossible) range (CASSANDRA-5573)
 + * cqlsh: handle CUSTOM 2i in DESCRIBE output (CASSANDRA-5760)
 + * Fix minor bug in Range.intersects(Bound) (CASSANDRA-5771)
 + * cqlsh: handle disabled compression in DESCRIBE output (CASSANDRA-5766)
 + * Ensure all UP events are notified on the native protocol (CASSANDRA-5769)
 + * Fix formatting of sstable2json with multiple -k arguments (CASSANDRA-5781)
 + * Don't rely on row marker for queries in general to hide lost markers
 +   after TTL expires (CASSANDRA-5762)
 + * Sort nodetool help output (CASSANDRA-5776)
 + * Fix column expiring during 2 phases compaction (CASSANDRA-5799)
 + * now() is being rejected in INSERTs when inside collections (CASSANDRA-5795)
 +
 +
 +1.2.6
 + * Fix tracing when operation completes before all responses arrive 
 +   (CASSANDRA-5668)
 + * Fix cross-DC mutation forwarding (CASSANDRA-5632)
 + * Reduce SSTableLoader memory usage (CASSANDRA-5555)
 + * Scale hinted_handoff_throttle_in_kb to cluster size (CASSANDRA-5272)
 + * (Hadoop) Add CQL3 input/output formats (CASSANDRA-4421, 5622)
 + * (Hadoop) Fix InputKeyRange in CFIF (CASSANDRA-5536)
 + * Fix dealing with ridiculously large max sstable sizes in LCS (CASSANDRA-5589)
 + * Ignore pre-truncate hints (CASSANDRA-4655)
 + * Move System.exit on OOM into a separate thread (CASSANDRA-5273)
 + * Write row markers when serializing schema (CASSANDRA-5572)
 + * Check only SSTables for the requested range when streaming (CASSANDRA-5569)
 + * Improve batchlog replay behavior and hint ttl handling (CASSANDRA-5314)
 + * Exclude localTimestamp from validation for tombstones (CASSANDRA-5398)
 + * cqlsh: add custom prompt support (CASSANDRA-5539)
 + * Reuse prepared statements in hot auth queries (CASSANDRA-5594)
 + * cqlsh: add vertical output option (see EXPAND) (CASSANDRA-5597)
 + * Add a rate limit option to stress (CASSANDRA-5004)
 + * have BulkLoader ignore snapshots directories (CASSANDRA-5587) 
 + * fix SnitchProperties logging context (CASSANDRA-5602)
 + * Expose whether jna is enabled and memory is locked via JMX (CASSANDRA-5508)
 + * cqlsh: fix COPY FROM with ReversedType (CASSANDRA-5610)
 + * Allow creating CUSTOM indexes on collections (CASSANDRA-5615)
 + * Evaluate now() function at execution time (CASSANDRA-5616)
 + * Expose detailed read repair metrics (CASSANDRA-5618)
 + * Correct blob literal + ReversedType parsing (CASSANDRA-5629)
 + * Allow GPFS to prefer the internal IP like EC2MRS (CASSANDRA-5630)
 + * fix help text for -tspw cassandra-cli (CASSANDRA-5643)
 + * don't throw away initial causes exceptions for internode encryption issues 
 +   (CASSANDRA-5644)
 + * Fix message spelling errors for cql select statements (CASSANDRA-5647)
 + * Suppress custom exceptions thru jmx (CASSANDRA-5652)
 + * Update CREATE CUSTOM INDEX syntax (CASSANDRA-5639)
 + * Fix PermissionDetails.equals() method (CASSANDRA-5655)
 + * Never allow partition key ranges in CQL3 without token() (CASSANDRA-5666)
 + * Gossiper incorrectly drops AppState for an upgrading node (CASSANDRA-5660)
 + * Connection thrashing during multi-region ec2 during upgrade, due to 
 +   messaging version (CASSANDRA-5669)
 + * Avoid over reconnecting in EC2MRS (CASSANDRA-5678)
 + * Fix ReadResponseSerializer.serializedSize() for digest reads (CASSANDRA-5476)
 + * allow sstable2json on 2i CFs (CASSANDRA-5694)
 +Merged from 1.1:
 + * Remove buggy thrift max message length option (CASSANDRA-5529)
 + * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 + * Add split size parameter to Pig and disable split combination (CASSANDRA-5544)
 +
 +
 +1.2.5
 + * make BytesToken.toString only return hex bytes (CASSANDRA-5566)
 + * Ensure that submitBackground enqueues at least one task (CASSANDRA-5554)
 + * fix 2i updates with identical values and timestamps (CASSANDRA-5540)
 + * fix compaction throttling bursty-ness (CASSANDRA-4316)
 + * reduce memory consumption of IndexSummary (CASSANDRA-5506)
 + * remove per-row column name bloom filters (CASSANDRA-5492)
 + * Include fatal errors in trace events (CASSANDRA-5447)
 + * Ensure that PerRowSecondaryIndex is notified of row-level deletes
 +   (CASSANDRA-5445)
 + * Allow empty blob literals in CQL3 (CASSANDRA-5452)
 + * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 + * Fix preparing statements when current keyspace is not set (CASSANDRA-5468)
 + * Fix SemanticVersion.isSupportedBy minor/patch handling (CASSANDRA-5496)
 + * Don't provide oldCfId for post-1.1 system cfs (CASSANDRA-5490)
 + * Fix primary range ignores replication strategy (CASSANDRA-5424)
 + * Fix shutdown of binary protocol server (CASSANDRA-5507)
 + * Fix repair -snapshot not working (CASSANDRA-5512)
 + * Set isRunning flag later in binary protocol server (CASSANDRA-5467)
 + * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472)
 + * Disallow renaming columns one at a time for thrift table in CQL3
 +   (CASSANDRA-5531)
 + * cqlsh: add CLUSTERING ORDER BY support to DESCRIBE (CASSANDRA-5528)
 + * Add custom secondary index support to CQL3 (CASSANDRA-5484)
 + * Fix repair hanging silently on unexpected error (CASSANDRA-5229)
 + * Fix Ec2Snitch regression introduced by CASSANDRA-5171 (CASSANDRA-5432)
 + * Add nodetool enablebackup/disablebackup (CASSANDRA-5556)
 + * cqlsh: fix DESCRIBE after case insensitive USE (CASSANDRA-5567)
 +Merged from 1.1
   * Remove buggy thrift max message length option (CASSANDRA-5529)
   * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393)
 - * Use allocator information to improve memtable memory usage estimate 
 + * Use allocator information to improve memtable memory usage estimate
     (CASSANDRA-5497)
 + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
   * fsync leveled manifest to avoid corruption (CASSANDRA-5535)
   * Fix Bound intersection computation (CASSANDRA-5551)
 - * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 + * sstablescrub now respects max memory size in cassandra.in.sh (CASSANDRA-5562)
 +
 +
 +1.2.4
 + * Ensure that PerRowSecondaryIndex updates see the most recent values
 +   (CASSANDRA-5397)
 + * avoid duplicate index entries ind PrecompactedRow and 
 +   ParallelCompactionIterable (CASSANDRA-5395)
 + * remove the index entry on oldColumn when new column is a tombstone 
 +   (CASSANDRA-5395)
 + * Change default stream throughput from 400 to 200 mbps (CASSANDRA-5036)
 + * Gossiper logs DOWN for symmetry with UP (CASSANDRA-5187)
 + * Fix mixing prepared statements between keyspaces (CASSANDRA-5352)
 + * Fix consistency level during bootstrap - strike 3 (CASSANDRA-5354)
 + * Fix transposed arguments in AlreadyExistsException (CASSANDRA-5362)
 + * Improve asynchronous hint delivery (CASSANDRA-5179)
 + * Fix Guava dependency version (12.0 -> 13.0.1) for Maven (CASSANDRA-5364)
 + * Validate that provided CQL3 collection value are < 64K (CASSANDRA-5355)
 + * Make upgradeSSTable skip current version sstables by default (CASSANDRA-5366)
 + * Optimize min/max timestamp collection (CASSANDRA-5373)
 + * Invalid streamId in cql binary protocol when using invalid CL 
 +   (CASSANDRA-5164)
 + * Fix validation for IN where clauses with collections (CASSANDRA-5376)
 + * Copy resultSet on count query to avoid ConcurrentModificationException 
 +   (CASSANDRA-5382)
 + * Correctly typecheck in CQL3 even with ReversedType (CASSANDRA-5386)
 + * Fix streaming compressed files when using encryption (CASSANDRA-5391)
 + * cassandra-all 1.2.0 pom missing netty dependency (CASSANDRA-5392)
 + * Fix writetime/ttl functions on null values (CASSANDRA-5341)
 + * Fix NPE during cql3 select with token() (CASSANDRA-5404)
 + * IndexHelper.skipBloomFilters won't skip non-SHA filters (CASSANDRA-5385)
 + * cqlsh: Print maps ordered by key, sort sets (CASSANDRA-5413)
 + * Add null syntax support in CQL3 for inserts (CASSANDRA-3783)
 + * Allow unauthenticated set_keyspace() calls (CASSANDRA-5423)
 + * Fix potential incremental backups race (CASSANDRA-5410)
 + * Fix prepared BATCH statements with batch-level timestamps (CASSANDRA-5415)
 + * Allow overriding superuser setup delay (CASSANDRA-5430)
 + * cassandra-shuffle with JMX usernames and passwords (CASSANDRA-5431)
 +Merged from 1.1:
 + * cli: Quote ks and cf names in schema output when needed (CASSANDRA-5052)
 + * Fix bad default for min/max timestamp in SSTableMetadata (CASSANDRA-5372)
 + * Fix cf name extraction from manifest in Directories.migrateFile() 
 +   (CASSANDRA-5242)
 + * Support pluggable internode authentication (CASSANDRA-5401)
  
  
 -1.1.11
 +1.2.3
 + * add check for sstable overlap within a level on startup (CASSANDRA-5327)
 + * replace ipv6 colons in jmx object names (CASSANDRA-5298, 5328)
 + * Avoid allocating SSTableBoundedScanner during repair when the range does 
 +   not intersect the sstable (CASSANDRA-5249)
 + * Don't lowercase property map keys (this breaks NTS) (CASSANDRA-5292)
 + * Fix composite comparator with super columns (CASSANDRA-5287)
 + * Fix insufficient validation of UPDATE queries against counter cfs
 +   (CASSANDRA-5300)
 + * Fix PropertyFileSnitch default DC/Rack behavior (CASSANDRA-5285)
 + * Handle null values when executing prepared statement (CASSANDRA-5081)
 + * Add netty to pom dependencies (CASSANDRA-5181)
 + * Include type arguments in Thrift CQLPreparedResult (CASSANDRA-5311)
 + * Fix compaction not removing columns when bf_fp_ratio is 1 (CASSANDRA-5182)
 + * cli: Warn about missing CQL3 tables in schema descriptions (CASSANDRA-5309)
 + * Re-enable unknown option in replication/compaction strategies option for
 +   backward compatibility (CASSANDRA-4795)
 + * Add binary protocol support to stress (CASSANDRA-4993)
 + * cqlsh: Fix COPY FROM value quoting and null handling (CASSANDRA-5305)
 + * Fix repair -pr for vnodes (CASSANDRA-5329)
 + * Relax CL for auth queries for non-default users (CASSANDRA-5310)
 + * Fix AssertionError during repair (CASSANDRA-5245)
 + * Don't announce migrations to pre-1.2 nodes (CASSANDRA-5334)
 +Merged from 1.1:
   * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
   * Update offline scrub for 1.0 -> 1.1 directory structure (CASSANDRA-5195)
   * add tmp flag to Descriptor hashcode (CASSANDRA-4021)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1143df1a/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Column.java
index 616f3c0,ea83146..412808b
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@@ -25,7 -25,11 +25,8 @@@ import java.util.ArrayList
  import java.util.Collection;
  import java.util.List;
  
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 -
  import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.CFDefinition;
  import org.apache.cassandra.db.marshal.*;
  import org.apache.cassandra.io.util.DataOutputBuffer;
  import org.apache.cassandra.utils.Allocator;
@@@ -287,7 -278,23 +288,23 @@@ public class Column implements IColum
      public void validateFields(CFMetaData metadata) throws MarshalException
      {
          validateName(metadata);
-         AbstractType<?> valueValidator = metadata.getValueValidator(name());
+         CFDefinition cfdef = metadata.getCfDef();
+ 
+         // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+         // (Note that COMPACT composites are handled by validateName, above.)
+         ByteBuffer internalName;
+         if (cfdef.isComposite && !cfdef.isCompact)
+         {
 -            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
++            CompositeType comparator = (CompositeType) metadata.comparator;
+             List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+             internalName = components.get(components.size() - 1).value;
+         }
+         else
+         {
+             internalName = name;
+         }
+ 
+         AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
          if (valueValidator != null)
              valueValidator.validate(value());
      }


[04/16] git commit: merge from 1.1

Posted by jb...@apache.org.
merge from 1.1


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

Branch: refs/heads/cassandra-2.0
Commit: 1143df1aa4397777618bff6005ca1cee34cb066a
Parents: 23d7bd8 eb884a5
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:46:52 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:46:52 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  2 ++
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1143df1a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 859e7f9,8790518..30ce976
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,214 -1,19 +1,216 @@@
 -1.1.next
 - * Backport compaction exception handling from 1.2
 +1.2.9
 + * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
 + * fix bulk-loading compressed sstables (CASSANDRA-5820)
 + * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
 +   (CASSANDRA-5824)
 + * update default LCS sstable size to 160MB (CASSANDRA-5727)
 + * Allow compacting 2Is via nodetool (CASSANDRA-5670)
 + * Hex-encode non-String keys in OPP (CASSANDRA-5793)
 + * nodetool history logging (CASSANDRA-5823)
 + * (Hadoop) fix support for Thrift tables in CqlPagingRecordReader 
 +   (CASSANDRA-5752)
 + * add "all time blocked" to StatusLogger output (CASSANDRA-5825)
 + * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
 + * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
 +   (CASSANDRA-5718)
++Merged from 1.1:
+  * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
  
  
 -1.1.12
 +1.2.8
 + * Fix reading DeletionTime from 1.1-format sstables (CASSANDRA-5814)
 + * cqlsh: add collections support to COPY (CASSANDRA-5698)
 + * retry important messages for any IOException (CASSANDRA-5804)
 + * Allow empty IN relations in SELECT/UPDATE/DELETE statements (CASSANDRA-5626)
 + * cqlsh: fix crashing on Windows due to libedit detection (CASSANDRA-5812)
 +
 +
 +1.2.7
 + * if no seeds can be a reached a node won't start in a ring by itself (CASSANDRA-5768)
 + * add cassandra.unsafesystem property (CASSANDRA-5704)
 + * (Hadoop) quote identifiers in CqlPagingRecordReader (CASSANDRA-5763)
 + * Add replace_node functionality for vnodes (CASSANDRA-5337)
 + * Add timeout events to query traces (CASSANDRA-5520)
 + * make starting native protocol server idempotent (CASSANDRA-5728)
 + * Fix loading key cache when a saved entry is no longer valid (CASSANDRA-5706)
 + * Fix serialization of the LEFT gossip value (CASSANDRA-5696)
 + * Pig: support for cql3 tables (CASSANDRA-5234)
 + * cqlsh: Don't show 'null' in place of empty values (CASSANDRA-5675)
 + * Race condition in detecting version on a mixed 1.1/1.2 cluster
 +   (CASSANDRA-5692)
 + * Fix skipping range tombstones with reverse queries (CASSANDRA-5712)
 + * Expire entries out of ThriftSessionManager (CASSANRDA-5719)
 + * Don't keep ancestor information in memory (CASSANDRA-5342)
 + * cqlsh: fix handling of semicolons inside BATCH queries (CASSANDRA-5697)
 + * Expose native protocol server status in nodetool info (CASSANDRA-5735)
 + * Fix pathetic performance of range tombstones (CASSANDRA-5677)
 + * Fix querying with an empty (impossible) range (CASSANDRA-5573)
 + * cqlsh: handle CUSTOM 2i in DESCRIBE output (CASSANDRA-5760)
 + * Fix minor bug in Range.intersects(Bound) (CASSANDRA-5771)
 + * cqlsh: handle disabled compression in DESCRIBE output (CASSANDRA-5766)
 + * Ensure all UP events are notified on the native protocol (CASSANDRA-5769)
 + * Fix formatting of sstable2json with multiple -k arguments (CASSANDRA-5781)
 + * Don't rely on row marker for queries in general to hide lost markers
 +   after TTL expires (CASSANDRA-5762)
 + * Sort nodetool help output (CASSANDRA-5776)
 + * Fix column expiring during 2 phases compaction (CASSANDRA-5799)
 + * now() is being rejected in INSERTs when inside collections (CASSANDRA-5795)
 +
 +
 +1.2.6
 + * Fix tracing when operation completes before all responses arrive 
 +   (CASSANDRA-5668)
 + * Fix cross-DC mutation forwarding (CASSANDRA-5632)
 + * Reduce SSTableLoader memory usage (CASSANDRA-5555)
 + * Scale hinted_handoff_throttle_in_kb to cluster size (CASSANDRA-5272)
 + * (Hadoop) Add CQL3 input/output formats (CASSANDRA-4421, 5622)
 + * (Hadoop) Fix InputKeyRange in CFIF (CASSANDRA-5536)
 + * Fix dealing with ridiculously large max sstable sizes in LCS (CASSANDRA-5589)
 + * Ignore pre-truncate hints (CASSANDRA-4655)
 + * Move System.exit on OOM into a separate thread (CASSANDRA-5273)
 + * Write row markers when serializing schema (CASSANDRA-5572)
 + * Check only SSTables for the requested range when streaming (CASSANDRA-5569)
 + * Improve batchlog replay behavior and hint ttl handling (CASSANDRA-5314)
 + * Exclude localTimestamp from validation for tombstones (CASSANDRA-5398)
 + * cqlsh: add custom prompt support (CASSANDRA-5539)
 + * Reuse prepared statements in hot auth queries (CASSANDRA-5594)
 + * cqlsh: add vertical output option (see EXPAND) (CASSANDRA-5597)
 + * Add a rate limit option to stress (CASSANDRA-5004)
 + * have BulkLoader ignore snapshots directories (CASSANDRA-5587) 
 + * fix SnitchProperties logging context (CASSANDRA-5602)
 + * Expose whether jna is enabled and memory is locked via JMX (CASSANDRA-5508)
 + * cqlsh: fix COPY FROM with ReversedType (CASSANDRA-5610)
 + * Allow creating CUSTOM indexes on collections (CASSANDRA-5615)
 + * Evaluate now() function at execution time (CASSANDRA-5616)
 + * Expose detailed read repair metrics (CASSANDRA-5618)
 + * Correct blob literal + ReversedType parsing (CASSANDRA-5629)
 + * Allow GPFS to prefer the internal IP like EC2MRS (CASSANDRA-5630)
 + * fix help text for -tspw cassandra-cli (CASSANDRA-5643)
 + * don't throw away initial causes exceptions for internode encryption issues 
 +   (CASSANDRA-5644)
 + * Fix message spelling errors for cql select statements (CASSANDRA-5647)
 + * Suppress custom exceptions thru jmx (CASSANDRA-5652)
 + * Update CREATE CUSTOM INDEX syntax (CASSANDRA-5639)
 + * Fix PermissionDetails.equals() method (CASSANDRA-5655)
 + * Never allow partition key ranges in CQL3 without token() (CASSANDRA-5666)
 + * Gossiper incorrectly drops AppState for an upgrading node (CASSANDRA-5660)
 + * Connection thrashing during multi-region ec2 during upgrade, due to 
 +   messaging version (CASSANDRA-5669)
 + * Avoid over reconnecting in EC2MRS (CASSANDRA-5678)
 + * Fix ReadResponseSerializer.serializedSize() for digest reads (CASSANDRA-5476)
 + * allow sstable2json on 2i CFs (CASSANDRA-5694)
 +Merged from 1.1:
 + * Remove buggy thrift max message length option (CASSANDRA-5529)
 + * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 + * Add split size parameter to Pig and disable split combination (CASSANDRA-5544)
 +
 +
 +1.2.5
 + * make BytesToken.toString only return hex bytes (CASSANDRA-5566)
 + * Ensure that submitBackground enqueues at least one task (CASSANDRA-5554)
 + * fix 2i updates with identical values and timestamps (CASSANDRA-5540)
 + * fix compaction throttling bursty-ness (CASSANDRA-4316)
 + * reduce memory consumption of IndexSummary (CASSANDRA-5506)
 + * remove per-row column name bloom filters (CASSANDRA-5492)
 + * Include fatal errors in trace events (CASSANDRA-5447)
 + * Ensure that PerRowSecondaryIndex is notified of row-level deletes
 +   (CASSANDRA-5445)
 + * Allow empty blob literals in CQL3 (CASSANDRA-5452)
 + * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 + * Fix preparing statements when current keyspace is not set (CASSANDRA-5468)
 + * Fix SemanticVersion.isSupportedBy minor/patch handling (CASSANDRA-5496)
 + * Don't provide oldCfId for post-1.1 system cfs (CASSANDRA-5490)
 + * Fix primary range ignores replication strategy (CASSANDRA-5424)
 + * Fix shutdown of binary protocol server (CASSANDRA-5507)
 + * Fix repair -snapshot not working (CASSANDRA-5512)
 + * Set isRunning flag later in binary protocol server (CASSANDRA-5467)
 + * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472)
 + * Disallow renaming columns one at a time for thrift table in CQL3
 +   (CASSANDRA-5531)
 + * cqlsh: add CLUSTERING ORDER BY support to DESCRIBE (CASSANDRA-5528)
 + * Add custom secondary index support to CQL3 (CASSANDRA-5484)
 + * Fix repair hanging silently on unexpected error (CASSANDRA-5229)
 + * Fix Ec2Snitch regression introduced by CASSANDRA-5171 (CASSANDRA-5432)
 + * Add nodetool enablebackup/disablebackup (CASSANDRA-5556)
 + * cqlsh: fix DESCRIBE after case insensitive USE (CASSANDRA-5567)
 +Merged from 1.1
   * Remove buggy thrift max message length option (CASSANDRA-5529)
   * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393)
 - * Use allocator information to improve memtable memory usage estimate 
 + * Use allocator information to improve memtable memory usage estimate
     (CASSANDRA-5497)
 + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
   * fsync leveled manifest to avoid corruption (CASSANDRA-5535)
   * Fix Bound intersection computation (CASSANDRA-5551)
 - * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 + * sstablescrub now respects max memory size in cassandra.in.sh (CASSANDRA-5562)
 +
 +
 +1.2.4
 + * Ensure that PerRowSecondaryIndex updates see the most recent values
 +   (CASSANDRA-5397)
 + * avoid duplicate index entries ind PrecompactedRow and 
 +   ParallelCompactionIterable (CASSANDRA-5395)
 + * remove the index entry on oldColumn when new column is a tombstone 
 +   (CASSANDRA-5395)
 + * Change default stream throughput from 400 to 200 mbps (CASSANDRA-5036)
 + * Gossiper logs DOWN for symmetry with UP (CASSANDRA-5187)
 + * Fix mixing prepared statements between keyspaces (CASSANDRA-5352)
 + * Fix consistency level during bootstrap - strike 3 (CASSANDRA-5354)
 + * Fix transposed arguments in AlreadyExistsException (CASSANDRA-5362)
 + * Improve asynchronous hint delivery (CASSANDRA-5179)
 + * Fix Guava dependency version (12.0 -> 13.0.1) for Maven (CASSANDRA-5364)
 + * Validate that provided CQL3 collection value are < 64K (CASSANDRA-5355)
 + * Make upgradeSSTable skip current version sstables by default (CASSANDRA-5366)
 + * Optimize min/max timestamp collection (CASSANDRA-5373)
 + * Invalid streamId in cql binary protocol when using invalid CL 
 +   (CASSANDRA-5164)
 + * Fix validation for IN where clauses with collections (CASSANDRA-5376)
 + * Copy resultSet on count query to avoid ConcurrentModificationException 
 +   (CASSANDRA-5382)
 + * Correctly typecheck in CQL3 even with ReversedType (CASSANDRA-5386)
 + * Fix streaming compressed files when using encryption (CASSANDRA-5391)
 + * cassandra-all 1.2.0 pom missing netty dependency (CASSANDRA-5392)
 + * Fix writetime/ttl functions on null values (CASSANDRA-5341)
 + * Fix NPE during cql3 select with token() (CASSANDRA-5404)
 + * IndexHelper.skipBloomFilters won't skip non-SHA filters (CASSANDRA-5385)
 + * cqlsh: Print maps ordered by key, sort sets (CASSANDRA-5413)
 + * Add null syntax support in CQL3 for inserts (CASSANDRA-3783)
 + * Allow unauthenticated set_keyspace() calls (CASSANDRA-5423)
 + * Fix potential incremental backups race (CASSANDRA-5410)
 + * Fix prepared BATCH statements with batch-level timestamps (CASSANDRA-5415)
 + * Allow overriding superuser setup delay (CASSANDRA-5430)
 + * cassandra-shuffle with JMX usernames and passwords (CASSANDRA-5431)
 +Merged from 1.1:
 + * cli: Quote ks and cf names in schema output when needed (CASSANDRA-5052)
 + * Fix bad default for min/max timestamp in SSTableMetadata (CASSANDRA-5372)
 + * Fix cf name extraction from manifest in Directories.migrateFile() 
 +   (CASSANDRA-5242)
 + * Support pluggable internode authentication (CASSANDRA-5401)
  
  
 -1.1.11
 +1.2.3
 + * add check for sstable overlap within a level on startup (CASSANDRA-5327)
 + * replace ipv6 colons in jmx object names (CASSANDRA-5298, 5328)
 + * Avoid allocating SSTableBoundedScanner during repair when the range does 
 +   not intersect the sstable (CASSANDRA-5249)
 + * Don't lowercase property map keys (this breaks NTS) (CASSANDRA-5292)
 + * Fix composite comparator with super columns (CASSANDRA-5287)
 + * Fix insufficient validation of UPDATE queries against counter cfs
 +   (CASSANDRA-5300)
 + * Fix PropertyFileSnitch default DC/Rack behavior (CASSANDRA-5285)
 + * Handle null values when executing prepared statement (CASSANDRA-5081)
 + * Add netty to pom dependencies (CASSANDRA-5181)
 + * Include type arguments in Thrift CQLPreparedResult (CASSANDRA-5311)
 + * Fix compaction not removing columns when bf_fp_ratio is 1 (CASSANDRA-5182)
 + * cli: Warn about missing CQL3 tables in schema descriptions (CASSANDRA-5309)
 + * Re-enable unknown option in replication/compaction strategies option for
 +   backward compatibility (CASSANDRA-4795)
 + * Add binary protocol support to stress (CASSANDRA-4993)
 + * cqlsh: Fix COPY FROM value quoting and null handling (CASSANDRA-5305)
 + * Fix repair -pr for vnodes (CASSANDRA-5329)
 + * Relax CL for auth queries for non-default users (CASSANDRA-5310)
 + * Fix AssertionError during repair (CASSANDRA-5245)
 + * Don't announce migrations to pre-1.2 nodes (CASSANDRA-5334)
 +Merged from 1.1:
   * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
   * Update offline scrub for 1.0 -> 1.1 directory structure (CASSANDRA-5195)
   * add tmp flag to Descriptor hashcode (CASSANDRA-4021)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1143df1a/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Column.java
index 616f3c0,ea83146..412808b
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@@ -25,7 -25,11 +25,8 @@@ import java.util.ArrayList
  import java.util.Collection;
  import java.util.List;
  
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 -
  import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.CFDefinition;
  import org.apache.cassandra.db.marshal.*;
  import org.apache.cassandra.io.util.DataOutputBuffer;
  import org.apache.cassandra.utils.Allocator;
@@@ -287,7 -278,23 +288,23 @@@ public class Column implements IColum
      public void validateFields(CFMetaData metadata) throws MarshalException
      {
          validateName(metadata);
-         AbstractType<?> valueValidator = metadata.getValueValidator(name());
+         CFDefinition cfdef = metadata.getCfDef();
+ 
+         // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+         // (Note that COMPACT composites are handled by validateName, above.)
+         ByteBuffer internalName;
+         if (cfdef.isComposite && !cfdef.isCompact)
+         {
 -            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
++            CompositeType comparator = (CompositeType) metadata.comparator;
+             List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+             internalName = components.get(components.size() - 1).value;
+         }
+         else
+         {
+             internalName = name;
+         }
+ 
+         AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
          if (valueValidator != null)
              valueValidator.validate(value());
      }


[13/16] git commit: Correctly validate sparse composite cells patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855

Posted by jb...@apache.org.
Correctly validate sparse composite cells
patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855


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

Branch: refs/heads/cassandra-1.2
Commit: 9e0efa3dbbeddf382c8e0f45cfdf2ebb91bc9db0
Parents: ccb32a8
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:33:39 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:51:34 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  2 ++
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e0efa3d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 859e7f9..30ce976 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -13,6 +13,8 @@
  * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
  * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
    (CASSANDRA-5718)
+Merged from 1.1:
+ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 
 
 1.2.8

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e0efa3d/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Column.java b/src/java/org/apache/cassandra/db/Column.java
index 616f3c0..7b614f5 100644
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@ -26,6 +26,7 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.utils.Allocator;
@@ -287,7 +288,23 @@ public class Column implements IColumn
     public void validateFields(CFMetaData metadata) throws MarshalException
     {
         validateName(metadata);
-        AbstractType<?> valueValidator = metadata.getValueValidator(name());
+        CFDefinition cfdef = metadata.getCfDef();
+
+        // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+        // (Note that COMPACT composites are handled by validateName, above.)
+        ByteBuffer internalName;
+        if (cfdef.isComposite && !cfdef.isCompact)
+        {
+            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
+            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+            internalName = components.get(components.size() - 1).value;
+        }
+        else
+        {
+            internalName = name;
+        }
+
+        AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
         if (valueValidator != null)
             valueValidator.validate(value());
     }


[11/16] git commit: Merge branch 'cassandra-2.0.0' into cassandra-2.0

Posted by jb...@apache.org.
Merge branch 'cassandra-2.0.0' 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/b5f445b4
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5f445b4
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5f445b4

Branch: refs/heads/cassandra-2.0
Commit: b5f445b448e6047e717b17985c117843e66a2d40
Parents: c526131 fdfdd5c
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:50:16 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:50:16 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  4 ++++
 src/java/org/apache/cassandra/db/Column.java | 18 +++++++++++++++++-
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


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


[05/16] git commit: merge from 1.1

Posted by jb...@apache.org.
merge from 1.1


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

Branch: refs/heads/trunk
Commit: 1143df1aa4397777618bff6005ca1cee34cb066a
Parents: 23d7bd8 eb884a5
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:46:52 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:46:52 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  2 ++
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1143df1a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 859e7f9,8790518..30ce976
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,214 -1,19 +1,216 @@@
 -1.1.next
 - * Backport compaction exception handling from 1.2
 +1.2.9
 + * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
 + * fix bulk-loading compressed sstables (CASSANDRA-5820)
 + * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
 +   (CASSANDRA-5824)
 + * update default LCS sstable size to 160MB (CASSANDRA-5727)
 + * Allow compacting 2Is via nodetool (CASSANDRA-5670)
 + * Hex-encode non-String keys in OPP (CASSANDRA-5793)
 + * nodetool history logging (CASSANDRA-5823)
 + * (Hadoop) fix support for Thrift tables in CqlPagingRecordReader 
 +   (CASSANDRA-5752)
 + * add "all time blocked" to StatusLogger output (CASSANDRA-5825)
 + * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
 + * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
 +   (CASSANDRA-5718)
++Merged from 1.1:
+  * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
  
  
 -1.1.12
 +1.2.8
 + * Fix reading DeletionTime from 1.1-format sstables (CASSANDRA-5814)
 + * cqlsh: add collections support to COPY (CASSANDRA-5698)
 + * retry important messages for any IOException (CASSANDRA-5804)
 + * Allow empty IN relations in SELECT/UPDATE/DELETE statements (CASSANDRA-5626)
 + * cqlsh: fix crashing on Windows due to libedit detection (CASSANDRA-5812)
 +
 +
 +1.2.7
 + * if no seeds can be a reached a node won't start in a ring by itself (CASSANDRA-5768)
 + * add cassandra.unsafesystem property (CASSANDRA-5704)
 + * (Hadoop) quote identifiers in CqlPagingRecordReader (CASSANDRA-5763)
 + * Add replace_node functionality for vnodes (CASSANDRA-5337)
 + * Add timeout events to query traces (CASSANDRA-5520)
 + * make starting native protocol server idempotent (CASSANDRA-5728)
 + * Fix loading key cache when a saved entry is no longer valid (CASSANDRA-5706)
 + * Fix serialization of the LEFT gossip value (CASSANDRA-5696)
 + * Pig: support for cql3 tables (CASSANDRA-5234)
 + * cqlsh: Don't show 'null' in place of empty values (CASSANDRA-5675)
 + * Race condition in detecting version on a mixed 1.1/1.2 cluster
 +   (CASSANDRA-5692)
 + * Fix skipping range tombstones with reverse queries (CASSANDRA-5712)
 + * Expire entries out of ThriftSessionManager (CASSANRDA-5719)
 + * Don't keep ancestor information in memory (CASSANDRA-5342)
 + * cqlsh: fix handling of semicolons inside BATCH queries (CASSANDRA-5697)
 + * Expose native protocol server status in nodetool info (CASSANDRA-5735)
 + * Fix pathetic performance of range tombstones (CASSANDRA-5677)
 + * Fix querying with an empty (impossible) range (CASSANDRA-5573)
 + * cqlsh: handle CUSTOM 2i in DESCRIBE output (CASSANDRA-5760)
 + * Fix minor bug in Range.intersects(Bound) (CASSANDRA-5771)
 + * cqlsh: handle disabled compression in DESCRIBE output (CASSANDRA-5766)
 + * Ensure all UP events are notified on the native protocol (CASSANDRA-5769)
 + * Fix formatting of sstable2json with multiple -k arguments (CASSANDRA-5781)
 + * Don't rely on row marker for queries in general to hide lost markers
 +   after TTL expires (CASSANDRA-5762)
 + * Sort nodetool help output (CASSANDRA-5776)
 + * Fix column expiring during 2 phases compaction (CASSANDRA-5799)
 + * now() is being rejected in INSERTs when inside collections (CASSANDRA-5795)
 +
 +
 +1.2.6
 + * Fix tracing when operation completes before all responses arrive 
 +   (CASSANDRA-5668)
 + * Fix cross-DC mutation forwarding (CASSANDRA-5632)
 + * Reduce SSTableLoader memory usage (CASSANDRA-5555)
 + * Scale hinted_handoff_throttle_in_kb to cluster size (CASSANDRA-5272)
 + * (Hadoop) Add CQL3 input/output formats (CASSANDRA-4421, 5622)
 + * (Hadoop) Fix InputKeyRange in CFIF (CASSANDRA-5536)
 + * Fix dealing with ridiculously large max sstable sizes in LCS (CASSANDRA-5589)
 + * Ignore pre-truncate hints (CASSANDRA-4655)
 + * Move System.exit on OOM into a separate thread (CASSANDRA-5273)
 + * Write row markers when serializing schema (CASSANDRA-5572)
 + * Check only SSTables for the requested range when streaming (CASSANDRA-5569)
 + * Improve batchlog replay behavior and hint ttl handling (CASSANDRA-5314)
 + * Exclude localTimestamp from validation for tombstones (CASSANDRA-5398)
 + * cqlsh: add custom prompt support (CASSANDRA-5539)
 + * Reuse prepared statements in hot auth queries (CASSANDRA-5594)
 + * cqlsh: add vertical output option (see EXPAND) (CASSANDRA-5597)
 + * Add a rate limit option to stress (CASSANDRA-5004)
 + * have BulkLoader ignore snapshots directories (CASSANDRA-5587) 
 + * fix SnitchProperties logging context (CASSANDRA-5602)
 + * Expose whether jna is enabled and memory is locked via JMX (CASSANDRA-5508)
 + * cqlsh: fix COPY FROM with ReversedType (CASSANDRA-5610)
 + * Allow creating CUSTOM indexes on collections (CASSANDRA-5615)
 + * Evaluate now() function at execution time (CASSANDRA-5616)
 + * Expose detailed read repair metrics (CASSANDRA-5618)
 + * Correct blob literal + ReversedType parsing (CASSANDRA-5629)
 + * Allow GPFS to prefer the internal IP like EC2MRS (CASSANDRA-5630)
 + * fix help text for -tspw cassandra-cli (CASSANDRA-5643)
 + * don't throw away initial causes exceptions for internode encryption issues 
 +   (CASSANDRA-5644)
 + * Fix message spelling errors for cql select statements (CASSANDRA-5647)
 + * Suppress custom exceptions thru jmx (CASSANDRA-5652)
 + * Update CREATE CUSTOM INDEX syntax (CASSANDRA-5639)
 + * Fix PermissionDetails.equals() method (CASSANDRA-5655)
 + * Never allow partition key ranges in CQL3 without token() (CASSANDRA-5666)
 + * Gossiper incorrectly drops AppState for an upgrading node (CASSANDRA-5660)
 + * Connection thrashing during multi-region ec2 during upgrade, due to 
 +   messaging version (CASSANDRA-5669)
 + * Avoid over reconnecting in EC2MRS (CASSANDRA-5678)
 + * Fix ReadResponseSerializer.serializedSize() for digest reads (CASSANDRA-5476)
 + * allow sstable2json on 2i CFs (CASSANDRA-5694)
 +Merged from 1.1:
 + * Remove buggy thrift max message length option (CASSANDRA-5529)
 + * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 + * Add split size parameter to Pig and disable split combination (CASSANDRA-5544)
 +
 +
 +1.2.5
 + * make BytesToken.toString only return hex bytes (CASSANDRA-5566)
 + * Ensure that submitBackground enqueues at least one task (CASSANDRA-5554)
 + * fix 2i updates with identical values and timestamps (CASSANDRA-5540)
 + * fix compaction throttling bursty-ness (CASSANDRA-4316)
 + * reduce memory consumption of IndexSummary (CASSANDRA-5506)
 + * remove per-row column name bloom filters (CASSANDRA-5492)
 + * Include fatal errors in trace events (CASSANDRA-5447)
 + * Ensure that PerRowSecondaryIndex is notified of row-level deletes
 +   (CASSANDRA-5445)
 + * Allow empty blob literals in CQL3 (CASSANDRA-5452)
 + * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 + * Fix preparing statements when current keyspace is not set (CASSANDRA-5468)
 + * Fix SemanticVersion.isSupportedBy minor/patch handling (CASSANDRA-5496)
 + * Don't provide oldCfId for post-1.1 system cfs (CASSANDRA-5490)
 + * Fix primary range ignores replication strategy (CASSANDRA-5424)
 + * Fix shutdown of binary protocol server (CASSANDRA-5507)
 + * Fix repair -snapshot not working (CASSANDRA-5512)
 + * Set isRunning flag later in binary protocol server (CASSANDRA-5467)
 + * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472)
 + * Disallow renaming columns one at a time for thrift table in CQL3
 +   (CASSANDRA-5531)
 + * cqlsh: add CLUSTERING ORDER BY support to DESCRIBE (CASSANDRA-5528)
 + * Add custom secondary index support to CQL3 (CASSANDRA-5484)
 + * Fix repair hanging silently on unexpected error (CASSANDRA-5229)
 + * Fix Ec2Snitch regression introduced by CASSANDRA-5171 (CASSANDRA-5432)
 + * Add nodetool enablebackup/disablebackup (CASSANDRA-5556)
 + * cqlsh: fix DESCRIBE after case insensitive USE (CASSANDRA-5567)
 +Merged from 1.1
   * Remove buggy thrift max message length option (CASSANDRA-5529)
   * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393)
 - * Use allocator information to improve memtable memory usage estimate 
 + * Use allocator information to improve memtable memory usage estimate
     (CASSANDRA-5497)
 + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
   * fsync leveled manifest to avoid corruption (CASSANDRA-5535)
   * Fix Bound intersection computation (CASSANDRA-5551)
 - * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 + * sstablescrub now respects max memory size in cassandra.in.sh (CASSANDRA-5562)
 +
 +
 +1.2.4
 + * Ensure that PerRowSecondaryIndex updates see the most recent values
 +   (CASSANDRA-5397)
 + * avoid duplicate index entries ind PrecompactedRow and 
 +   ParallelCompactionIterable (CASSANDRA-5395)
 + * remove the index entry on oldColumn when new column is a tombstone 
 +   (CASSANDRA-5395)
 + * Change default stream throughput from 400 to 200 mbps (CASSANDRA-5036)
 + * Gossiper logs DOWN for symmetry with UP (CASSANDRA-5187)
 + * Fix mixing prepared statements between keyspaces (CASSANDRA-5352)
 + * Fix consistency level during bootstrap - strike 3 (CASSANDRA-5354)
 + * Fix transposed arguments in AlreadyExistsException (CASSANDRA-5362)
 + * Improve asynchronous hint delivery (CASSANDRA-5179)
 + * Fix Guava dependency version (12.0 -> 13.0.1) for Maven (CASSANDRA-5364)
 + * Validate that provided CQL3 collection value are < 64K (CASSANDRA-5355)
 + * Make upgradeSSTable skip current version sstables by default (CASSANDRA-5366)
 + * Optimize min/max timestamp collection (CASSANDRA-5373)
 + * Invalid streamId in cql binary protocol when using invalid CL 
 +   (CASSANDRA-5164)
 + * Fix validation for IN where clauses with collections (CASSANDRA-5376)
 + * Copy resultSet on count query to avoid ConcurrentModificationException 
 +   (CASSANDRA-5382)
 + * Correctly typecheck in CQL3 even with ReversedType (CASSANDRA-5386)
 + * Fix streaming compressed files when using encryption (CASSANDRA-5391)
 + * cassandra-all 1.2.0 pom missing netty dependency (CASSANDRA-5392)
 + * Fix writetime/ttl functions on null values (CASSANDRA-5341)
 + * Fix NPE during cql3 select with token() (CASSANDRA-5404)
 + * IndexHelper.skipBloomFilters won't skip non-SHA filters (CASSANDRA-5385)
 + * cqlsh: Print maps ordered by key, sort sets (CASSANDRA-5413)
 + * Add null syntax support in CQL3 for inserts (CASSANDRA-3783)
 + * Allow unauthenticated set_keyspace() calls (CASSANDRA-5423)
 + * Fix potential incremental backups race (CASSANDRA-5410)
 + * Fix prepared BATCH statements with batch-level timestamps (CASSANDRA-5415)
 + * Allow overriding superuser setup delay (CASSANDRA-5430)
 + * cassandra-shuffle with JMX usernames and passwords (CASSANDRA-5431)
 +Merged from 1.1:
 + * cli: Quote ks and cf names in schema output when needed (CASSANDRA-5052)
 + * Fix bad default for min/max timestamp in SSTableMetadata (CASSANDRA-5372)
 + * Fix cf name extraction from manifest in Directories.migrateFile() 
 +   (CASSANDRA-5242)
 + * Support pluggable internode authentication (CASSANDRA-5401)
  
  
 -1.1.11
 +1.2.3
 + * add check for sstable overlap within a level on startup (CASSANDRA-5327)
 + * replace ipv6 colons in jmx object names (CASSANDRA-5298, 5328)
 + * Avoid allocating SSTableBoundedScanner during repair when the range does 
 +   not intersect the sstable (CASSANDRA-5249)
 + * Don't lowercase property map keys (this breaks NTS) (CASSANDRA-5292)
 + * Fix composite comparator with super columns (CASSANDRA-5287)
 + * Fix insufficient validation of UPDATE queries against counter cfs
 +   (CASSANDRA-5300)
 + * Fix PropertyFileSnitch default DC/Rack behavior (CASSANDRA-5285)
 + * Handle null values when executing prepared statement (CASSANDRA-5081)
 + * Add netty to pom dependencies (CASSANDRA-5181)
 + * Include type arguments in Thrift CQLPreparedResult (CASSANDRA-5311)
 + * Fix compaction not removing columns when bf_fp_ratio is 1 (CASSANDRA-5182)
 + * cli: Warn about missing CQL3 tables in schema descriptions (CASSANDRA-5309)
 + * Re-enable unknown option in replication/compaction strategies option for
 +   backward compatibility (CASSANDRA-4795)
 + * Add binary protocol support to stress (CASSANDRA-4993)
 + * cqlsh: Fix COPY FROM value quoting and null handling (CASSANDRA-5305)
 + * Fix repair -pr for vnodes (CASSANDRA-5329)
 + * Relax CL for auth queries for non-default users (CASSANDRA-5310)
 + * Fix AssertionError during repair (CASSANDRA-5245)
 + * Don't announce migrations to pre-1.2 nodes (CASSANDRA-5334)
 +Merged from 1.1:
   * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
   * Update offline scrub for 1.0 -> 1.1 directory structure (CASSANDRA-5195)
   * add tmp flag to Descriptor hashcode (CASSANDRA-4021)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1143df1a/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Column.java
index 616f3c0,ea83146..412808b
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@@ -25,7 -25,11 +25,8 @@@ import java.util.ArrayList
  import java.util.Collection;
  import java.util.List;
  
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 -
  import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.CFDefinition;
  import org.apache.cassandra.db.marshal.*;
  import org.apache.cassandra.io.util.DataOutputBuffer;
  import org.apache.cassandra.utils.Allocator;
@@@ -287,7 -278,23 +288,23 @@@ public class Column implements IColum
      public void validateFields(CFMetaData metadata) throws MarshalException
      {
          validateName(metadata);
-         AbstractType<?> valueValidator = metadata.getValueValidator(name());
+         CFDefinition cfdef = metadata.getCfDef();
+ 
+         // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+         // (Note that COMPACT composites are handled by validateName, above.)
+         ByteBuffer internalName;
+         if (cfdef.isComposite && !cfdef.isCompact)
+         {
 -            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
++            CompositeType comparator = (CompositeType) metadata.comparator;
+             List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+             internalName = components.get(components.size() - 1).value;
+         }
+         else
+         {
+             internalName = name;
+         }
+ 
+         AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
          if (valueValidator != null)
              valueValidator.validate(value());
      }


[08/16] git commit: merge from 1.2

Posted by jb...@apache.org.
merge from 1.2


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

Branch: refs/heads/cassandra-2.0.0
Commit: fdfdd5c12ed9dcc85fa85a038b6149d06e26e1be
Parents: f30629b 1143df1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:50:08 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:50:08 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  4 ++++
 src/java/org/apache/cassandra/db/Column.java | 18 +++++++++++++++++-
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fdfdd5c1/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 86401d9,30ce976..756e45c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,22 @@@
 -1.2.9
 - * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
 +2.0.0
 + * fix CAS contention timeout (CASSANDRA-5830)
 + * fix HsHa to respect max frame size (CASSANDRA-4573)
++Merged from 1.2:
++ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 +
 +
 +2.0.0-rc1
 + * improve DecimalSerializer performance (CASSANDRA-5837)
 + * fix potential spurious wakeup in AsyncOneResponse (CASSANDRA-5690)
 + * fix schema-related trigger issues (CASSANDRA-5774)
 + * Better validation when accessing CQL3 table from thrift (CASSANDRA-5138)
 + * Fix assertion error during repair (CASSANDRA-5801)
 + * Fix range tombstone bug (CASSANDRA-5805)
 + * DC-local CAS (CASSANDRA-5797)
 + * Add a native_protocol_version column to the system.local table (CASSANRDA-5819)
 + * Use index_interval from cassandra.yaml when upgraded (CASSANDRA-5822)
 + * Fix buffer underflow on socket close (CASSANDRA-5792)
 +Merged from 1.2:
   * fix bulk-loading compressed sstables (CASSANDRA-5820)
   * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
     (CASSANDRA-5824)
@@@ -26,6 -11,13 +28,8 @@@
     (CASSANDRA-5752)
   * add "all time blocked" to StatusLogger output (CASSANDRA-5825)
   * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
+  * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
+    (CASSANDRA-5718)
 -Merged from 1.1:
 - * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 -
 -
 -1.2.8
   * Fix reading DeletionTime from 1.1-format sstables (CASSANDRA-5814)
   * cqlsh: add collections support to COPY (CASSANDRA-5698)
   * retry important messages for any IOException (CASSANDRA-5804)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fdfdd5c1/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Column.java
index 297b280,412808b..d9e0416
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@@ -24,19 -22,15 +24,20 @@@ import java.net.InetAddress
  import java.nio.ByteBuffer;
  import java.security.MessageDigest;
  import java.util.ArrayList;
 -import java.util.Collection;
 +import java.util.Iterator;
  import java.util.List;
  
 +import com.google.common.collect.AbstractIterator;
 +
  import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.CFDefinition;
  import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.util.DataOutputBuffer;
 +import org.apache.cassandra.serializers.MarshalException;
  import org.apache.cassandra.utils.Allocator;
  import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.HeapAllocator;
  
  /**
@@@ -297,7 -288,23 +298,22 @@@ public class Column implements OnDiskAt
      public void validateFields(CFMetaData metadata) throws MarshalException
      {
          validateName(metadata);
-         AbstractType<?> valueValidator = metadata.getValueValidator(name());
+         CFDefinition cfdef = metadata.getCfDef();
+ 
+         // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+         // (Note that COMPACT composites are handled by validateName, above.)
+         ByteBuffer internalName;
+         if (cfdef.isComposite && !cfdef.isCompact)
+         {
+             CompositeType comparator = (CompositeType) metadata.comparator;
 -            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
 -            internalName = components.get(components.size() - 1).value;
++            internalName = comparator.extractLastComponent(name);
+         }
+         else
+         {
+             internalName = name;
+         }
+ 
+         AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
          if (valueValidator != null)
              valueValidator.validate(value());
      }


[10/16] git commit: Merge branch 'cassandra-2.0.0' into cassandra-2.0

Posted by jb...@apache.org.
Merge branch 'cassandra-2.0.0' 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/b5f445b4
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5f445b4
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5f445b4

Branch: refs/heads/trunk
Commit: b5f445b448e6047e717b17985c117843e66a2d40
Parents: c526131 fdfdd5c
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:50:16 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:50:16 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  4 ++++
 src/java/org/apache/cassandra/db/Column.java | 18 +++++++++++++++++-
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


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


[14/16] git commit: Merge remote-tracking branch 'origin/cassandra-2.0.0' into cassandra-2.0.0

Posted by jb...@apache.org.
Merge remote-tracking branch 'origin/cassandra-2.0.0' into cassandra-2.0.0


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

Branch: refs/heads/cassandra-2.0.0
Commit: 5411e5ff850bd1bbbf8d9f0057ff4136402b2bff
Parents: fdfdd5c 7a8cbb3
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:51:58 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:51:58 2013 -0500

----------------------------------------------------------------------
 lib/licenses/disruptor-3.0.1.txt                | 201 ++++++++++++++++++
 lib/licenses/thrift-server-0.3.0.txt            | 202 +++++++++++++++++++
 .../cassandra/service/CassandraDaemon.java      |  14 +-
 .../cassandra/service/StorageService.java       |   5 +-
 4 files changed, 406 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[02/16] git commit: Correctly validate sparse composite cells patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855

Posted by jb...@apache.org.
Correctly validate sparse composite cells
patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855


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

Branch: refs/heads/cassandra-2.0.0
Commit: eb884a582b709d67fdd075aedf19facf6803bf93
Parents: 238139c
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:33:39 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:43:43 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  1 +
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb884a58/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 75b349f..8790518 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 1.1.next
  * Backport compaction exception handling from 1.2
+ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 
 
 1.1.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb884a58/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Column.java b/src/java/org/apache/cassandra/db/Column.java
index 97aece0..ea83146 100644
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.utils.Allocator;
@@ -277,7 +278,23 @@ public class Column implements IColumn
     public void validateFields(CFMetaData metadata) throws MarshalException
     {
         validateName(metadata);
-        AbstractType<?> valueValidator = metadata.getValueValidator(name());
+        CFDefinition cfdef = metadata.getCfDef();
+
+        // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+        // (Note that COMPACT composites are handled by validateName, above.)
+        ByteBuffer internalName;
+        if (cfdef.isComposite && !cfdef.isCompact)
+        {
+            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
+            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+            internalName = components.get(components.size() - 1).value;
+        }
+        else
+        {
+            internalName = name;
+        }
+
+        AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
         if (valueValidator != null)
             valueValidator.validate(value());
     }


[07/16] git commit: merge from 1.2

Posted by jb...@apache.org.
merge from 1.2


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

Branch: refs/heads/trunk
Commit: fdfdd5c12ed9dcc85fa85a038b6149d06e26e1be
Parents: f30629b 1143df1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:50:08 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:50:08 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  4 ++++
 src/java/org/apache/cassandra/db/Column.java | 18 +++++++++++++++++-
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fdfdd5c1/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 86401d9,30ce976..756e45c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,22 @@@
 -1.2.9
 - * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
 +2.0.0
 + * fix CAS contention timeout (CASSANDRA-5830)
 + * fix HsHa to respect max frame size (CASSANDRA-4573)
++Merged from 1.2:
++ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 +
 +
 +2.0.0-rc1
 + * improve DecimalSerializer performance (CASSANDRA-5837)
 + * fix potential spurious wakeup in AsyncOneResponse (CASSANDRA-5690)
 + * fix schema-related trigger issues (CASSANDRA-5774)
 + * Better validation when accessing CQL3 table from thrift (CASSANDRA-5138)
 + * Fix assertion error during repair (CASSANDRA-5801)
 + * Fix range tombstone bug (CASSANDRA-5805)
 + * DC-local CAS (CASSANDRA-5797)
 + * Add a native_protocol_version column to the system.local table (CASSANRDA-5819)
 + * Use index_interval from cassandra.yaml when upgraded (CASSANDRA-5822)
 + * Fix buffer underflow on socket close (CASSANDRA-5792)
 +Merged from 1.2:
   * fix bulk-loading compressed sstables (CASSANDRA-5820)
   * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
     (CASSANDRA-5824)
@@@ -26,6 -11,13 +28,8 @@@
     (CASSANDRA-5752)
   * add "all time blocked" to StatusLogger output (CASSANDRA-5825)
   * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
+  * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
+    (CASSANDRA-5718)
 -Merged from 1.1:
 - * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 -
 -
 -1.2.8
   * Fix reading DeletionTime from 1.1-format sstables (CASSANDRA-5814)
   * cqlsh: add collections support to COPY (CASSANDRA-5698)
   * retry important messages for any IOException (CASSANDRA-5804)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fdfdd5c1/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Column.java
index 297b280,412808b..d9e0416
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@@ -24,19 -22,15 +24,20 @@@ import java.net.InetAddress
  import java.nio.ByteBuffer;
  import java.security.MessageDigest;
  import java.util.ArrayList;
 -import java.util.Collection;
 +import java.util.Iterator;
  import java.util.List;
  
 +import com.google.common.collect.AbstractIterator;
 +
  import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.CFDefinition;
  import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.util.DataOutputBuffer;
 +import org.apache.cassandra.serializers.MarshalException;
  import org.apache.cassandra.utils.Allocator;
  import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.HeapAllocator;
  
  /**
@@@ -297,7 -288,23 +298,22 @@@ public class Column implements OnDiskAt
      public void validateFields(CFMetaData metadata) throws MarshalException
      {
          validateName(metadata);
-         AbstractType<?> valueValidator = metadata.getValueValidator(name());
+         CFDefinition cfdef = metadata.getCfDef();
+ 
+         // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+         // (Note that COMPACT composites are handled by validateName, above.)
+         ByteBuffer internalName;
+         if (cfdef.isComposite && !cfdef.isCompact)
+         {
+             CompositeType comparator = (CompositeType) metadata.comparator;
 -            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
 -            internalName = components.get(components.size() - 1).value;
++            internalName = comparator.extractLastComponent(name);
+         }
+         else
+         {
+             internalName = name;
+         }
+ 
+         AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
          if (valueValidator != null)
              valueValidator.validate(value());
      }


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

Posted by jb...@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/670979d4
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/670979d4
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/670979d4

Branch: refs/heads/trunk
Commit: 670979d4ef0c528c5c1fcc8f61ff05f8293387ed
Parents: c7782f2 b5f445b
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:50:30 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:50:30 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  4 ++++
 src/java/org/apache/cassandra/db/Column.java | 18 +++++++++++++++++-
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[16/16] git commit: Merge remote-tracking branch 'origin/trunk' into trunk

Posted by jb...@apache.org.
Merge remote-tracking branch 'origin/trunk' into trunk


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

Branch: refs/heads/trunk
Commit: a1212c597482ba91dc41036f23199273e9177040
Parents: 670979d 28b827a
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:52:15 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:52:15 2013 -0500

----------------------------------------------------------------------
 .../org/apache/cassandra/service/CassandraDaemon.java | 14 ++------------
 .../org/apache/cassandra/service/StorageService.java  |  5 +----
 2 files changed, 3 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[03/16] git commit: Correctly validate sparse composite cells patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855

Posted by jb...@apache.org.
Correctly validate sparse composite cells
patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5855


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

Branch: refs/heads/trunk
Commit: eb884a582b709d67fdd075aedf19facf6803bf93
Parents: 238139c
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:33:39 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:43:43 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  1 +
 src/java/org/apache/cassandra/db/Column.java | 19 ++++++++++++++++++-
 2 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb884a58/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 75b349f..8790518 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 1.1.next
  * Backport compaction exception handling from 1.2
+ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 
 
 1.1.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/eb884a58/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Column.java b/src/java/org/apache/cassandra/db/Column.java
index 97aece0..ea83146 100644
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.utils.Allocator;
@@ -277,7 +278,23 @@ public class Column implements IColumn
     public void validateFields(CFMetaData metadata) throws MarshalException
     {
         validateName(metadata);
-        AbstractType<?> valueValidator = metadata.getValueValidator(name());
+        CFDefinition cfdef = metadata.getCfDef();
+
+        // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+        // (Note that COMPACT composites are handled by validateName, above.)
+        ByteBuffer internalName;
+        if (cfdef.isComposite && !cfdef.isCompact)
+        {
+            AbstractCompositeType comparator = (AbstractCompositeType) metadata.comparator;
+            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
+            internalName = components.get(components.size() - 1).value;
+        }
+        else
+        {
+            internalName = name;
+        }
+
+        AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
         if (valueValidator != null)
             valueValidator.validate(value());
     }


[15/16] git commit: Merge remote-tracking branch 'origin/cassandra-2.0' into cassandra-2.0

Posted by jb...@apache.org.
Merge remote-tracking branch 'origin/cassandra-2.0' 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/e911b767
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e911b767
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e911b767

Branch: refs/heads/cassandra-2.0
Commit: e911b767e1ca90acbb4be65475a239b4a663308f
Parents: b5f445b 5d9a33a
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:52:07 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:52:07 2013 -0500

----------------------------------------------------------------------
 .../org/apache/cassandra/service/CassandraDaemon.java | 14 ++------------
 .../org/apache/cassandra/service/StorageService.java  |  5 +----
 2 files changed, 3 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[09/16] git commit: merge from 1.2

Posted by jb...@apache.org.
merge from 1.2


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

Branch: refs/heads/cassandra-2.0
Commit: fdfdd5c12ed9dcc85fa85a038b6149d06e26e1be
Parents: f30629b 1143df1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Aug 8 15:50:08 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Aug 8 15:50:08 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                  |  4 ++++
 src/java/org/apache/cassandra/db/Column.java | 18 +++++++++++++++++-
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fdfdd5c1/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 86401d9,30ce976..756e45c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,22 @@@
 -1.2.9
 - * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
 +2.0.0
 + * fix CAS contention timeout (CASSANDRA-5830)
 + * fix HsHa to respect max frame size (CASSANDRA-4573)
++Merged from 1.2:
++ * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 +
 +
 +2.0.0-rc1
 + * improve DecimalSerializer performance (CASSANDRA-5837)
 + * fix potential spurious wakeup in AsyncOneResponse (CASSANDRA-5690)
 + * fix schema-related trigger issues (CASSANDRA-5774)
 + * Better validation when accessing CQL3 table from thrift (CASSANDRA-5138)
 + * Fix assertion error during repair (CASSANDRA-5801)
 + * Fix range tombstone bug (CASSANDRA-5805)
 + * DC-local CAS (CASSANDRA-5797)
 + * Add a native_protocol_version column to the system.local table (CASSANRDA-5819)
 + * Use index_interval from cassandra.yaml when upgraded (CASSANDRA-5822)
 + * Fix buffer underflow on socket close (CASSANDRA-5792)
 +Merged from 1.2:
   * fix bulk-loading compressed sstables (CASSANDRA-5820)
   * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
     (CASSANDRA-5824)
@@@ -26,6 -11,13 +28,8 @@@
     (CASSANDRA-5752)
   * add "all time blocked" to StatusLogger output (CASSANDRA-5825)
   * Future-proof inter-major-version schema migrations (CASSANDRA-5845)
+  * (Hadoop) add CqlPagingRecordReader support for ReversedType in Thrift table
+    (CASSANDRA-5718)
 -Merged from 1.1:
 - * Correctly validate sparse composite cells in scrub (CASSANDRA-5855)
 -
 -
 -1.2.8
   * Fix reading DeletionTime from 1.1-format sstables (CASSANDRA-5814)
   * cqlsh: add collections support to COPY (CASSANDRA-5698)
   * retry important messages for any IOException (CASSANDRA-5804)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fdfdd5c1/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Column.java
index 297b280,412808b..d9e0416
--- a/src/java/org/apache/cassandra/db/Column.java
+++ b/src/java/org/apache/cassandra/db/Column.java
@@@ -24,19 -22,15 +24,20 @@@ import java.net.InetAddress
  import java.nio.ByteBuffer;
  import java.security.MessageDigest;
  import java.util.ArrayList;
 -import java.util.Collection;
 +import java.util.Iterator;
  import java.util.List;
  
 +import com.google.common.collect.AbstractIterator;
 +
  import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.CFDefinition;
  import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.util.DataOutputBuffer;
 +import org.apache.cassandra.serializers.MarshalException;
  import org.apache.cassandra.utils.Allocator;
  import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.HeapAllocator;
  
  /**
@@@ -297,7 -288,23 +298,22 @@@ public class Column implements OnDiskAt
      public void validateFields(CFMetaData metadata) throws MarshalException
      {
          validateName(metadata);
-         AbstractType<?> valueValidator = metadata.getValueValidator(name());
+         CFDefinition cfdef = metadata.getCfDef();
+ 
+         // If this is a CQL table, we need to pull out the CQL column name to look up the correct column type.
+         // (Note that COMPACT composites are handled by validateName, above.)
+         ByteBuffer internalName;
+         if (cfdef.isComposite && !cfdef.isCompact)
+         {
+             CompositeType comparator = (CompositeType) metadata.comparator;
 -            List<AbstractCompositeType.CompositeComponent> components = comparator.deconstruct(name);
 -            internalName = components.get(components.size() - 1).value;
++            internalName = comparator.extractLastComponent(name);
+         }
+         else
+         {
+             internalName = name;
+         }
+ 
+         AbstractType<?> valueValidator = metadata.getValueValidator(internalName);
          if (valueValidator != null)
              valueValidator.validate(value());
      }