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 2013/05/26 13:17:50 UTC

[1/5] git commit: Remove buggy thrift max message length option patch by jbellis; reviewed by tjake for CASSANDRA-5529

Updated Branches:
  refs/heads/cassandra-1.2 1b5edee00 -> e771b0795


Remove buggy thrift max message length option
patch by jbellis; reviewed by tjake for CASSANDRA-5529


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

Branch: refs/heads/cassandra-1.2
Commit: 9879fa6122d325951d98f8bc601ff64dd04c2c67
Parents: 7d2ce5f
Author: Jonathan Ellis <jb...@apache.org>
Authored: Tue May 21 13:36:26 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Tue May 21 13:50:45 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                        |    2 +
 conf/cassandra.yaml                                |    6 +----
 src/java/org/apache/cassandra/config/Config.java   |    2 +
 .../cassandra/config/DatabaseDescriptor.java       |   10 --------
 .../cassandra/hadoop/ColumnFamilyOutputFormat.java |    4 +-
 .../cassandra/hadoop/ColumnFamilyRecordReader.java |    2 +-
 .../org/apache/cassandra/hadoop/ConfigHelper.java  |   15 ++---------
 .../apache/cassandra/thrift/CassandraDaemon.java   |    2 +-
 .../apache/cassandra/thrift/TBinaryProtocol.java   |   19 ---------------
 9 files changed, 12 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 256e69a..69df3de 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.1.12
+ * 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 
    (CASSANDRA-5497)
@@ -6,6 +7,7 @@
  * Fix Bound intersection computation (CASSANDRA-5551)
  * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 
+
 1.1.11
  * 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)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 37f41fb..027479d 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -330,15 +330,11 @@ rpc_server_type: sync
 # rpc_send_buff_size_in_bytes:
 # rpc_recv_buff_size_in_bytes:
 
-# Frame size for thrift (maximum field length).
+# Frame size for thrift (maximum message length).
 # 0 disables TFramedTransport in favor of TSocket. This option
 # is deprecated; we strongly recommend using Framed mode.
 thrift_framed_transport_size_in_mb: 15
 
-# The max length of a thrift message, including all fields and
-# internal thrift overhead.
-thrift_max_message_length_in_mb: 16
-
 # Set to true to have Cassandra create a hard link to each sstable
 # flushed or streamed locally in a backups/ subdirectory of the
 # Keyspace data.  Removing these links is the operator's

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index a08a694..11beea6 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -77,7 +77,9 @@ public class Config
     public Integer rpc_send_buff_size_in_bytes;
     public Integer rpc_recv_buff_size_in_bytes;
 
+    @Deprecated
     public Integer thrift_max_message_length_in_mb = 16;
+
     public Integer thrift_framed_transport_size_in_mb = 15;
     public Boolean snapshot_before_compaction = false;
     public Boolean auto_snapshot = true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 0c460dc..f55c89a 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -316,11 +316,6 @@ public class DatabaseDescriptor
             if (conf.thrift_framed_transport_size_in_mb <= 0)
                 throw new ConfigurationException("thrift_framed_transport_size_in_mb must be positive");
 
-            if (conf.thrift_framed_transport_size_in_mb > 0 && conf.thrift_max_message_length_in_mb < conf.thrift_framed_transport_size_in_mb)
-            {
-                throw new ConfigurationException("thrift_max_message_length_in_mb must be greater than thrift_framed_transport_size_in_mb when using TFramedTransport");
-            }
-
             /* end point snitch */
             if (conf.endpoint_snitch == null)
             {
@@ -582,11 +577,6 @@ public class DatabaseDescriptor
         return authority;
     }
 
-    public static int getThriftMaxMessageLength()
-    {
-        return conf.thrift_max_message_length_in_mb * 1024 * 1024;
-    }
-
     public static int getThriftFramedTransportSize()
     {
         return conf.thrift_framed_transport_size_in_mb * 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
index caea616..3818e53 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
@@ -118,7 +118,7 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
     }
 
     /** Fills the deprecated OutputFormat interface for streaming. */
-    @Deprecated @Override
+    @Deprecated
     public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
     {
         return new ColumnFamilyRecordWriter(job, new Progressable(progress));
@@ -155,7 +155,7 @@ public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutat
     {
         logger.debug("Creating authenticated client for CF output format");
         TTransport transport = ConfigHelper.getOutputTransportFactory(conf).openTransport(socket, conf);
-        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, ConfigHelper.getThriftMaxMessageLength(conf));
+        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
         Cassandra.Client client = new Cassandra.Client(binaryProtocol);
         client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
         if (ConfigHelper.getOutputKeyspaceUserName(conf) != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index e1ce326..2e7f09c 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -162,7 +162,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             String location = getLocation();
             socket = new TSocket(location, ConfigHelper.getInputRpcPort(conf));
             TTransport transport = ConfigHelper.getInputTransportFactory(conf).openTransport(socket, conf);
-            TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, ConfigHelper.getThriftMaxMessageLength(conf));
+            TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
             client = new Cassandra.Client(binaryProtocol);
 
             // log in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 9adc545..1cb4584 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -77,7 +77,6 @@ public class ConfigHelper
     private static final String INPUT_TRANSPORT_FACTORY_CLASS = "cassandra.input.transport.factory.class";
     private static final String OUTPUT_TRANSPORT_FACTORY_CLASS = "cassandra.output.transport.factory.class";
     private static final String THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB = "cassandra.thrift.framed.size_mb";
-    private static final String THRIFT_MAX_MESSAGE_LENGTH_IN_MB = "cassandra.thrift.message.max_size_mb";
 
     private static final Logger logger = LoggerFactory.getLogger(ConfigHelper.class);
 
@@ -482,18 +481,10 @@ public class ConfigHelper
         return conf.getInt(THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB, 15) * 1024 * 1024; // 15MB is default in Cassandra
     }
 
+    @Deprecated
     public static void setThriftMaxMessageLengthInMb(Configuration conf, int maxMessageSizeInMB)
     {
-        conf.setInt(THRIFT_MAX_MESSAGE_LENGTH_IN_MB, maxMessageSizeInMB);
-    }
-
-    /**
-     * @param conf The configuration to use.
-     * @return Value (converts MBs to Bytes) set by {@link setThriftMaxMessageLengthInMb(Configuration, int)} or default of 16MB
-     */
-    public static int getThriftMaxMessageLength(Configuration conf)
-    {
-        return conf.getInt(THRIFT_MAX_MESSAGE_LENGTH_IN_MB, 16) * 1024 * 1024; // 16MB is default in Cassandra
+        // SEE CASSANDRA-5529
     }
 
     public static CompressionParameters getOutputCompressionParamaters(Configuration conf)
@@ -557,7 +548,7 @@ public class ConfigHelper
         {
             TSocket socket = new TSocket(host, port);
             TTransport transport = getInputTransportFactory(conf).openTransport(socket, conf);
-            return new Cassandra.Client(new TBinaryProtocol(transport, getThriftMaxMessageLength(conf)));
+            return new Cassandra.Client(new TBinaryProtocol(transport, true, true));
         }
         catch (LoginException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraDaemon.java b/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
index 572e3e0..77c7808 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
@@ -125,7 +125,7 @@ public class CassandraDaemon extends org.apache.cassandra.service.AbstractCassan
             logger.info(String.format("Binding thrift service to %s:%s", listenAddr, listenPort));
 
             // Protocol factory
-            TProtocolFactory tProtocolFactory = new TBinaryProtocol.Factory(true, true, DatabaseDescriptor.getThriftMaxMessageLength());
+            TProtocolFactory tProtocolFactory = new TBinaryProtocol.Factory(true, true);
 
             // Transport factory
             int tFramedTransportSize = DatabaseDescriptor.getThriftFramedTransportSize();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9879fa61/src/java/org/apache/cassandra/thrift/TBinaryProtocol.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/TBinaryProtocol.java b/src/java/org/apache/cassandra/thrift/TBinaryProtocol.java
index 3d59f72..7d3ffcb 100644
--- a/src/java/org/apache/cassandra/thrift/TBinaryProtocol.java
+++ b/src/java/org/apache/cassandra/thrift/TBinaryProtocol.java
@@ -31,20 +31,6 @@ import java.nio.ByteBuffer;
 
 public class TBinaryProtocol extends org.apache.thrift.protocol.TBinaryProtocol
 {
-
-    public TBinaryProtocol(TTransport trans)
-    {
-        this(trans, false, true);
-    }
-
-    public TBinaryProtocol(TTransport trans, int readLength)
-    {
-        this(trans);
-
-        if (readLength > 0)
-            setReadLength(readLength);
-    }
-
     public TBinaryProtocol(TTransport trans, boolean strictRead, boolean strictWrite)
     {
         super(trans);
@@ -64,11 +50,6 @@ public class TBinaryProtocol extends org.apache.thrift.protocol.TBinaryProtocol
             super(strictRead, strictWrite, 0);
         }
 
-        public Factory(boolean strictRead, boolean strictWrite, int readLength)
-        {
-            super(strictRead, strictWrite, readLength);
-        }
-
         public TProtocol getProtocol(TTransport trans)
         {
             TBinaryProtocol protocol = new TBinaryProtocol(trans, strictRead_, strictWrite_);


[5/5] git commit: Merge branch 'cassandra-1.1' into cassandra-1.2

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

Conflicts:
	CHANGES.txt
	NEWS.txt
	build.xml
	conf/cassandra.yaml
	debian/changelog
	src/java/org/apache/cassandra/config/Config.java
	src/java/org/apache/cassandra/config/DatabaseDescriptor.java
	src/java/org/apache/cassandra/thrift/CassandraDaemon.java


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

Branch: refs/heads/cassandra-1.2
Commit: e771b07952ef3643ede5cf90226093de95397a41
Parents: 1b5edee 2dd73d1
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Sun May 26 13:17:15 2013 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Sun May 26 13:17:15 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../cassandra/hadoop/pig/CassandraStorage.java     |   34 ++++++++++-----
 2 files changed, 24 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e771b079/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 7b1970c,69df3de..f49a6f7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,123 -1,14 +1,124 @@@
 -1.1.12
 +1.2.6
 + * 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)
 +Merged from 1.1:
 + * Remove buggy thrift max message length option (CASSANDRA-5529)
 + * Fix NPE in Pig's widerow mode (CASSANDRA-5488)
 +
 +
 +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/e771b079/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------


[2/5] git commit: spilleng

Posted by sl...@apache.org.
spilleng


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

Branch: refs/heads/cassandra-1.2
Commit: 93cfbc187133d90d88a28826ad5989a53bde3e2f
Parents: 9879fa6
Author: Dave Brosius <db...@apache.org>
Authored: Tue May 21 20:51:04 2013 -0400
Committer: Dave Brosius <db...@apache.org>
Committed: Tue May 21 20:51:04 2013 -0400

----------------------------------------------------------------------
 .../org/apache/cassandra/tools/SSTableExport.java  |    2 +-
 1 files changed, 1 insertions(+), 1 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/93cfbc18/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index 9ac2123..37f158b 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -401,7 +401,7 @@ public class SSTableExport
         Descriptor descriptor = Descriptor.fromFilename(ssTableFileName);
         if (Schema.instance.getCFMetaData(descriptor) == null)
         {
-            System.err.println(String.format("The provided column family is not part of this cassandra database: keysapce = %s, column family = %s",
+            System.err.println(String.format("The provided column family is not part of this cassandra database: keyspace = %s, column family = %s",
                                              descriptor.ksname, descriptor.cfname));
             System.exit(1);
         }


[3/5] git commit: Updates news file, version and missing licenses for 1.1.12 release

Posted by sl...@apache.org.
Updates news file, version and missing licenses for 1.1.12 release


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

Branch: refs/heads/cassandra-1.2
Commit: 0db94069550b9c38b9f749e2087b196bb519664e
Parents: 93cfbc1
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed May 22 09:10:40 2013 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed May 22 09:10:40 2013 +0200

----------------------------------------------------------------------
 NEWS.txt                                           |    9 ++++++
 build.xml                                          |    2 +-
 debian/changelog                                   |    6 ++++
 .../org/apache/cassandra/MethodComparator.java     |   21 +++++++++++++
 .../apache/cassandra/OrderedJUnit4ClassRunner.java |   23 ++++++++++++++-
 5 files changed, 59 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0db94069/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index a768ddd..d5ba882 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -8,6 +8,15 @@ upgrade, just in case you need to roll back to the previous version.
 (Cassandra version X + 1 will always be able to read data files created
 by version X, but the inverse is not necessarily the case.)
 
+1.1.12
+======
+
+Upgrading
+---------
+    - Nothing specific to this release, but please see the previous instructions
+      if you are not upgrading from 1.1.11.
+
+
 1.1.11
 ======
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0db94069/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index b77c417..945fff7 100644
--- a/build.xml
+++ b/build.xml
@@ -25,7 +25,7 @@
     <property name="debuglevel" value="source,lines,vars"/>
 
     <!-- default version and SCM information -->
-    <property name="base.version" value="1.1.11"/>
+    <property name="base.version" value="1.1.12"/>
     <property name="scm.connection" value="scm:git://git.apache.org/cassandra.git"/>
     <property name="scm.developerConnection" value="scm:git://git.apache.org/cassandra.git"/>
     <property name="scm.url" value="http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0db94069/debian/changelog
----------------------------------------------------------------------
diff --git a/debian/changelog b/debian/changelog
index 76bac83..9e33d7c 100644
--- a/debian/changelog
+++ b/debian/changelog
@@ -1,3 +1,9 @@
+cassandra (1.1.12) unstable; urgency=low
+
+  * New release
+
+ -- Sylvain Lebresne <sl...@apache.org>  Wed, 22 May 2013 08:54:45 +0200
+
 cassandra (1.1.11) unstable; urgency=low
 
   * New release

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0db94069/test/unit/org/apache/cassandra/MethodComparator.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/MethodComparator.java b/test/unit/org/apache/cassandra/MethodComparator.java
index 690ae57..8cc163a 100644
--- a/test/unit/org/apache/cassandra/MethodComparator.java
+++ b/test/unit/org/apache/cassandra/MethodComparator.java
@@ -1,4 +1,25 @@
 package org.apache.cassandra;
+/*
+ * 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ * 
+ */
+
 
 import org.junit.Ignore;
 import org.junit.runners.model.FrameworkMethod;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0db94069/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java b/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java
index d84aedb..d0dec24 100644
--- a/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java
+++ b/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java
@@ -1,4 +1,25 @@
 package org.apache.cassandra;
+/*
+ * 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ * 
+ */
+
 
 import org.junit.runners.BlockJUnit4ClassRunner;
 import org.junit.runners.model.FrameworkMethod;
@@ -31,4 +52,4 @@ public class OrderedJUnit4ClassRunner extends BlockJUnit4ClassRunner
             return list;
         }
     }
-}
\ No newline at end of file
+}


[4/5] git commit: Fix 5488 round 2

Posted by sl...@apache.org.
Fix 5488 round 2


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

Branch: refs/heads/cassandra-1.2
Commit: 2dd73d171068d743befcd445a14751032d232e4e
Parents: 0db9406
Author: Brandon Williams <br...@apache.org>
Authored: Wed May 22 11:18:59 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Wed May 22 11:19:05 2013 -0500

----------------------------------------------------------------------
 .../cassandra/hadoop/pig/CassandraStorage.java     |   34 ++++++++++-----
 1 files changed, 23 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2dd73d17/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index b681ee3..cf1c08f 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -130,7 +130,7 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
     {
         CfDef cfDef = getCfDef(loadSignature);
         ByteBuffer key = null;
-        Tuple tuple = TupleFactory.getInstance().newTuple();
+        Tuple tuple = null; 
         DefaultDataBag bag = new DefaultDataBag();
         try
         {
@@ -139,12 +139,15 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
                 hasNext = reader.nextKeyValue();
                 if (!hasNext)
                 {
+                    if (tuple == null)
+                        tuple = TupleFactory.getInstance().newTuple();
+
                     if (lastRow != null)
                     {
                         if (tuple.size() == 0) // lastRow is a new one
                         {
                             key = (ByteBuffer)reader.getCurrentKey();
-                            tuple = addKeyToTuple(tuple, key, cfDef, parseType(cfDef.getKey_validation_class()));
+                            tuple = keyToTuple(key, cfDef, parseType(cfDef.getKey_validation_class()));
                         }
                         for (Map.Entry<ByteBuffer, IColumn> entry : lastRow.entrySet())
                         {
@@ -180,7 +183,10 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
                     key = (ByteBuffer)reader.getCurrentKey();
                     if (lastKey != null && !(key.equals(lastKey))) // last key only had one value
                     {
-                        tuple = addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
+                        if (tuple == null)
+                            tuple = keyToTuple(lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
+                        else
+                            addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
                         for (Map.Entry<ByteBuffer, IColumn> entry : lastRow.entrySet())
                         {
                             bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
@@ -190,7 +196,10 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
                         lastRow = (SortedMap<ByteBuffer,IColumn>)reader.getCurrentValue();
                         return tuple;
                     }
-                    tuple = addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
+                    if (tuple == null)
+                        tuple = keyToTuple(key, cfDef, parseType(cfDef.getKey_validation_class()));
+                    else
+                        addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
                 }
                 SortedMap<ByteBuffer,IColumn> row = (SortedMap<ByteBuffer,IColumn>)reader.getCurrentValue();
                 if (lastRow != null) // prepend what was read last time
@@ -233,7 +242,7 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
             // output tuple, will hold the key, each indexed column in a tuple, then a bag of the rest
             // NOTE: we're setting the tuple size here only for the key so we can use setTupleValue on it
 
-            Tuple tuple = addKeyToTuple(null, key, cfDef, parseType(cfDef.getKey_validation_class()));
+            Tuple tuple = keyToTuple(key, cfDef, parseType(cfDef.getKey_validation_class()));
             DefaultDataBag bag = new DefaultDataBag();
 
             // we must add all the indexed columns first to match the schema
@@ -292,12 +301,15 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
         return t;
     }
 
-    private Tuple addKeyToTuple(Tuple tuple, ByteBuffer key, CfDef cfDef, AbstractType comparator) throws IOException
+    private Tuple keyToTuple(ByteBuffer key, CfDef cfDef, AbstractType comparator) throws IOException
+    {
+        Tuple tuple = TupleFactory.getInstance().newTuple(1);
+        addKeyToTuple(tuple, key, cfDef, comparator);
+        return tuple;
+    }
+
+    private void addKeyToTuple(Tuple tuple, ByteBuffer key, CfDef cfDef, AbstractType comparator) throws IOException
     {
-        if( tuple == null )
-        {
-            tuple = TupleFactory.getInstance().newTuple(1);
-        }
         if( comparator instanceof AbstractCompositeType )
         {
             setTupleValue(tuple, 0, composeComposite((AbstractCompositeType)comparator,key));
@@ -306,7 +318,7 @@ public class CassandraStorage extends LoadFunc implements StoreFuncInterface, Lo
         {
             setTupleValue(tuple, 0, getDefaultMarshallers(cfDef).get(MarshallerType.KEY_VALIDATOR).compose(key));
         }
-        return tuple;
+
     }
 
     private Tuple columnToTuple(IColumn col, CfDef cfDef, AbstractType comparator) throws IOException