[jira] [Comment Edited] (CASSANDRA-1472) Add bitmap secondary indexes

2013-04-11 Thread Stu Hood (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13003216#comment-13003216
 ] 

Stu Hood edited comment on CASSANDRA-1472 at 4/12/13 5:23 AM:
--

tjake: Yea: opening a separate ticket to discuss a generic file format makes 
sense.

I had a realization about this implementation of secondary indexes: I was 
originally thinking we'd be able to push all boolean queries down to the 
indexes on a per-sstable basis, but this is unfortunately not the case. We will 
not be able to push 'AND' on separate indexes down to the sstables themselves: 
we'd need to join the index from all sstables, since a row might contain one 
clause in one sstable, and another clause in another sstable.

EDIT: This is roughly equivalent to what we'd need to do with a KEYS index 
(seek-wise), meaning that the advantage is mostly in space utilization and lack 
of locks.
EDIT2: So, there _is_ a way to execute AND queries directly per SSTable, but it 
involves some uncertainty. For a particular row, if a value involved in a 
multi-clause-query is NULL in a particular SSTable, then you have to accept the 
row as a possible match, and resolve the uncertainty later. I'm sure there is a 
way to incoporate the CASSANDRA-2498 timestamp resolution as well, although it 
doesn't occur to me at the moment.

  was (Author: stuhood):
tjake: Yea: opening a separate ticket to discuss a generic file format 
makes sense.

I had a realization about this implementation of secondary indexes: I was 
originally thinking we'd be able to push all boolean queries down to the 
indexes on a per-sstable basis, but this is unfortunately not the case. We will 
not be able to push 'AND' on separate indexes down to the sstables themselves: 
we'd need to join the index from all sstables, since a row might contain one 
clause in one sstable, and another clause in another sstable. EDIT: This is 
roughly equivalent to what we'd need to do with a KEYS index (seek-wise), 
meaning that the advantage is mostly in space utilization and lack of locks.
  
> Add bitmap secondary indexes
> 
>
> Key: CASSANDRA-1472
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1472
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Stu Hood
> Attachments: 0.7-1472-v5.tgz, 0.7-1472-v6.tgz, 1472-v3.tgz, 
> 1472-v4.tgz, 1472-v5.tgz, anatomy.png, 
> ASF.LICENSE.NOT.GRANTED--0001-CASSANDRA-1472-rebased-to-0.7-branch.txt, 
> ASF.LICENSE.NOT.GRANTED--0019-Rename-bugfixes-and-fileclose.txt, 
> v4-bench-c32.txt
>
>
> Bitmap indexes are a very efficient structure for dealing with immutable 
> data. We can take advantage of the fact that SSTables are immutable by 
> attaching them directly to SSTables as a new component (supported by 
> CASSANDRA-1471).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5361) Enable ThreadLocal allocation in the JVM

2013-04-11 Thread Ahmed Bashir (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629795#comment-13629795
 ] 

Ahmed Bashir commented on CASSANDRA-5361:
-

Incidentally we've operated with this option enabled for some a few months now 
and have observed reduced latency as well

> Enable ThreadLocal allocation in the JVM
> 
>
> Key: CASSANDRA-5361
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5361
> Project: Cassandra
>  Issue Type: Improvement
> Environment: JDK 6
>Reporter: T Jake Luciani
>Assignee: T Jake Luciani
> Fix For: 1.1.11, 1.2.4
>
>
> Adding -XX:+UseTLAB to cassandra-env.sh yields a ~15% read performance boost.
> Works with usual stress tool but also in our production.
> It enables thread local allocation of new objects which makes sense for SEDA 
> based systems.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-5361) Enable ThreadLocal allocation in the JVM

2013-04-11 Thread Ahmed Bashir (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629795#comment-13629795
 ] 

Ahmed Bashir edited comment on CASSANDRA-5361 at 4/12/13 5:01 AM:
--

Incidentally we've operated with this option enabled for a few months now and 
have observed reduced latency as well

  was (Author: abashir):
Incidentally we've operated with this option enabled for some a few months 
now and have observed reduced latency as well
  
> Enable ThreadLocal allocation in the JVM
> 
>
> Key: CASSANDRA-5361
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5361
> Project: Cassandra
>  Issue Type: Improvement
> Environment: JDK 6
>Reporter: T Jake Luciani
>Assignee: T Jake Luciani
> Fix For: 1.1.11, 1.2.4
>
>
> Adding -XX:+UseTLAB to cassandra-env.sh yields a ~15% read performance boost.
> Works with usual stress tool but also in our production.
> It enables thread local allocation of new objects which makes sense for SEDA 
> based systems.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4860) Estimated Row Cache Entry size incorrect (always 24?)

2013-04-11 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629783#comment-13629783
 ] 

Vijay commented on CASSANDRA-4860:
--

Added unit test and pushed to 
https://github.com/Vijay2win/cassandra/commits/4860-v4, Thanks!

> Estimated Row Cache Entry size incorrect (always 24?)
> -
>
> Key: CASSANDRA-4860
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4860
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.0, 1.2.3, 2.0
>Reporter: Chris Burroughs
>Assignee: Vijay
> Fix For: 1.2.0 beta 3
>
> Attachments: 0001-4860-v2.patch, 0001-4860-v3.patch, 
> 0001-CASSANDRA-4860-for-11.patch, 0001-CASSANDRA-4860.patch, 
> 4860-perf-test.zip, trunk-4860-revert.patch
>
>
> After running for several hours the RowCacheSize was suspicious low (ie 70 
> something MB)  I used  CASSANDRA-4859 to measure the size and number of 
> entries on a node:
> In [3]: 1560504./65021
> Out[3]: 24.0
> In [4]: 2149464./89561
> Out[4]: 24.0
> In [6]: 7216096./300785
> Out[6]: 23.990877204647838
> That's RowCacheSize/RowCacheNumEntires  .  Just to prove I don't have crazy 
> small rows the mean size of the row *keys* in the saved cache is 67 and 
> Compacted row mean size: 355.  No jamm errors in the log
> Config notes:
> row_cache_provider: ConcurrentLinkedHashCacheProvider
> row_cache_size_in_mb: 2048
> Version info:
>  * C*: 1.1.6
>  * centos 2.6.32-220.13.1.el6.x86_64
>  * java 6u31 Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4860) Estimated Row Cache Entry size incorrect (always 24?)

2013-04-11 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629757#comment-13629757
 ] 

Vijay edited comment on CASSANDRA-4860 at 4/12/13 4:00 AM:
---

This is my theory:

2M KV with Measure.measure() will take 96,000,000 or 96M (2 *24 * 200 
bytes) will fit in key cache.
2M KV with measureDeep() will take 96M + 48M (48 * 200 + 24 * 200) 
where 48 is the index min size and 24 is the key size.

Hence there is a eviction overhead on the keycache which you dont have in 
Measure.measure().
Give the above if you have the key cache of 300M and re test both v3 should 
show a better performance.

{code}
Completed warmup!, Number of Iteratoions: 100
Using reflection took: 8037
Using 4860-v3 took: 90
Using MemoryMeter meter.measure(key) took: 190
Using MemoryMeter meter.measureDeep(key) took: 1002
Using 4860-v3 RowIndexEntry took: 14
Using MemoryMeter meter.measure(RowIndexEntry(i)) took: 104
Using MemoryMeter meter.measureDeep(RowIndexEntry(i)) took: 459
Size of Meter.measure key: 24
Size of Meter.measure index: 24
Size of Meter.measureDeep key: 48
Size of Meter.measureDeep index: 24
Size of key: 48
Size of index: 24
{code}

  was (Author: vijay2...@yahoo.com):
This is my theory:

2M KV with Measure.measure() will take 96,000,000 or 96M (2 *24 * 200 
bytes) will fit in key cache.
2M KV with measureDeep() will take 96M + 48M (48 * 200 + 24 * 200) 
where 48 is the index min size and 24 is the key size.

Give the above if you have the key cache of 300M and re test both v3 should 
show a better performance.

{code}
Completed warmup!, Number of Iteratoions: 100
Using reflection took: 8037
Using 4860-v3 took: 90
Using MemoryMeter meter.measure(key) took: 190
Using MemoryMeter meter.measureDeep(key) took: 1002
Using 4860-v3 RowIndexEntry took: 14
Using MemoryMeter meter.measure(RowIndexEntry(i)) took: 104
Using MemoryMeter meter.measureDeep(RowIndexEntry(i)) took: 459
Size of Meter.measure key: 24
Size of Meter.measure index: 24
Size of Meter.measureDeep key: 48
Size of Meter.measureDeep index: 24
Size of key: 48
Size of index: 24
{code}
  
> Estimated Row Cache Entry size incorrect (always 24?)
> -
>
> Key: CASSANDRA-4860
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4860
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.0, 1.2.3, 2.0
>Reporter: Chris Burroughs
>Assignee: Vijay
> Fix For: 1.2.0 beta 3
>
> Attachments: 0001-4860-v2.patch, 0001-4860-v3.patch, 
> 0001-CASSANDRA-4860-for-11.patch, 0001-CASSANDRA-4860.patch, 
> 4860-perf-test.zip, trunk-4860-revert.patch
>
>
> After running for several hours the RowCacheSize was suspicious low (ie 70 
> something MB)  I used  CASSANDRA-4859 to measure the size and number of 
> entries on a node:
> In [3]: 1560504./65021
> Out[3]: 24.0
> In [4]: 2149464./89561
> Out[4]: 24.0
> In [6]: 7216096./300785
> Out[6]: 23.990877204647838
> That's RowCacheSize/RowCacheNumEntires  .  Just to prove I don't have crazy 
> small rows the mean size of the row *keys* in the saved cache is 67 and 
> Compacted row mean size: 355.  No jamm errors in the log
> Config notes:
> row_cache_provider: ConcurrentLinkedHashCacheProvider
> row_cache_size_in_mb: 2048
> Version info:
>  * C*: 1.1.6
>  * centos 2.6.32-220.13.1.el6.x86_64
>  * java 6u31 Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4860) Estimated Row Cache Entry size incorrect (always 24?)

2013-04-11 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629757#comment-13629757
 ] 

Vijay commented on CASSANDRA-4860:
--

This is my theory:

2M KV with Measure.measure() will take 96,000,000 or 96M (2 *24 * 200 
bytes) will fit in key cache.
2M KV with measureDeep() will take 96M + 48M (48 * 200 + 24 * 200) 
where 48 is the index min size and 24 is the key size.

Give the above if you have the key cache of 300M and re test both v3 should 
show a better performance.

{code}
Completed warmup!, Number of Iteratoions: 100
Using reflection took: 8037
Using 4860-v3 took: 90
Using MemoryMeter meter.measure(key) took: 190
Using MemoryMeter meter.measureDeep(key) took: 1002
Using 4860-v3 RowIndexEntry took: 14
Using MemoryMeter meter.measure(RowIndexEntry(i)) took: 104
Using MemoryMeter meter.measureDeep(RowIndexEntry(i)) took: 459
Size of Meter.measure key: 24
Size of Meter.measure index: 24
Size of Meter.measureDeep key: 48
Size of Meter.measureDeep index: 24
Size of key: 48
Size of index: 24
{code}

> Estimated Row Cache Entry size incorrect (always 24?)
> -
>
> Key: CASSANDRA-4860
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4860
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.0, 1.2.3, 2.0
>Reporter: Chris Burroughs
>Assignee: Vijay
> Fix For: 1.2.0 beta 3
>
> Attachments: 0001-4860-v2.patch, 0001-4860-v3.patch, 
> 0001-CASSANDRA-4860-for-11.patch, 0001-CASSANDRA-4860.patch, 
> 4860-perf-test.zip, trunk-4860-revert.patch
>
>
> After running for several hours the RowCacheSize was suspicious low (ie 70 
> something MB)  I used  CASSANDRA-4859 to measure the size and number of 
> entries on a node:
> In [3]: 1560504./65021
> Out[3]: 24.0
> In [4]: 2149464./89561
> Out[4]: 24.0
> In [6]: 7216096./300785
> Out[6]: 23.990877204647838
> That's RowCacheSize/RowCacheNumEntires  .  Just to prove I don't have crazy 
> small rows the mean size of the row *keys* in the saved cache is 67 and 
> Compacted row mean size: 355.  No jamm errors in the log
> Config notes:
> row_cache_provider: ConcurrentLinkedHashCacheProvider
> row_cache_size_in_mb: 2048
> Version info:
>  * C*: 1.1.6
>  * centos 2.6.32-220.13.1.el6.x86_64
>  * java 6u31 Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


git commit: make CompositesIndex.IndexEntry static to reduce footprint

2013-04-11 Thread dbrosius
Updated Branches:
  refs/heads/trunk 8ce9edf04 -> 6b4f2ee59


make CompositesIndex.IndexEntry static to reduce footprint


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

Branch: refs/heads/trunk
Commit: 6b4f2ee591e63b435300cd602aa5a8c66aaabfb5
Parents: 8ce9edf
Author: Dave Brosius 
Authored: Thu Apr 11 23:01:43 2013 -0400
Committer: Dave Brosius 
Committed: Thu Apr 11 23:01:43 2013 -0400

--
 .../db/index/composites/CompositesIndex.java   |2 +-
 1 files changed, 1 insertions(+), 1 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6b4f2ee5/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
--
diff --git 
a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java 
b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index 202efbf..131b8c6 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -135,7 +135,7 @@ public abstract class CompositesIndex extends 
AbstractSimplePerColumnSecondaryIn
 throw new ConfigurationException("Unknown options provided for 
COMPOSITES index: " + options.keySet());
 }
 
-public class IndexedEntry
+public static class IndexedEntry
 {
 public final DecoratedKey indexValue;
 public final ByteBuffer indexEntry;



buildbot success in ASF Buildbot on cassandra-trunk

2013-04-11 Thread buildbot
The Buildbot has detected a restored build on builder cassandra-trunk while 
building cassandra.
Full details are available at:
 http://ci.apache.org/builders/cassandra-trunk/builds/2566

Buildbot URL: http://ci.apache.org/

Buildslave for this Build: portunus_ubuntu

Build Reason: scheduler
Build Source Stamp: [branch trunk] 8ce9edf0409945c81f36c08f0e8e39c1ccb7db4b
Blamelist: Jonathan Ellis 

Build succeeded!

sincerely,
 -The Buildbot





git commit: fix SerializationsTest

2013-04-11 Thread jbellis
Updated Branches:
  refs/heads/trunk f04359d91 -> 8ce9edf04


fix SerializationsTest


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

Branch: refs/heads/trunk
Commit: 8ce9edf0409945c81f36c08f0e8e39c1ccb7db4b
Parents: f04359d
Author: Jonathan Ellis 
Authored: Thu Apr 11 21:31:27 2013 -0500
Committer: Jonathan Ellis 
Committed: Thu Apr 11 21:31:27 2013 -0500

--
 src/java/org/apache/cassandra/db/RowMutation.java  |5 -
 test/data/serialization/2.0/db.RowMutation.bin |  Bin 3770 -> 3599 bytes
 .../cassandra/AbstractSerializationsTester.java|1 -
 .../apache/cassandra/db/SerializationsTest.java|   12 
 4 files changed, 8 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ce9edf0/src/java/org/apache/cassandra/db/RowMutation.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutation.java 
b/src/java/org/apache/cassandra/db/RowMutation.java
index ef63027..9e0e78f 100644
--- a/src/java/org/apache/cassandra/db/RowMutation.java
+++ b/src/java/org/apache/cassandra/db/RowMutation.java
@@ -40,7 +40,10 @@ public class RowMutation implements IMutation
 public static final String FORWARD_TO = "FWD_TO";
 public static final String FORWARD_FROM = "FWD_FRM";
 
-private final String table; // todo this is redundant
+// todo this is redundant
+// when we remove it, also restore SerializationsTest.testRowMutationRead 
to not regenerate new RowMutations each test
+private final String table;
+
 private final ByteBuffer key;
 // map of column family id to mutations for that column family.
 private final Map modifications;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ce9edf0/test/data/serialization/2.0/db.RowMutation.bin
--
diff --git a/test/data/serialization/2.0/db.RowMutation.bin 
b/test/data/serialization/2.0/db.RowMutation.bin
index 0f024ac..94bb810 100644
Binary files a/test/data/serialization/2.0/db.RowMutation.bin and 
b/test/data/serialization/2.0/db.RowMutation.bin differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ce9edf0/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
--
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java 
b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 807fd98..22b2424 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -44,7 +44,6 @@ public class AbstractSerializationsTester extends SchemaLoader
 put("2.0", MessagingService.VERSION_20);
 }};
 
-// TODO ant doesn't pass this -D up to the test, so it's kind of useless
 protected static final boolean EXECUTE_WRITES = 
Boolean.getBoolean("cassandra.test-serialization-writes");
 
 protected final int getVersion()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ce9edf0/test/unit/org/apache/cassandra/db/SerializationsTest.java
--
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java 
b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 8c9af0f..f87a0f8 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -210,7 +210,6 @@ public class SerializationsTest extends 
AbstractSerializationsTester
 
 private void testRowMutationWrite() throws IOException
 {
-RowMutation emptyRm = new RowMutation(statics.KS, statics.Key);
 RowMutation standardRowRm = new RowMutation(statics.KS, 
statics.StandardRow);
 RowMutation superRowRm = new RowMutation(statics.KS, statics.SuperRow);
 RowMutation standardRm = new RowMutation(statics.KS, statics.Key, 
statics.StandardCf);
@@ -221,14 +220,12 @@ public class SerializationsTest extends 
AbstractSerializationsTester
 RowMutation mixedRm = new RowMutation(statics.KS, statics.Key, mods);
 
 DataOutputStream out = getOutput("db.RowMutation.bin");
-RowMutation.serializer.serialize(emptyRm, out, getVersion());
 RowMutation.serializer.serialize(standardRowRm, out, getVersion());
 RowMutation.serializer.serialize(superRowRm, out, getVersion());
 RowMutation.serializer.serialize(standardRm, out, getVersion());
 RowMutation.serializer.serialize(superRm, out, getVersion());

buildbot failure in ASF Buildbot on cassandra-trunk

2013-04-11 Thread buildbot
The Buildbot has detected a new failure on builder cassandra-trunk while 
building cassandra.
Full details are available at:
 http://ci.apache.org/builders/cassandra-trunk/builds/2565

Buildbot URL: http://ci.apache.org/

Buildslave for this Build: portunus_ubuntu

Build Reason: scheduler
Build Source Stamp: [branch trunk] f04359d91ae3ea3adca6e13f74a7c54e3845cab7
Blamelist: Jonathan Ellis 

BUILD FAILED: failed shell

sincerely,
 -The Buildbot





[jira] [Commented] (CASSANDRA-5357) Query cache

2013-04-11 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629695#comment-13629695
 ] 

Vijay commented on CASSANDRA-5357:
--

{quote}
I don't follow – how can you have both O(1) [Map key is the row key] and also 
promote QF into the key?
{quote}
Well I am talking about the best case, where the queries are fairly limited on 
the rows.

{quote}
We're talking about K=Map Key, right? How do you see QF increasing by row size?
{quote}
K (Map Key) where K is a list of queries, (RowKey + [,]). 
NOTE: we will only hash/equals based of the rowKey and not the entier Map Key, 
once we reach the map key we can verify if the particular query exist in the 
key by a linear scan. 

The value being a set of columns which match all the Key's queries

The main reason for complicating the above is to avoid 2 maps (POC code in 
#1956 does that), one to map  and other to hold the actual query 
 to help invalidate or even update the cache when there is an update.



> Query cache
> ---
>
> Key: CASSANDRA-5357
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5357
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Jonathan Ellis
>Assignee: Vijay
>
> I think that most people expect the row cache to act like a query cache, 
> because that's a reasonable model.  Caching the entire partition is, in 
> retrospect, not really reasonable, so it's not surprising that it catches 
> people off guard, especially given the confusion we've inflicted on ourselves 
> as to what a "row" constitutes.
> I propose replacing it with a true query cache.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


git commit: Avoid serializing keyspace redundantly in RowMutation patch by jbellis; reviewed by vijay for CASSANDRA-5458

2013-04-11 Thread jbellis
Updated Branches:
  refs/heads/trunk 0c7141ee8 -> f04359d91


Avoid serializing keyspace redundantly in RowMutation
patch by jbellis; reviewed by vijay for CASSANDRA-5458


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

Branch: refs/heads/trunk
Commit: f04359d91ae3ea3adca6e13f74a7c54e3845cab7
Parents: 0c7141e
Author: Jonathan Ellis 
Authored: Thu Apr 11 21:06:28 2013 -0500
Committer: Jonathan Ellis 
Committed: Thu Apr 11 21:06:28 2013 -0500

--
 src/java/org/apache/cassandra/db/RowMutation.java |   22 ---
 1 files changed, 17 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f04359d9/src/java/org/apache/cassandra/db/RowMutation.java
--
diff --git a/src/java/org/apache/cassandra/db/RowMutation.java 
b/src/java/org/apache/cassandra/db/RowMutation.java
index dcc62c5..ef63027 100644
--- a/src/java/org/apache/cassandra/db/RowMutation.java
+++ b/src/java/org/apache/cassandra/db/RowMutation.java
@@ -69,7 +69,7 @@ public class RowMutation implements IMutation
 
 public RowMutation(ByteBuffer key, ColumnFamily cf)
 {
-this(Schema.instance.getCFMetaData(cf.id()).ksName, key, cf);
+this(cf.metadata().ksName, key, cf);
 }
 
 public String getTable()
@@ -239,13 +239,15 @@ public class RowMutation implements IMutation
 {
 public void serialize(RowMutation rm, DataOutput out, int version) 
throws IOException
 {
-out.writeUTF(rm.getTable());
+if (version < MessagingService.VERSION_20)
+out.writeUTF(rm.getTable());
+
 ByteBufferUtil.writeWithShortLength(rm.key(), out);
 
 /* serialize the modifications in the mutation */
 int size = rm.modifications.size();
 out.writeInt(size);
-assert size >= 0;
+assert size > 0;
 for (Map.Entry entry : 
rm.modifications.entrySet())
 {
 if (version < MessagingService.VERSION_12)
@@ -256,15 +258,20 @@ public class RowMutation implements IMutation
 
 public RowMutation deserialize(DataInput in, int version, 
ColumnSerializer.Flag flag) throws IOException
 {
-String table = in.readUTF();
+String table = null; // will always be set from cf.metadata but 
javac isn't smart enough to see that
+if (version < MessagingService.VERSION_20)
+table = in.readUTF();
+
 ByteBuffer key = ByteBufferUtil.readWithShortLength(in);
 int size = in.readInt();
+assert size > 0;
 
 Map modifications;
 if (size == 1)
 {
 ColumnFamily cf = deserializeOneCf(in, version, flag);
 modifications = Collections.singletonMap(cf.id(), cf);
+table = cf.metadata().ksName;
 }
 else
 {
@@ -273,6 +280,7 @@ public class RowMutation implements IMutation
 {
 ColumnFamily cf = deserializeOneCf(in, version, flag);
 modifications.put(cf.id(), cf);
+table = cf.metadata().ksName;
 }
 }
 
@@ -298,7 +306,11 @@ public class RowMutation implements IMutation
 public long serializedSize(RowMutation rm, int version)
 {
 TypeSizes sizes = TypeSizes.NATIVE;
-int size = sizes.sizeof(rm.getTable());
+int size = 0;
+
+if (version < MessagingService.VERSION_20)
+size += sizes.sizeof(rm.getTable());
+
 int keySize = rm.key().remaining();
 size += sizes.sizeof((short) keySize) + keySize;
 



[jira] [Commented] (CASSANDRA-5458) Avoid serializing keyspace redundantly in RowMutation

2013-04-11 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5458?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629667#comment-13629667
 ] 

Vijay commented on CASSANDRA-5458:
--

+1 LGTM

> Avoid serializing keyspace redundantly in RowMutation
> -
>
> Key: CASSANDRA-5458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5458
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Jonathan Ellis
>Assignee: Jonathan Ellis
>Priority: Minor
> Fix For: 2.0
>
> Attachments: 5458.txt
>
>
> We can infer the table from the CFID, so there's no need to de/serialize it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4338) Experiment with direct buffer in SequentialWriter

2013-04-11 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4338?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629663#comment-13629663
 ] 

Jonathan Ellis commented on CASSANDRA-4338:
---

Relevant: 
http://mechanical-sympathy.blogspot.com/2011/12/java-sequential-io-performance.html

> Experiment with direct buffer in SequentialWriter
> -
>
> Key: CASSANDRA-4338
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4338
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Jonathan Ellis
>Assignee: Aleksey Yeschenko
>Priority: Minor
> Attachments: 4338-gc.tar.gz, gc-4338-patched.png, gc-trunk.png
>
>
> Using a direct buffer instead of a heap-based byte[] should let us avoid a 
> copy into native memory when we flush the buffer.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

2013-04-11 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629622#comment-13629622
 ] 

Jonathan Ellis commented on CASSANDRA-4718:
---

bq. LMAX have recently open sourced a coalescing ring buffer that uses similar 
underlying techniques to the disruptor

More for the curious: 
http://nickzeeb.wordpress.com/2013/03/07/the-coalescing-ring-buffer/

"It is a component that we have written in Java to efficiently buffer messages 
between a producer and a consumer thread where only the latest value for a 
given topic is of interest. All other messages can be discarded immediately."

> More-efficient ExecutorService for improved throughput
> --
>
> Key: CASSANDRA-4718
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jonathan Ellis
>Priority: Minor
> Attachments: baq vs trunk.png, PerThreadQueue.java
>
>
> Currently all our execution stages dequeue tasks one at a time.  This can 
> result in contention between producers and consumers (although we do our best 
> to minimize this by using LinkedBlockingQueue).
> One approach to mitigating this would be to make consumer threads do more 
> work in "bulk" instead of just one task per dequeue.  (Producer threads tend 
> to be single-task oriented by nature, so I don't see an equivalent 
> opportunity there.)
> BlockingQueue has a drainTo(collection, int) method that would be perfect for 
> this.  However, no ExecutorService in the jdk supports using drainTo, nor 
> could I google one.
> What I would like to do here is create just such a beast and wire it into (at 
> least) the write and read stages.  (Other possible candidates for such an 
> optimization, such as the CommitLog and OutboundTCPConnection, are not 
> ExecutorService-based and will need to be one-offs.)
> AbstractExecutorService may be useful.  The implementations of 
> ICommitLogExecutorService may also be useful. (Despite the name these are not 
> actual ExecutorServices, although they share the most important properties of 
> one.)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

2013-04-11 Thread Darach Ennis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629542#comment-13629542
 ] 

Darach Ennis commented on CASSANDRA-4718:
-

+1 to [~nitsanw]'s observations w.r.t. disruptor and thread affinity, although 
YMMV on Mac OS X w.r.t. affinity.
I would take Java-Thread-Affinity with a pinch of salt on OS X w.r.t claims of 
something equivalent to true affinity. Sounds like disruptor is a good 
candidate here though given [~jbellis]'s added color.

An aside, LMAX have recently open sourced a coalescing ring buffer that uses 
similar underlying techniques to
the disruptor. It's designed for event streams that tend to benefit from 
coalescing or combining in some way. Perhaps this may suit some use cases. If 
not it may still serve as inspiration / color.

https://github.com/LMAX-Exchange/LMAXCollections/tree/master/CoalescingRingBuffer/src/com/lmax/collections/coalescing/ring/buffer


> More-efficient ExecutorService for improved throughput
> --
>
> Key: CASSANDRA-4718
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jonathan Ellis
>Priority: Minor
> Attachments: baq vs trunk.png, PerThreadQueue.java
>
>
> Currently all our execution stages dequeue tasks one at a time.  This can 
> result in contention between producers and consumers (although we do our best 
> to minimize this by using LinkedBlockingQueue).
> One approach to mitigating this would be to make consumer threads do more 
> work in "bulk" instead of just one task per dequeue.  (Producer threads tend 
> to be single-task oriented by nature, so I don't see an equivalent 
> opportunity there.)
> BlockingQueue has a drainTo(collection, int) method that would be perfect for 
> this.  However, no ExecutorService in the jdk supports using drainTo, nor 
> could I google one.
> What I would like to do here is create just such a beast and wire it into (at 
> least) the write and read stages.  (Other possible candidates for such an 
> optimization, such as the CommitLog and OutboundTCPConnection, are not 
> ExecutorService-based and will need to be one-offs.)
> AbstractExecutorService may be useful.  The implementations of 
> ICommitLogExecutorService may also be useful. (Despite the name these are not 
> actual ExecutorServices, although they share the most important properties of 
> one.)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Assigned] (CASSANDRA-5457) Ordering is ignored when using 'CLUSTERING ORDER BY'

2013-04-11 Thread Brandon Williams (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams reassigned CASSANDRA-5457:
---

Assignee: Sylvain Lebresne

> Ordering is ignored when using 'CLUSTERING ORDER BY' 
> -
>
> Key: CASSANDRA-5457
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5457
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.4
>Reporter: Patrick McFadin
>Assignee: Sylvain Lebresne
>
> Creating the following table:
> create table reverse_sort_test (
>   id int,
>   field1 int,
>   field2 int
>   PRIMARY KEY (id, field1, field2)
> ) WITH CLUSTERING ORDER BY (field1 DESC);
> I would expect field1 to be reverse ordered. 
> Inserting this data:
> insert into reverse_sort_test (id,field1,field2) values (1,1,1);
> insert into reverse_sort_test (id,field1,field2) values (3,3,3);
> insert into reverse_sort_test (id,field1,field2) values (2,2,2);
> insert into reverse_sort_test (id,field1,field2) values (4,4,4);
> insert into reverse_sort_test (id,field1,field2) values (6,6,6);
> insert into reverse_sort_test (id,field1,field2) values (5,5,5);
> And running a select:
> select * from reverse_sort_test;
>  id | field1 | field2
> ++
>   5 |  5 |  5
>   1 |  1 |  1
>   2 |  2 |  2
>   4 |  4 |  4
>   6 |  6 |  6
>   3 |  3 |  3
> The order looks random.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5458) Avoid serializing keyspace redundantly in RowMutation

2013-04-11 Thread Jonathan Ellis (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis updated CASSANDRA-5458:
--

Attachment: 5458.txt

> Avoid serializing keyspace redundantly in RowMutation
> -
>
> Key: CASSANDRA-5458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5458
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Jonathan Ellis
>Assignee: Jonathan Ellis
>Priority: Minor
> Fix For: 2.0
>
> Attachments: 5458.txt
>
>
> We can infer the table from the CFID, so there's no need to de/serialize it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-5458) Avoid serializing keyspace redundantly in RowMutation

2013-04-11 Thread Jonathan Ellis (JIRA)
Jonathan Ellis created CASSANDRA-5458:
-

 Summary: Avoid serializing keyspace redundantly in RowMutation
 Key: CASSANDRA-5458
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5458
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
 Fix For: 2.0
 Attachments: 5458.txt

We can infer the table from the CFID, so there's no need to de/serialize it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-5443) Add CAS CQL support

2013-04-11 Thread Rick Shaw (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629462#comment-13629462
 ] 

Rick Shaw edited comment on CASSANDRA-5443 at 4/11/13 10:12 PM:


With regard to the result, it would be consistent with SQL to return a count of 
affected logical rows; not a boolean.

  was (Author: ardot):
With regard to the result, it would be cosistent with SQL to return a count 
of affected logical rows; not a boolean.
  
> Add CAS CQL support
> ---
>
> Key: CASSANDRA-5443
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5443
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
>Assignee: Sylvain Lebresne
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5443) Add CAS CQL support

2013-04-11 Thread Rick Shaw (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629462#comment-13629462
 ] 

Rick Shaw commented on CASSANDRA-5443:
--

With regard to the result, it would be cosistent with SQL to return a count of 
affected logical rows; not a boolean.

> Add CAS CQL support
> ---
>
> Key: CASSANDRA-5443
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5443
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
>Assignee: Sylvain Lebresne
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-2698) Instrument repair to be able to assess it's efficiency (precision)

2013-04-11 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629454#comment-13629454
 ] 

Jonathan Ellis commented on CASSANDRA-2698:
---

Jake has an Eclipse formatter at 
https://github.com/tjake/cassandra-style-eclipse.  Not sure if it's 100% 
accurate but Jake's patches look okay to me so it's probably close. :)

> Instrument repair to be able to assess it's efficiency (precision)
> --
>
> Key: CASSANDRA-2698
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2698
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Sylvain Lebresne
>Assignee: Benedict
>Priority: Minor
>  Labels: lhf
> Attachments: nodetool_repair_and_cfhistogram.tar.gz, 
> patch_2698_v1.txt, patch.diff, patch-rebased.diff
>
>
> Some reports indicate that repair sometime transfer huge amounts of data. One 
> hypothesis is that the merkle tree precision may deteriorate too much at some 
> data size. To check this hypothesis, it would be reasonably to gather 
> statistic during the merkle tree building of how many rows each merkle tree 
> range account for (and the size that this represent). It is probably an 
> interesting statistic to have anyway.   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

2013-04-11 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629447#comment-13629447
 ] 

Jonathan Ellis commented on CASSANDRA-4718:
---

Let me give a little more color as to what our existing stages are.  Most of 
these are ThreadPoolExecutors connected by LinkedBlockingQueue.

A client sends each request to a node in the cluster called the Coordinator.  
The coordinator stages are
# Request: either Thrift or Netty reads the request from the client
# StorageProxy: the coordinator validates the request and decides which 
replicas need to be contacted
# MessagingService (out): the coordinator sends the requests to the appropriate 
replicas
# MessagingService (in): the coordinator reads the reply
# Response: the coordinator processes callbacks for the reply
# StorageProxy: this thread will have been waiting on a Future or a Condition 
for the callbacks, and can now reply to the client

When a replica receives a message, it also goes through a few stages:
# MessagingService (in): the replica reads the coordinator's request
# Read or Write: fetch or append the data specified by the request
# MessagingService (out): the replica sends the result to the coordinator

So the obstacles I see to incorporating Disruptor are

- MessagingService.  This is an exception to the rule in that it is not 
actually a ThreadPoolExecutor; we have a custom thread pool per replica that 
does some gymnastics to keep its queue from growing indefinitely when a replica 
gets behind (CASSANDRA-3005).  MS uses blocking sockets; long ago, we observed 
this to give better performance than NIO.  I'd be willing to evaluate redoing 
this on e.g. Netty, but:
- More generally, requests are not constant-size, which makes disruptor Entry 
re-use difficult
- The read stage is basically forced to be a separate thread pool because of 
blocking i/o from disk
- StorageProxy is not yet asynchronous

Addressing the last of these is straightforward, but the others give me pause.

What I'd like to do is pick part of the system and see if converting that to 
Disruptor gives a big enough win to be worth pursuing with a full-scale 
conversion, but given how Disruptor wants to manage everything I'm not sure how 
to do that either!

> More-efficient ExecutorService for improved throughput
> --
>
> Key: CASSANDRA-4718
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jonathan Ellis
>Priority: Minor
> Attachments: baq vs trunk.png, PerThreadQueue.java
>
>
> Currently all our execution stages dequeue tasks one at a time.  This can 
> result in contention between producers and consumers (although we do our best 
> to minimize this by using LinkedBlockingQueue).
> One approach to mitigating this would be to make consumer threads do more 
> work in "bulk" instead of just one task per dequeue.  (Producer threads tend 
> to be single-task oriented by nature, so I don't see an equivalent 
> opportunity there.)
> BlockingQueue has a drainTo(collection, int) method that would be perfect for 
> this.  However, no ExecutorService in the jdk supports using drainTo, nor 
> could I google one.
> What I would like to do here is create just such a beast and wire it into (at 
> least) the write and read stages.  (Other possible candidates for such an 
> optimization, such as the CommitLog and OutboundTCPConnection, are not 
> ExecutorService-based and will need to be one-offs.)
> AbstractExecutorService may be useful.  The implementations of 
> ICommitLogExecutorService may also be useful. (Despite the name these are not 
> actual ExecutorServices, although they share the most important properties of 
> one.)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-2698) Instrument repair to be able to assess it's efficiency (precision)

2013-04-11 Thread Yuki Morishita (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629448#comment-13629448
 ] 

Yuki Morishita commented on CASSANDRA-2698:
---

Benedict,

bq. Since I need to reinsert all the records once I've decided this anyway, I 
need to retain them all, which I chose to do in EstimatedHistogram as they do, 
...

If you do that, you should create your own class with labels and array since 
you're not using default offsets nor other histogram related methods. It 
confused me at first why you are doing addToIndex to EstimatedHistogram.

But looking at this from the begining again, what we want to see is if we have 
Merkle tree of evenly distributed keys(or rows) in each hash. You can use 
EstimatedHistogram or your own to show that. For now, just use logger to log 
that distribution at the end of Merkle Tree creation with corresponding repair 
session Id is fine, instead of sending stats back to the coordinator.

For the streaming part, it is hard to distinguish which stream session belongs 
to certain repair on current code base(we can only see if it is repair related 
or not by looking at OperationType). So we need to improve that, and I'm 
working on as part of repair and streaming redesign(CASSANDRA-5426, 
CASSANDRA-5286). So, let's focus on the former, validation part.

bq. Do you have an Eclipse formatter profile I could use for your coding 
convention?

Sorry, I use intellij, but I think someone on #cassandra-dev on irc has one.

> Instrument repair to be able to assess it's efficiency (precision)
> --
>
> Key: CASSANDRA-2698
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2698
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Sylvain Lebresne
>Assignee: Benedict
>Priority: Minor
>  Labels: lhf
> Attachments: nodetool_repair_and_cfhistogram.tar.gz, 
> patch_2698_v1.txt, patch.diff, patch-rebased.diff
>
>
> Some reports indicate that repair sometime transfer huge amounts of data. One 
> hypothesis is that the merkle tree precision may deteriorate too much at some 
> data size. To check this hypothesis, it would be reasonably to gather 
> statistic during the merkle tree building of how many rows each merkle tree 
> range account for (and the size that this represent). It is probably an 
> interesting statistic to have anyway.   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-5457) Ordering is ignored when using 'CLUSTERING ORDER BY'

2013-04-11 Thread Patrick McFadin (JIRA)
Patrick McFadin created CASSANDRA-5457:
--

 Summary: Ordering is ignored when using 'CLUSTERING ORDER BY' 
 Key: CASSANDRA-5457
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5457
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.2.4
Reporter: Patrick McFadin


Creating the following table:

create table reverse_sort_test (
id int,
field1 int,
field2 int
PRIMARY KEY (id, field1, field2)
) WITH CLUSTERING ORDER BY (field1 DESC);

I would expect field1 to be reverse ordered. 

Inserting this data:

insert into reverse_sort_test (id,field1,field2) values (1,1,1);
insert into reverse_sort_test (id,field1,field2) values (3,3,3);
insert into reverse_sort_test (id,field1,field2) values (2,2,2);
insert into reverse_sort_test (id,field1,field2) values (4,4,4);
insert into reverse_sort_test (id,field1,field2) values (6,6,6);
insert into reverse_sort_test (id,field1,field2) values (5,5,5);

And running a select:

select * from reverse_sort_test;

 id | field1 | field2
++
  5 |  5 |  5
  1 |  1 |  1
  2 |  2 |  2
  4 |  4 |  4
  6 |  6 |  6
  3 |  3 |  3

The order looks random.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

2013-04-11 Thread Nitsan Wakart (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629415#comment-13629415
 ] 

Nitsan Wakart commented on CASSANDRA-4718:
--

OK, off the bat without knowing anything about nothing (following on from 
twitter chat with Jake Luciani ‏@tjake):
I understand your process to be a pipeline of work being performed in stages 
such that tasks have to complete one stage to go to the next. You need some 
fast queue mechanism to deliver tasks between stages with minimum contention. 
Two approaches come to mind:
1. To build on existing frameworks you will probably want to look at the 
Disruptor (https://github.com/LMAX-Exchange/disruptor) and thread affinity 
(https://github.com/peter-lawrey/Java-Thread-Affinity) this is going to work 
well for rigid processes with an abundance of CPUs to play with, but with 
planning can also work well for more restrictive environments.
2. Go wild and build something new!!! the typical work pool model will have all 
the workers contending for work, this will create contention over the head of 
the queue. We can try for a new algorithm where we have some way of removing 
this contention for example by having an work arbitration thread deliver work 
to the least busy worker or some other crazy scheme. I plan to give it a go and 
see what happens :)
Option 1 is sure to deliver improved performance over your JDK variety of 
collections. Thread affinity will help deliver more predictable results but 
will require some layout algorithm for the available resources(i.e. make a 
different plan for a dual socket 8 core than a 4 socket 4 core machine). 
Affinity is likely to prove important in any case, but I am not aware of any 
public framework making use of it.
An overlooked aspect of the Disruptor is the fact that it is a queue and an 
object pool merged into one. To make the most of it you will need to have your 
pipelined events form some super structure which will cycle through the process 
and then get reused.
In any case, here's an initial brain dump, happy to get involved in the 
solution if this sounds interesting. 

> More-efficient ExecutorService for improved throughput
> --
>
> Key: CASSANDRA-4718
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jonathan Ellis
>Priority: Minor
> Attachments: baq vs trunk.png, PerThreadQueue.java
>
>
> Currently all our execution stages dequeue tasks one at a time.  This can 
> result in contention between producers and consumers (although we do our best 
> to minimize this by using LinkedBlockingQueue).
> One approach to mitigating this would be to make consumer threads do more 
> work in "bulk" instead of just one task per dequeue.  (Producer threads tend 
> to be single-task oriented by nature, so I don't see an equivalent 
> opportunity there.)
> BlockingQueue has a drainTo(collection, int) method that would be perfect for 
> this.  However, no ExecutorService in the jdk supports using drainTo, nor 
> could I google one.
> What I would like to do here is create just such a beast and wire it into (at 
> least) the write and read stages.  (Other possible candidates for such an 
> optimization, such as the CommitLog and OutboundTCPConnection, are not 
> ExecutorService-based and will need to be one-offs.)
> AbstractExecutorService may be useful.  The implementations of 
> ICommitLogExecutorService may also be useful. (Despite the name these are not 
> actual ExecutorServices, although they share the most important properties of 
> one.)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

2013-04-11 Thread T Jake Luciani (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4718?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629390#comment-13629390
 ] 

T Jake Luciani commented on CASSANDRA-4718:
---

[~stuhood] mentions 
http://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ForkJoinPool.html
which is only in java7 but that's required for 2.0 no?

> More-efficient ExecutorService for improved throughput
> --
>
> Key: CASSANDRA-4718
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jonathan Ellis
>Priority: Minor
> Attachments: baq vs trunk.png, PerThreadQueue.java
>
>
> Currently all our execution stages dequeue tasks one at a time.  This can 
> result in contention between producers and consumers (although we do our best 
> to minimize this by using LinkedBlockingQueue).
> One approach to mitigating this would be to make consumer threads do more 
> work in "bulk" instead of just one task per dequeue.  (Producer threads tend 
> to be single-task oriented by nature, so I don't see an equivalent 
> opportunity there.)
> BlockingQueue has a drainTo(collection, int) method that would be perfect for 
> this.  However, no ExecutorService in the jdk supports using drainTo, nor 
> could I google one.
> What I would like to do here is create just such a beast and wire it into (at 
> least) the write and read stages.  (Other possible candidates for such an 
> optimization, such as the CommitLog and OutboundTCPConnection, are not 
> ExecutorService-based and will need to be one-offs.)
> AbstractExecutorService may be useful.  The implementations of 
> ICommitLogExecutorService may also be useful. (Despite the name these are not 
> actual ExecutorServices, although they share the most important properties of 
> one.)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4905) Repair should exclude gcable tombstones from merkle-tree computation

2013-04-11 Thread Christian Spriegel (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629363#comment-13629363
 ] 

Christian Spriegel commented on CASSANDRA-4905:
---

Yeah, repair with TTLed columns can be nasty. Since November, I've seen repairs 
streaming up to 90GB of data for a single repair. According to nodetool, this 
cluster had no dropped writes. So I would assume it was consistent already.

Before, Sun Dec 23 08:00:01 UTC 2012:
192.168.1.1 datacenter1 rack1   Up Normal  404.17 GB   33.33%  
0   
192.168.1.2 datacenter1 rack1   Up Normal  410.9 GB33.33%  
56713727820156410577229101238628035242  
192.168.1.3 datacenter1 rack1   Up Normal  404.27 GB   33.33%  
113427455640312821154458202477256070484

After, Sun Dec 23 12:19:38 UTC 2012:
192.168.1.1 datacenter1 rack1   Up Normal  497.95 GB   33.33%  
0   
192.168.1.2 datacenter1 rack1   Up Normal  413.26 GB   33.33%  
56713727820156410577229101238628035242  
192.168.1.3 datacenter1 rack1   Up Normal  449.83 GB   33.33%  
113427455640312821154458202477256070484 

I'm not saying I want this patch in 1.1. I just wanted to share this rather 
spectecular repair :-)

> Repair should exclude gcable tombstones from merkle-tree computation
> 
>
> Key: CASSANDRA-4905
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4905
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Christian Spriegel
>Assignee: Sylvain Lebresne
> Fix For: 1.2.0 beta 3
>
> Attachments: 4905.txt
>
>
> Currently gcable tombstones get repaired if some replicas compacted already, 
> but some are not compacted.
> This could be avoided by ignoring all gcable tombstones during merkle tree 
> calculation.
> This was discussed with Sylvain on the mailing list:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5230) cql3 doesn't support multiple clauses on primary key components

2013-04-11 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5230?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629306#comment-13629306
 ] 

Aleksey Yeschenko commented on CASSANDRA-5230:
--

bq. Is this the expected behavior?

Yes, only the last part of the key can be restricted by IN. In your case it's 
'd'.

> cql3 doesn't support multiple clauses on primary key components
> ---
>
> Key: CASSANDRA-5230
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5230
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.0
>Reporter: Brandon Williams
>Assignee: Sylvain Lebresne
> Fix For: 1.2.2
>
> Attachments: 5230.patch
>
>
> In trying to write a dtest for CASSANDRA-5225, I noticed that given a table 
> such as:
> {noformat}
> CREATE TABLE foo (
>   key text,
>   c text,
>   v text,
>   PRIMARY KEY (key, c)
> )
> {noformat}
> It is possible to slice the values of 1 or 2 for c:
> {noformat}
> select c from foo where key = 'foo' and c > '0' and c < '3';
> {noformat}
> However, there is no way to get these explicitly by name, even though it 
> should be possible:
> {noformat}
> cqlsh:Keyspace1> select c from foo where key = 'foo' and c in ('1', '2');
> Bad Request: PRIMARY KEY part c cannot be restricted by IN relation
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5456) Large number of bootstrapping nodes cause gossip to stop working

2013-04-11 Thread Brandon Williams (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-5456:


Reviewer: brandon.williams

> Large number of bootstrapping nodes cause gossip to stop working
> 
>
> Key: CASSANDRA-5456
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5456
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.10
>Reporter: Oleg Kibirev
> Attachments: PendingRangeCalculatorService.patch
>
>
> Long running section of code in PendingRangeCalculatorService is synchronized 
> on bootstrapTokens. This causes gossip to stop working as it waits for the 
> same lock when a large number of nodes (hundreds in our case) are 
> bootstrapping. Consequently, the whole cluster becomes non-functional. 
> I experimented with the following change in 
> PendingRangeCalculatorService.java and it resolved the problem in our case. 
> Prior code had synchronized around the for loop.
> synchronized(bootstrapTokens) {
> bootstrapTokens = new LinkedHashMap(bootstrapTokens);
> }
> for (Map.Entry entry : bootstrapTokens.entrySet())
> {
>InetAddress endpoint = entry.getValue();
>allLeftMetadata.updateNormalToken(entry.getKey(), endpoint);
>for (Range range : 
> strategy.getAddressRanges(allLeftMetadata).get(endpoint))
>pendingRanges.put(range, endpoint);
>allLeftMetadata.removeEndpoint(endpoint);
> }
>  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4860) Estimated Row Cache Entry size incorrect (always 24?)

2013-04-11 Thread Ryan McGuire (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629297#comment-13629297
 ] 

Ryan McGuire commented on CASSANDRA-4860:
-

I'm hitting the same key multiple times in the write, but not the read:

stress -F 200 -n 2000 -i 1
stress -n 200 -o read -i 1


> Estimated Row Cache Entry size incorrect (always 24?)
> -
>
> Key: CASSANDRA-4860
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4860
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.0, 1.2.3, 2.0
>Reporter: Chris Burroughs
>Assignee: Vijay
> Fix For: 1.2.0 beta 3
>
> Attachments: 0001-4860-v2.patch, 0001-4860-v3.patch, 
> 0001-CASSANDRA-4860-for-11.patch, 0001-CASSANDRA-4860.patch, 
> 4860-perf-test.zip, trunk-4860-revert.patch
>
>
> After running for several hours the RowCacheSize was suspicious low (ie 70 
> something MB)  I used  CASSANDRA-4859 to measure the size and number of 
> entries on a node:
> In [3]: 1560504./65021
> Out[3]: 24.0
> In [4]: 2149464./89561
> Out[4]: 24.0
> In [6]: 7216096./300785
> Out[6]: 23.990877204647838
> That's RowCacheSize/RowCacheNumEntires  .  Just to prove I don't have crazy 
> small rows the mean size of the row *keys* in the saved cache is 67 and 
> Compacted row mean size: 355.  No jamm errors in the log
> Config notes:
> row_cache_provider: ConcurrentLinkedHashCacheProvider
> row_cache_size_in_mb: 2048
> Version info:
>  * C*: 1.1.6
>  * centos 2.6.32-220.13.1.el6.x86_64
>  * java 6u31 Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5456) Large number of bootstrapping nodes cause gossip to stop working

2013-04-11 Thread Oleg Kibirev (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Oleg Kibirev updated CASSANDRA-5456:


Attachment: PendingRangeCalculatorService.patch

Making a copy of bootstrapTokens before a time consuming loop rather than 
holding a synchronized lock for the whole duration

> Large number of bootstrapping nodes cause gossip to stop working
> 
>
> Key: CASSANDRA-5456
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5456
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.10
>Reporter: Oleg Kibirev
> Attachments: PendingRangeCalculatorService.patch
>
>
> Long running section of code in PendingRangeCalculatorService is synchronized 
> on bootstrapTokens. This causes gossip to stop working as it waits for the 
> same lock when a large number of nodes (hundreds in our case) are 
> bootstrapping. Consequently, the whole cluster becomes non-functional. 
> I experimented with the following change in 
> PendingRangeCalculatorService.java and it resolved the problem in our case. 
> Prior code had synchronized around the for loop.
> synchronized(bootstrapTokens) {
> bootstrapTokens = new LinkedHashMap(bootstrapTokens);
> }
> for (Map.Entry entry : bootstrapTokens.entrySet())
> {
>InetAddress endpoint = entry.getValue();
>allLeftMetadata.updateNormalToken(entry.getKey(), endpoint);
>for (Range range : 
> strategy.getAddressRanges(allLeftMetadata).get(endpoint))
>pendingRanges.put(range, endpoint);
>allLeftMetadata.removeEndpoint(endpoint);
> }
>  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Issue Comment Deleted] (CASSANDRA-5456) Large number of bootstrapping nodes cause gossip to stop working

2013-04-11 Thread Oleg Kibirev (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Oleg Kibirev updated CASSANDRA-5456:


Comment: was deleted

(was: Copying bootstrapTokens rather than holding a lock on the same for entire 
loop)

> Large number of bootstrapping nodes cause gossip to stop working
> 
>
> Key: CASSANDRA-5456
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5456
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.10
>Reporter: Oleg Kibirev
>
> Long running section of code in PendingRangeCalculatorService is synchronized 
> on bootstrapTokens. This causes gossip to stop working as it waits for the 
> same lock when a large number of nodes (hundreds in our case) are 
> bootstrapping. Consequently, the whole cluster becomes non-functional. 
> I experimented with the following change in 
> PendingRangeCalculatorService.java and it resolved the problem in our case. 
> Prior code had synchronized around the for loop.
> synchronized(bootstrapTokens) {
> bootstrapTokens = new LinkedHashMap(bootstrapTokens);
> }
> for (Map.Entry entry : bootstrapTokens.entrySet())
> {
>InetAddress endpoint = entry.getValue();
>allLeftMetadata.updateNormalToken(entry.getKey(), endpoint);
>for (Range range : 
> strategy.getAddressRanges(allLeftMetadata).get(endpoint))
>pendingRanges.put(range, endpoint);
>allLeftMetadata.removeEndpoint(endpoint);
> }
>  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4905) Repair should exclude gcable tombstones from merkle-tree computation

2013-04-11 Thread Michael Theroux (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629279#comment-13629279
 ] 

Michael Theroux edited comment on CASSANDRA-4905 at 4/11/13 8:04 PM:
-

I believe we are hitting a situation where this bug is being problematic in 
1.1.9.  We have a column family, for historical reasons, we run staggered major 
compactions on.  This column family also has many deletes.  We've noticed our 
bloom filters increasing in size by an amount over time.  Bloom filters on a 
specific node would go down a great deal after a major compaction, only to 
increase back to near their original level over a few days.

What I believe is happening is we had a staggered repair schedule, along with a 
staggered major compaction schedule.  The major compaction would remove the 
tombstones, but the repair would stream them back.

To test the theory, I adjusted the major compaction schedule to perform a major 
compaction across all nodes on the same day.  This weeks behavior and bloom 
filter growth has been much better.

Is there a reason why this patch was not applied to 1.1.X?  Are there stability 
concerns?  We aren't ready to make the jump to 1.2, and would prefer not to 
move this table to Leveled Compaction if we don't have to.

  was (Author: mtheroux2):
I believe we are hitting a situation where this bug is being problematic in 
1.1.9.  We have a column family, for historical reasons, we run staggered major 
compactions on.  This column family also has many deletes.  We've noticed our 
bloom filters increasing in size by an amount over time.  Bloom filters on a 
specific node would go down a great deal after a major compaction, only to 
increase back to near their original level over a few days.

What I believe is happening is we had a staggered repair schedule, along with a 
staggered major compaction schedule.  The major compaction would remove the 
tombstones, but the repair would stream them back.

To test the theory, I adjusted the major compaction schedule to perform a major 
compaction across all nodes on the same day.  This weeks behavior and bloom 
filter growth has been much better.

Is there a reason why this patch was not applied to 1.1.X?  Are their stability 
concerns?  We aren't ready to make the jump to 1.2, and would prefer not to 
move this table to Leveled Compaction if we don't have to.
  
> Repair should exclude gcable tombstones from merkle-tree computation
> 
>
> Key: CASSANDRA-4905
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4905
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Christian Spriegel
>Assignee: Sylvain Lebresne
> Fix For: 1.2.0 beta 3
>
> Attachments: 4905.txt
>
>
> Currently gcable tombstones get repaired if some replicas compacted already, 
> but some are not compacted.
> This could be avoided by ignoring all gcable tombstones during merkle tree 
> calculation.
> This was discussed with Sylvain on the mailing list:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4905) Repair should exclude gcable tombstones from merkle-tree computation

2013-04-11 Thread Michael Theroux (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629279#comment-13629279
 ] 

Michael Theroux commented on CASSANDRA-4905:


I believe we are hitting a situation where this bug is being problematic in 
1.1.9.  We have a column family, for historical reasons, we run staggered major 
compactions on.  This column family also has many deletes.  We've noticed our 
bloom filters increasing in size by an amount over time.  Bloom filters on a 
specific node would go down a great deal after a major compaction, only to 
increase back to near their original level over a few days.

What I believe is happening is we had a staggered repair schedule, along with a 
staggered major compaction schedule.  The major compaction would remove the 
tombstones, but the repair would stream them back.

To test the theory, I adjusted the major compaction schedule to perform a major 
compaction across all nodes on the same day.  This weeks behavior and bloom 
filter growth has been much better.

Is there a reason why this patch was not applied to 1.1.X?  Are their stability 
concerns?  We aren't ready to make the jump to 1.2, and would prefer not to 
move this table to Leveled Compaction if we don't have to.

> Repair should exclude gcable tombstones from merkle-tree computation
> 
>
> Key: CASSANDRA-4905
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4905
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Christian Spriegel
>Assignee: Sylvain Lebresne
> Fix For: 1.2.0 beta 3
>
> Attachments: 4905.txt
>
>
> Currently gcable tombstones get repaired if some replicas compacted already, 
> but some are not compacted.
> This could be avoided by ignoring all gcable tombstones during merkle tree 
> calculation.
> This was discussed with Sylvain on the mailing list:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5230) cql3 doesn't support multiple clauses on primary key components

2013-04-11 Thread Olivier LHeureux (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5230?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629262#comment-13629262
 ] 

Olivier LHeureux commented on CASSANDRA-5230:
-

If I add another key like:

CREATE TABLE foo (
  key text,
  c text,
  d text,
  v text,
  PRIMARY KEY (key, c, d)
);

select c from foo where key = 'foo' and c in ('1', '2') will still result in a 
Bad Request error.

Is this the expected behavior?

> cql3 doesn't support multiple clauses on primary key components
> ---
>
> Key: CASSANDRA-5230
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5230
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.0
>Reporter: Brandon Williams
>Assignee: Sylvain Lebresne
> Fix For: 1.2.2
>
> Attachments: 5230.patch
>
>
> In trying to write a dtest for CASSANDRA-5225, I noticed that given a table 
> such as:
> {noformat}
> CREATE TABLE foo (
>   key text,
>   c text,
>   v text,
>   PRIMARY KEY (key, c)
> )
> {noformat}
> It is possible to slice the values of 1 or 2 for c:
> {noformat}
> select c from foo where key = 'foo' and c > '0' and c < '3';
> {noformat}
> However, there is no way to get these explicitly by name, even though it 
> should be possible:
> {noformat}
> cqlsh:Keyspace1> select c from foo where key = 'foo' and c in ('1', '2');
> Bad Request: PRIMARY KEY part c cannot be restricted by IN relation
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-5456) Large number of bootstrapping nodes cause gossip to stop working

2013-04-11 Thread Oleg Kibirev (JIRA)
Oleg Kibirev created CASSANDRA-5456:
---

 Summary: Large number of bootstrapping nodes cause gossip to stop 
working
 Key: CASSANDRA-5456
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5456
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.1.10
Reporter: Oleg Kibirev


Long running section of code in PendingRangeCalculatorService is synchronized 
on bootstrapTokens. This causes gossip to stop working as it waits for the same 
lock when a large number of nodes (hundreds in our case) are bootstrapping. 
Consequently, the whole cluster becomes non-functional. 

I experimented with the following change in PendingRangeCalculatorService.java 
and it resolved the problem in our case. Prior code had synchronized around the 
for loop.

synchronized(bootstrapTokens) {
bootstrapTokens = new LinkedHashMap(bootstrapTokens);
}

for (Map.Entry entry : bootstrapTokens.entrySet())
{
   InetAddress endpoint = entry.getValue();

   allLeftMetadata.updateNormalToken(entry.getKey(), endpoint);
   for (Range range : 
strategy.getAddressRanges(allLeftMetadata).get(endpoint))
   pendingRanges.put(range, endpoint);
   allLeftMetadata.removeEndpoint(endpoint);
}
 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Assigned] (CASSANDRA-5443) Add CAS CQL support

2013-04-11 Thread Sylvain Lebresne (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5443?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne reassigned CASSANDRA-5443:
---

Assignee: Sylvain Lebresne

> Add CAS CQL support
> ---
>
> Key: CASSANDRA-5443
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5443
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
>Assignee: Sylvain Lebresne
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5443) Add CAS CQL support

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629111#comment-13629111
 ] 

Sylvain Lebresne commented on CASSANDRA-5443:
-

bq. ATOMICALLY is redundant with the IF, isn't it?

It is, strictly speaking. But adding it nails down the fact that this is a 
relatively specific operation, which fairly different performance 
characteristics, and one that returns a value. Also, I figured we may use 
something like "atomic updates" to describe them in the docs/presentations and 
so having atomic in the syntax would avoid having to add every time "that's the 
ones with a IF". Overall, I have the feeling that having a clearer syntactic 
differentiation is probably good.

bq. I do have a stronger preference for IF being the last clause.

I'm perfectly fine with that.

> Add CAS CQL support
> ---
>
> Key: CASSANDRA-5443
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5443
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5435) Support range tombstones from thrift

2013-04-11 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629104#comment-13629104
 ] 

Aleksey Yeschenko commented on CASSANDRA-5435:
--

cassandra-dtest is more of a personal preference. You could place this test 
into CassandraServerTest instead.

> Support range tombstones from thrift
> 
>
> Key: CASSANDRA-5435
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5435
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API
>Reporter: Edward Capriolo
>Assignee: Edward Capriolo
>Priority: Minor
>
> I see a RangeTomstone test and methods in row mutation. However thrift's 
> validate method throws exception when Deletion's have a slice predicate. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5435) Support range tombstones from thrift

2013-04-11 Thread Edward Capriolo (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629102#comment-13629102
 ] 

Edward Capriolo commented on CASSANDRA-5435:


As for not caring about thrift. ROFL don't get me started. 

> Support range tombstones from thrift
> 
>
> Key: CASSANDRA-5435
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5435
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API
>Reporter: Edward Capriolo
>Assignee: Edward Capriolo
>Priority: Minor
>
> I see a RangeTomstone test and methods in row mutation. However thrift's 
> validate method throws exception when Deletion's have a slice predicate. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5443) Add CAS CQL support

2013-04-11 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629100#comment-13629100
 ] 

Jonathan Ellis commented on CASSANDRA-5443:
---

ATOMICALLY is redundant with the IF, isn't it?  I'd prefer leaving it out but I 
guess I can live with it.

I do have a stronger preference for IF being the last clause.  That leaves the 
mutation (UPDATE ... WHERE) more separate from the condition instead of mixed 
together.  (Yes, they do overlap at the WHERE still.)

Let's ignore batches for now.

+1 for DELETE.

> Add CAS CQL support
> ---
>
> Key: CASSANDRA-5443
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5443
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5435) Support range tombstones from thrift

2013-04-11 Thread Edward Capriolo (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629092#comment-13629092
 ] 

Edward Capriolo commented on CASSANDRA-5435:


Right, the first test was more documentation. It shows that we no longer 
through an exception on this condition, but it does not have a great need.  As 
for cassandra-dtest. I do not mind moving the test there, I assumed doing the 
change and providing in-project tests was the way to go??? I will add the super 
column support. I was not sure if that was handled by the first case. 

> Support range tombstones from thrift
> 
>
> Key: CASSANDRA-5435
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5435
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API
>Reporter: Edward Capriolo
>Assignee: Edward Capriolo
>Priority: Minor
>
> I see a RangeTomstone test and methods in row mutation. However thrift's 
> validate method throws exception when Deletion's have a slice predicate. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


buildbot success in ASF Buildbot on cassandra-trunk

2013-04-11 Thread buildbot
The Buildbot has detected a restored build on builder cassandra-trunk while 
building cassandra.
Full details are available at:
 http://ci.apache.org/builders/cassandra-trunk/builds/2563

Buildbot URL: http://ci.apache.org/

Buildslave for this Build: portunus_ubuntu

Build Reason: scheduler
Build Source Stamp: [branch trunk] eba27a641341878c08180c88814e33287ca8b05b
Blamelist: Yuki Morishita 

Build succeeded!

sincerely,
 -The Buildbot





git commit: remove PBSPredictor patch by jbellis; reviewed by marcuse for CASSANDRA-5455

2013-04-11 Thread jbellis
Updated Branches:
  refs/heads/trunk eba27a641 -> 0c7141ee8


remove PBSPredictor
patch by jbellis; reviewed by marcuse for CASSANDRA-5455


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

Branch: refs/heads/trunk
Commit: 0c7141ee8afa05d46680ec22635a5bf41f827b75
Parents: eba27a6
Author: Jonathan Ellis 
Authored: Thu Apr 11 11:29:33 2013 -0500
Committer: Jonathan Ellis 
Committed: Thu Apr 11 11:29:41 2013 -0500

--
 CHANGES.txt|1 +
 build.xml  |5 -
 .../org/apache/cassandra/net/MessagingService.java |   24 -
 .../cassandra/service/PBSPredictionResult.java |  127 ---
 .../org/apache/cassandra/service/PBSPredictor.java |  630 ---
 .../cassandra/service/PBSPredictorMBean.java   |   35 -
 .../apache/cassandra/service/StorageService.java   |2 -
 src/java/org/apache/cassandra/tools/NodeCmd.java   |   59 --
 src/java/org/apache/cassandra/tools/NodeProbe.java |8 -
 .../apache/cassandra/service/PBSPredictorTest.java |  114 ---
 10 files changed, 1 insertions(+), 1004 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c7141ee/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 2306887..48b91da 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0
+ * removed PBSPredictor (CASSANDRA-5455)
  * CAS support (CASSANDRA-5062, )
  * Leveled compaction performs size-tiered compactions in L0 (CASSANDRA-5371)
  * Add yaml network topology snitch for mixed ec2/other envs (CASSANDRA-5339)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c7141ee/build.xml
--
diff --git a/build.xml b/build.xml
index 86178c3..8889794 100644
--- a/build.xml
+++ b/build.xml
@@ -1163,11 +1163,6 @@
 
   
 
-  
-
-  
-
   
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c7141ee/src/java/org/apache/cassandra/net/MessagingService.java
--
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java 
b/src/java/org/apache/cassandra/net/MessagingService.java
index d9688a9..c23f566 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -591,16 +591,6 @@ public final class MessagingService implements 
MessagingServiceMBean
 public int sendRR(MessageOut message, InetAddress to, IAsyncCallback cb, 
long timeout)
 {
 int id = addCallback(cb, message, to, timeout);
-
-if (cb instanceof AbstractWriteResponseHandler)
-{
-PBSPredictor.instance().startWriteOperation(id);
-}
-else if (cb instanceof ReadCallback)
-{
-PBSPredictor.instance().startReadOperation(id);
-}
-
 sendOneWay(message, id, to);
 return id;
 }
@@ -740,20 +730,6 @@ public final class MessagingService implements 
MessagingServiceMBean
 ExecutorService stage = 
StageManager.getStage(message.getMessageType());
 assert stage != null : "No stage for message type " + message.verb;
 
-if (message.verb == Verb.REQUEST_RESPONSE && 
PBSPredictor.instance().isLoggingEnabled())
-{
-IAsyncCallback cb = 
MessagingService.instance().getRegisteredCallback(id).callback;
-
-if (cb instanceof AbstractWriteResponseHandler)
-{
-PBSPredictor.instance().logWriteResponse(id, timestamp);
-}
-else if (cb instanceof ReadCallback)
-{
-PBSPredictor.instance().logReadResponse(id, timestamp);
-}
-}
-
 stage.execute(runnable);
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c7141ee/src/java/org/apache/cassandra/service/PBSPredictionResult.java
--
diff --git a/src/java/org/apache/cassandra/service/PBSPredictionResult.java 
b/src/java/org/apache/cassandra/service/PBSPredictionResult.java
deleted file mode 100644
index 92c5491..000
--- a/src/java/org/apache/cassandra/service/PBSPredictionResult.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * 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 no

[jira] [Commented] (CASSANDRA-5443) Add CAS CQL support

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629075#comment-13629075
 ] 

Sylvain Lebresne commented on CASSANDRA-5443:
-

So, here's my suggestion for a syntax:
{noformat}
UPDATE foo ATOMICALLY SET x = 3, y = 5, z = 'bar' IF x = 4, y = null WHERE k = 
'mykey'
{noformat}

This would return a result set, and that result set might look like:
{noformat}
result
--
  true
{noformat}
where 'result' might be replaced by something more appropriate if someone has 
an idea.

There is the question of whether we'd want to allow that in batches (in which 
case we would need to number the results or something), but I wonder if it's 
really worth bothering with that.

Also, I think we might want to support a DELETE version, because otherwise I 
don't think one can delete a row conditionally. So something like:
{noformat}
DELETE ATOMICALLY FROM foo IF x = 3 WHERE k = 'key';
DELETE ATOMICALLY y, z FROM foo IF x = 3 WHERE k = 'key';
{noformat}


> Add CAS CQL support
> ---
>
> Key: CASSANDRA-5443
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5443
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5455) Remove PBSPredictor

2013-04-11 Thread Marcus Eriksson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5455?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629073#comment-13629073
 ] 

Marcus Eriksson commented on CASSANDRA-5455:


there is an ant target "pbs-test" that should go away as well

lgtm other than that

> Remove PBSPredictor
> ---
>
> Key: CASSANDRA-5455
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5455
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Jonathan Ellis
>Assignee: Jonathan Ellis
> Fix For: 2.0
>
> Attachments: 5455.txt
>
>
> It was a fun experiment, but it's unmaintained and the bar to understanding 
> what is going on is high.  Case in point: PBSTest has been failing 
> intermittently for some time now, possibly even since it was created.  Or 
> possibly not and it was a regression from a refactoring we did.  Who knows?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5435) Support range tombstones from thrift

2013-04-11 Thread Aleksey Yeschenko (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629074#comment-13629074
 ] 

Aleksey Yeschenko commented on CASSANDRA-5435:
--

I don't care deeply about thrift or supercolumns, but, if we are going to add 
this, this should be done right: when super_column is set, slice range should 
refer to the subcolumns - it's trivial to implement with composites.

The test in the first commit is sort of useless. As for the test from the 
second commit - I'd rather see it in cassandra-dtest. Also should add a test 
for non-null super_column+slice range.

> Support range tombstones from thrift
> 
>
> Key: CASSANDRA-5435
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5435
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API
>Reporter: Edward Capriolo
>Assignee: Edward Capriolo
>Priority: Minor
>
> I see a RangeTomstone test and methods in row mutation. However thrift's 
> validate method throws exception when Deletion's have a slice predicate. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


git commit: fix merge failure

2013-04-11 Thread yukim
Updated Branches:
  refs/heads/trunk 0aaf67a74 -> eba27a641


fix merge failure


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

Branch: refs/heads/trunk
Commit: eba27a641341878c08180c88814e33287ca8b05b
Parents: 0aaf67a
Author: Yuki Morishita 
Authored: Thu Apr 11 11:23:08 2013 -0500
Committer: Yuki Morishita 
Committed: Thu Apr 11 11:23:08 2013 -0500

--
 src/java/org/apache/cassandra/db/ColumnIndex.java  |1 -
 .../cassandra/streaming/StreamingTransferTest.java |4 ++--
 2 files changed, 2 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/eba27a64/src/java/org/apache/cassandra/db/ColumnIndex.java
--
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java 
b/src/java/org/apache/cassandra/db/ColumnIndex.java
index daaac63..b523814 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -58,7 +58,6 @@ public class ColumnIndex
 
 public Builder(ColumnFamily cf,
ByteBuffer key,
-   DataOutput output)
DataOutput output,
boolean fromStream)
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/eba27a64/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
--
diff --git 
a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java 
b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 335f98c..e3d9b8a 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -146,8 +146,8 @@ public class StreamingTransferTest extends SchemaLoader
 String key = "key1";
 RowMutation rm = new RowMutation(ks, ByteBufferUtil.bytes(key));
 // add columns of size slightly less than column_index_size to force 
insert column index
-rm.add(new QueryPath(cfname, null, ByteBufferUtil.bytes(1)), 
ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize() - 64]), 2);
-rm.add(new QueryPath(cfname, null, ByteBufferUtil.bytes(6)), 
ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize()]), 2);
+rm.add(cfname, ByteBufferUtil.bytes(1), ByteBuffer.wrap(new 
byte[DatabaseDescriptor.getColumnIndexSize() - 64]), 2);
+rm.add(cfname, ByteBufferUtil.bytes(6), ByteBuffer.wrap(new 
byte[DatabaseDescriptor.getColumnIndexSize()]), 2);
 ColumnFamily cf = rm.addOrGet(cfname);
 // add RangeTombstones
 cf.delete(new DeletionInfo(ByteBufferUtil.bytes(2), 
ByteBufferUtil.bytes(3), cf.getComparator(), 1, (int) 
(System.currentTimeMillis() / 1000)));



[jira] [Updated] (CASSANDRA-5455) Remove PBSPredictor

2013-04-11 Thread Jonathan Ellis (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis updated CASSANDRA-5455:
--

Attachment: 5455.txt

> Remove PBSPredictor
> ---
>
> Key: CASSANDRA-5455
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5455
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Jonathan Ellis
>Assignee: Jonathan Ellis
> Fix For: 2.0
>
> Attachments: 5455.txt
>
>
> It was a fun experiment, but it's unmaintained and the bar to understanding 
> what is going on is high.  Case in point: PBSTest has been failing 
> intermittently for some time now, possibly even since it was created.  Or 
> possibly not and it was a regression from a refactoring we did.  Who knows?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


buildbot failure in ASF Buildbot on cassandra-trunk

2013-04-11 Thread buildbot
The Buildbot has detected a new failure on builder cassandra-trunk while 
building cassandra.
Full details are available at:
 http://ci.apache.org/builders/cassandra-trunk/builds/2561

Buildbot URL: http://ci.apache.org/

Buildslave for this Build: portunus_ubuntu

Build Reason: scheduler
Build Source Stamp: [branch trunk] 8b0e1868e8cf813ddfc98d11448aa2ad363eccc1
Blamelist: Jonathan Ellis 

BUILD FAILED: failed shell

sincerely,
 -The Buildbot





[1/3] git commit: Fix streaming RangeTombstones at column index boundary; patch by slebresne reviewed by yukim for CASSANDRA-5418

2013-04-11 Thread yukim
Updated Branches:
  refs/heads/cassandra-1.2 83ed1cbda -> 0f1fb4340
  refs/heads/trunk 8b0e1868e -> 0aaf67a74


Fix streaming RangeTombstones at column index boundary; patch by slebresne 
reviewed by yukim for CASSANDRA-5418


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

Branch: refs/heads/cassandra-1.2
Commit: 0f1fb4340ca1f6360487c76909883bfedc63e4ce
Parents: 83ed1cb
Author: Yuki Morishita 
Authored: Thu Apr 11 10:57:42 2013 -0500
Committer: Yuki Morishita 
Committed: Thu Apr 11 10:57:42 2013 -0500

--
 CHANGES.txt|3 +-
 src/java/org/apache/cassandra/db/ColumnIndex.java  |   26 +++---
 .../apache/cassandra/io/sstable/SSTableWriter.java |2 +-
 .../cassandra/streaming/StreamingTransferTest.java |   36 +++
 4 files changed, 57 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0f1fb434/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 5c26014..2124b15 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,7 +2,8 @@
  * 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)
+ * Allow empty blob literals in CQL3 (CASSANDRA-5452)
+ * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 Merged from 1.1:
  * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0f1fb434/src/java/org/apache/cassandra/db/ColumnIndex.java
--
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java 
b/src/java/org/apache/cassandra/db/ColumnIndex.java
index bd1c35a..bcd0eef 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -68,13 +68,22 @@ public class ColumnIndex
 public Builder(ColumnFamily cf,
ByteBuffer key,
int estimatedColumnCount,
-   DataOutput output)
+   DataOutput output,
+   boolean fromStream)
 {
 this.indexOffset = rowHeaderSize(key, cf.deletionInfo());
 this.result = new ColumnIndex(estimatedColumnCount);
 this.output = output;
 this.atomSerializer = cf.getOnDiskSerializer();
-this.tombstoneTracker = new 
RangeTombstone.Tracker(cf.getComparator());
+this.tombstoneTracker = fromStream ? null : new 
RangeTombstone.Tracker(cf.getComparator());
+}
+
+public Builder(ColumnFamily cf,
+   ByteBuffer key,
+   int estimatedColumnCount,
+   DataOutput output)
+{
+this(cf, key, estimatedColumnCount, output, false);
 }
 
 /**
@@ -99,7 +108,7 @@ public class ColumnIndex
 
 public int writtenAtomCount()
 {
-return atomCount + tombstoneTracker.writtenAtom();
+return tombstoneTracker == null ? atomCount : atomCount + 
tombstoneTracker.writtenAtom();
 }
 
 /**
@@ -153,11 +162,11 @@ public class ColumnIndex
 {
 firstColumn = column;
 startPosition = endPosition;
-// TODO: have that use the firstColumn as min + make sure we
-// optimize that on read
-endPosition += tombstoneTracker.writeOpenedMarker(firstColumn, 
output, atomSerializer);
+// TODO: have that use the firstColumn as min + make sure we 
optimize that on read
+if (tombstoneTracker != null)
+endPosition += 
tombstoneTracker.writeOpenedMarker(firstColumn, output, atomSerializer);
 blockSize = 0; // We don't count repeated tombstone marker in 
the block size, to avoid a situation
-   // where we wouldn't make any problem because a 
block is filled by said marker
+   // where we wouldn't make any progress because 
a block is filled by said marker
 }
 
 long size = column.serializedSizeForSSTable();
@@ -177,7 +186,8 @@ public class ColumnIndex
 atomSerializer.serializeForSSTable(column, output);
 
 // TODO: Should deal with removing unneeded tombstones
-tombstoneTracker.update(column);
+if (tombstoneTracker != null)
+ 

[3/3] git commit: Merge branch 'cassandra-1.2' into trunk

2013-04-11 Thread yukim
Merge branch 'cassandra-1.2' into trunk

Conflicts:
src/java/org/apache/cassandra/db/ColumnIndex.java
src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java


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

Branch: refs/heads/trunk
Commit: 0aaf67a74f441a8d3c4b54920f35c93f064be4f5
Parents: 8b0e186 0f1fb43
Author: Yuki Morishita 
Authored: Thu Apr 11 11:12:36 2013 -0500
Committer: Yuki Morishita 
Committed: Thu Apr 11 11:12:36 2013 -0500

--
 CHANGES.txt|3 +-
 src/java/org/apache/cassandra/db/ColumnIndex.java  |   24 +++---
 .../apache/cassandra/io/sstable/SSTableWriter.java |2 +-
 .../cassandra/streaming/StreamingTransferTest.java |   36 +++
 4 files changed, 56 insertions(+), 9 deletions(-)
--


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0aaf67a7/src/java/org/apache/cassandra/db/ColumnIndex.java
--
diff --cc src/java/org/apache/cassandra/db/ColumnIndex.java
index e307458,bcd0eef..daaac63
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@@ -58,12 -67,23 +58,21 @@@ public class ColumnInde
  
  public Builder(ColumnFamily cf,
 ByteBuffer key,
 -   int estimatedColumnCount,
 +   DataOutput output)
+DataOutput output,
+boolean fromStream)
  {
  this.indexOffset = rowHeaderSize(key, cf.deletionInfo());
 -this.result = new ColumnIndex(estimatedColumnCount);
 +this.result = new ColumnIndex(new 
ArrayList());
  this.output = output;
- this.tombstoneTracker = new 
RangeTombstone.Tracker(cf.getComparator());
 -this.atomSerializer = cf.getOnDiskSerializer();
+ this.tombstoneTracker = fromStream ? null : new 
RangeTombstone.Tracker(cf.getComparator());
+ }
+ 
+ public Builder(ColumnFamily cf,
+ByteBuffer key,
 -   int estimatedColumnCount,
+DataOutput output)
+ {
 -this(cf, key, estimatedColumnCount, output, false);
++this(cf, key, output, false);
  }
  
  /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0aaf67a7/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
--
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
index 9b6db46,c64fd27..43e3aa5
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@@ -240,13 -236,12 +240,13 @@@ public class SSTableWriter extends SSTa
  // deserialize each column to obtain maxTimestamp and immediately 
serialize it.
  long minTimestamp = Long.MAX_VALUE;
  long maxTimestamp = Long.MIN_VALUE;
 +int maxLocalDeletionTime = Integer.MIN_VALUE;
  StreamingHistogram tombstones = new 
StreamingHistogram(TOMBSTONE_HISTOGRAM_BIN_SIZE);
 -ColumnFamily cf = ColumnFamily.create(metadata, 
ArrayBackedSortedColumns.factory());
 +ColumnFamily cf = ArrayBackedSortedColumns.factory.create(metadata);
  cf.delete(deletionInfo);
  
- ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, 
key.key, dataFile.stream);
 -ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, 
key.key, columnCount, dataFile.stream, true);
 -OnDiskAtom.Serializer atomSerializer = cf.getOnDiskSerializer();
++ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, 
key.key, dataFile.stream, true);
 +OnDiskAtom.Serializer atomSerializer = Column.onDiskSerializer();
  for (int i = 0; i < columnCount; i++)
  {
  // deserialize column with PRESERVE_SIZE because we've written 
the dataSize based on the

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0aaf67a7/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
--
diff --cc test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 46e0e6c,2befe45..335f98c
--- a/test/unit/org/apache/cassandra/streaming/Streaming

[2/3] git commit: Fix streaming RangeTombstones at column index boundary; patch by slebresne reviewed by yukim for CASSANDRA-5418

2013-04-11 Thread yukim
Fix streaming RangeTombstones at column index boundary; patch by slebresne 
reviewed by yukim for CASSANDRA-5418


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

Branch: refs/heads/trunk
Commit: 0f1fb4340ca1f6360487c76909883bfedc63e4ce
Parents: 83ed1cb
Author: Yuki Morishita 
Authored: Thu Apr 11 10:57:42 2013 -0500
Committer: Yuki Morishita 
Committed: Thu Apr 11 10:57:42 2013 -0500

--
 CHANGES.txt|3 +-
 src/java/org/apache/cassandra/db/ColumnIndex.java  |   26 +++---
 .../apache/cassandra/io/sstable/SSTableWriter.java |2 +-
 .../cassandra/streaming/StreamingTransferTest.java |   36 +++
 4 files changed, 57 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0f1fb434/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 5c26014..2124b15 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,7 +2,8 @@
  * 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)
+ * Allow empty blob literals in CQL3 (CASSANDRA-5452)
+ * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418)
 Merged from 1.1:
  * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0f1fb434/src/java/org/apache/cassandra/db/ColumnIndex.java
--
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java 
b/src/java/org/apache/cassandra/db/ColumnIndex.java
index bd1c35a..bcd0eef 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -68,13 +68,22 @@ public class ColumnIndex
 public Builder(ColumnFamily cf,
ByteBuffer key,
int estimatedColumnCount,
-   DataOutput output)
+   DataOutput output,
+   boolean fromStream)
 {
 this.indexOffset = rowHeaderSize(key, cf.deletionInfo());
 this.result = new ColumnIndex(estimatedColumnCount);
 this.output = output;
 this.atomSerializer = cf.getOnDiskSerializer();
-this.tombstoneTracker = new 
RangeTombstone.Tracker(cf.getComparator());
+this.tombstoneTracker = fromStream ? null : new 
RangeTombstone.Tracker(cf.getComparator());
+}
+
+public Builder(ColumnFamily cf,
+   ByteBuffer key,
+   int estimatedColumnCount,
+   DataOutput output)
+{
+this(cf, key, estimatedColumnCount, output, false);
 }
 
 /**
@@ -99,7 +108,7 @@ public class ColumnIndex
 
 public int writtenAtomCount()
 {
-return atomCount + tombstoneTracker.writtenAtom();
+return tombstoneTracker == null ? atomCount : atomCount + 
tombstoneTracker.writtenAtom();
 }
 
 /**
@@ -153,11 +162,11 @@ public class ColumnIndex
 {
 firstColumn = column;
 startPosition = endPosition;
-// TODO: have that use the firstColumn as min + make sure we
-// optimize that on read
-endPosition += tombstoneTracker.writeOpenedMarker(firstColumn, 
output, atomSerializer);
+// TODO: have that use the firstColumn as min + make sure we 
optimize that on read
+if (tombstoneTracker != null)
+endPosition += 
tombstoneTracker.writeOpenedMarker(firstColumn, output, atomSerializer);
 blockSize = 0; // We don't count repeated tombstone marker in 
the block size, to avoid a situation
-   // where we wouldn't make any problem because a 
block is filled by said marker
+   // where we wouldn't make any progress because 
a block is filled by said marker
 }
 
 long size = column.serializedSizeForSSTable();
@@ -177,7 +186,8 @@ public class ColumnIndex
 atomSerializer.serializeForSSTable(column, output);
 
 // TODO: Should deal with removing unneeded tombstones
-tombstoneTracker.update(column);
+if (tombstoneTracker != null)
+tombstoneTracker.update(column);
 
 lastColumn = column;
 }

http://git-wip-us.apache.org/rep

[jira] [Created] (CASSANDRA-5455) Remove PBSPredictor

2013-04-11 Thread Jonathan Ellis (JIRA)
Jonathan Ellis created CASSANDRA-5455:
-

 Summary: Remove PBSPredictor
 Key: CASSANDRA-5455
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5455
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
 Fix For: 2.0


It was a fun experiment, but it's unmaintained and the bar to understanding 
what is going on is high.  Case in point: PBSTest has been failing 
intermittently for some time now, possibly even since it was created.  Or 
possibly not and it was a regression from a refactoring we did.  Who knows?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5062) Support CAS

2013-04-11 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13629047#comment-13629047
 ] 

Jonathan Ellis commented on CASSANDRA-5062:
---

I ended up inlining the old isEmpty and creating a new one for CAS to use.  
Also AbstractSSTableSimpleWriter should be using the "new" isEmpty.  The rest I 
left alone.

SerializationsTest was failing because I'd inadvertently removed the MUTATION 
entry for MS.verbSerializers.  (So MessageIn said, oh, serializer == null?  
Here's an empty Message for you, and the next Message in the test started 
reading partway through the last one.)

Fixed and committed!

> Support CAS
> ---
>
> Key: CASSANDRA-5062
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5062
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
> Attachments: half-baked commit 1.jpg, half-baked commit 2.jpg, 
> half-baked commit 3.jpg
>
>
> "Strong" consistency is not enough to prevent race conditions.  The classic 
> example is user account creation: we want to ensure usernames are unique, so 
> we only want to signal account creation success if nobody else has created 
> the account yet.  But naive read-then-write allows clients to race and both 
> think they have a green light to create.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[4/4] git commit: initial CAS support patch by jbellis and slebresne for CASSANDRA-5062

2013-04-11 Thread jbellis
initial CAS support
patch by jbellis and slebresne for CASSANDRA-5062


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

Branch: refs/heads/trunk
Commit: 8b0e1868e8cf813ddfc98d11448aa2ad363eccc1
Parents: f5ec4c7
Author: Jonathan Ellis 
Authored: Thu Apr 11 10:57:48 2013 -0500
Committer: Jonathan Ellis 
Committed: Thu Apr 11 10:57:48 2013 -0500

--
 CHANGES.txt|1 +
 conf/cassandra.yaml|3 +
 interface/cassandra.thrift |   11 +-
 .../org/apache/cassandra/thrift/Cassandra.java | 6469 +--
 .../cassandra/thrift/cassandraConstants.java   |2 +-
 .../org/apache/cassandra/config/CFMetaData.java|   12 +-
 src/java/org/apache/cassandra/config/Config.java   |2 +
 .../cassandra/config/DatabaseDescriptor.java   |   10 +
 .../org/apache/cassandra/config/KSMetaData.java|1 +
 src/java/org/apache/cassandra/config/Schema.java   |2 +-
 .../org/apache/cassandra/cql3/QueryProcessor.java  |2 +-
 .../cql3/statements/ModificationStatement.java |2 +-
 .../db/AbstractThreadUnsafeSortedColumns.java  |5 -
 .../cassandra/db/ArrayBackedSortedColumns.java |2 +-
 .../apache/cassandra/db/AtomicSortedColumns.java   |5 -
 src/java/org/apache/cassandra/db/ColumnFamily.java |   45 +-
 .../org/apache/cassandra/db/ConsistencyLevel.java  |3 +-
 src/java/org/apache/cassandra/db/DefsTable.java|   12 +-
 src/java/org/apache/cassandra/db/Memtable.java |2 +-
 src/java/org/apache/cassandra/db/Row.java  |8 +
 src/java/org/apache/cassandra/db/RowMutation.java  |8 +-
 src/java/org/apache/cassandra/db/SystemTable.java  |   59 +
 src/java/org/apache/cassandra/db/WriteType.java|3 +-
 .../db/index/composites/CompositesSearcher.java|2 +-
 .../org/apache/cassandra/net/IAsyncCallback.java   |   14 +
 src/java/org/apache/cassandra/net/MessageIn.java   |5 +
 .../org/apache/cassandra/net/MessagingService.java |   31 +-
 .../service/AbstractWriteResponseHandler.java  |   10 -
 .../org/apache/cassandra/service/StorageProxy.java |  173 +-
 .../cassandra/service/StorageProxyMBean.java   |2 +
 .../apache/cassandra/service/StorageService.java   |   21 +-
 .../service/paxos/AbstractPaxosCallback.java   |   45 +
 .../org/apache/cassandra/service/paxos/Commit.java |  130 +
 .../cassandra/service/paxos/CommitVerbHandler.java |   12 +
 .../apache/cassandra/service/paxos/PaxosState.java |  100 +
 .../cassandra/service/paxos/PrepareCallback.java   |   66 +
 .../cassandra/service/paxos/PrepareResponse.java   |   72 +
 .../service/paxos/PrepareVerbHandler.java  |   16 +
 .../cassandra/service/paxos/ProposeCallback.java   |   34 +
 .../service/paxos/ProposeVerbHandler.java  |   17 +
 .../apache/cassandra/thrift/CassandraServer.java   |   99 +-
 .../apache/cassandra/utils/BooleanSerializer.java  |   46 +
 .../org/apache/cassandra/utils/FBUtilities.java|1 +
 src/java/org/apache/cassandra/utils/UUIDGen.java   |   28 +-
 .../org/apache/cassandra/utils/UUIDSerializer.java |4 -
 test/cassandra.in.sh   |2 +-
 test/system/__init__.py|4 +-
 test/system/test_thrift_server.py  |   12 +
 .../apache/cassandra/db/ColumnFamilyStoreTest.java |4 +-
 .../cassandra/db/compaction/CompactionsTest.java   |6 +-
 50 files changed, 5105 insertions(+), 2520 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b0e1868/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 848e843..667c901 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0
+ * CAS support (CASSANDRA-5062, )
  * Leveled compaction performs size-tiered compactions in L0 (CASSANDRA-5371)
  * Add yaml network topology snitch for mixed ec2/other envs (CASSANDRA-5339)
  * Log when a node is down longer than the hint window (CASSANDRA-4554)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b0e1868/conf/cassandra.yaml
--
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index ed61a26..9c1eb11 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -475,6 +475,9 @@ read_request_timeout_in_ms: 1
 range_request_timeout_in_ms: 1
 # How long the coordinator should wait for writes to complete
 write_request_timeout_in_ms: 1
+# how long a coordinator should continue to retry a CAS operation
+# that contends with other proposals for the same r

[1/4] initial CAS support patch by jbellis and slebresne for CASSANDRA-5062

2013-04-11 Thread jbellis
Updated Branches:
  refs/heads/trunk f5ec4c7c3 -> 8b0e1868e


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b0e1868/test/system/test_thrift_server.py
--
diff --git a/test/system/test_thrift_server.py 
b/test/system/test_thrift_server.py
index bcb75e8..8fecd29 100644
--- a/test/system/test_thrift_server.py
+++ b/test/system/test_thrift_server.py
@@ -230,6 +230,18 @@ class TestMutations(ThriftTester):
 assert _big_slice('key1', ColumnParent('Standard2')) == []
 assert _big_slice('key1', ColumnParent('Super1')) == []
 
+def test_cas(self):
+_set_keyspace('Keyspace1')
+assert not client.cas('key1', 'Standard1', _SIMPLE_COLUMNS, 
_SIMPLE_COLUMNS)
+
+assert client.cas('key1', 'Standard1', None, _SIMPLE_COLUMNS)
+
+result = [cosc.column for cosc in _big_slice('key1', 
ColumnParent('Standard1'))]
+# CAS will use its own timestamp, so we can't just compare result == 
_SIMPLE_COLUMNS
+assert dict((c.name, c.value) for c in result) == dict((c.name, 
c.value) for c in _SIMPLE_COLUMNS), result
+
+assert not client.cas('key1', 'Standard1', None, _SIMPLE_COLUMNS)
+
 def test_missing_super(self):
 _set_keyspace('Keyspace1')
 _expect_missing(lambda: client.get('key1', ColumnPath('Super1', 'sc1', 
_i64(1)), ConsistencyLevel.ONE))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b0e1868/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
--
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java 
b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index c986c7b..c11fcf2 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -138,12 +138,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 QueryFilter sliceFilter = 
QueryFilter.getSliceFilter(Util.dk("key1"), "Standard2", 
ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
 ColumnFamily cf = store.getColumnFamily(sliceFilter);
 assert cf.isMarkedForDelete();
-assert cf.isEmpty();
+assert cf.getColumnCount() == 0;
 
 QueryFilter namesFilter = 
QueryFilter.getNamesFilter(Util.dk("key1"), "Standard2", 
ByteBufferUtil.bytes("a"));
 cf = store.getColumnFamily(namesFilter);
 assert cf.isMarkedForDelete();
-assert cf.isEmpty();
+assert cf.getColumnCount() == 0;
 }
 };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b0e1868/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
--
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java 
b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 218b0fa..1f3aef0 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -316,7 +316,7 @@ public class CompactionsTest extends SchemaLoader
 Collection sstablesBefore = cfs.getSSTables();
 
 QueryFilter filter = QueryFilter.getIdentityFilter(key, cfname);
-assert !cfs.getColumnFamily(filter).isEmpty();
+assert !(cfs.getColumnFamily(filter).getColumnCount() == 0);
 
 // Remove key
 rm = new RowMutation(TABLE1, key.key);
@@ -324,7 +324,7 @@ public class CompactionsTest extends SchemaLoader
 rm.apply();
 
 ColumnFamily cf = cfs.getColumnFamily(filter);
-assert cf == null || cf.isEmpty() : "should be empty: " + cf;
+assert cf == null || cf.getColumnCount() == 0 : "should be empty: " + 
cf;
 
 // Sleep one second so that the removal is indeed purgeable even with 
gcgrace == 0
 Thread.sleep(1000);
@@ -340,6 +340,6 @@ public class CompactionsTest extends SchemaLoader
 Util.compact(cfs, toCompact);
 
 cf = cfs.getColumnFamily(filter);
-assert cf == null || cf.isEmpty() : "should be empty: " + cf;
+assert cf == null || cf.getColumnCount() == 0 : "should be empty: " + 
cf;
 }
 }



svn commit: r1466930 - in /cassandra/site: publish/download/index.html publish/index.html src/settings.py

2013-04-11 Thread slebresne
Author: slebresne
Date: Thu Apr 11 15:49:41 2013
New Revision: 1466930

URL: http://svn.apache.org/r1466930
Log:
Update website for 1.2.4 release

Modified:
cassandra/site/publish/download/index.html
cassandra/site/publish/index.html
cassandra/site/src/settings.py

Modified: cassandra/site/publish/download/index.html
URL: 
http://svn.apache.org/viewvc/cassandra/site/publish/download/index.html?rev=1466930&r1=1466929&r2=1466930&view=diff
==
--- cassandra/site/publish/download/index.html (original)
+++ cassandra/site/publish/download/index.html Thu Apr 11 15:49:41 2013
@@ -49,8 +49,8 @@
   Cassandra releases include the core server, the http://wiki.apache.org/cassandra/NodeTool";>nodetool administration 
command-line interface, and a development shell (http://cassandra.apache.org/doc/cql/CQL.html";>cqlsh and the 
old cassandra-cli).
 
   
-  The latest stable release of Apache Cassandra is 1.2.3
-  (released on 2013-03-18).  If you're just
+  The latest stable release of Apache Cassandra is 1.2.4
+  (released on 2013-04-11).  If you're just
   starting out, download this one.
   
 
@@ -59,13 +59,13 @@
   
 
 http://www.apache.org/dyn/closer.cgi?path=/cassandra/1.2.3/apache-cassandra-1.2.3-bin.tar.gz";
+   
href="http://www.apache.org/dyn/closer.cgi?path=/cassandra/1.2.4/apache-cassandra-1.2.4-bin.tar.gz";
onclick="javascript: 
pageTracker._trackPageview('/clicks/binary_download');">
-  apache-cassandra-1.2.3-bin.tar.gz
+  apache-cassandra-1.2.4-bin.tar.gz
 
-[http://www.apache.org/dist/cassandra/1.2.3/apache-cassandra-1.2.3-bin.tar.gz.asc";>PGP]
-[http://www.apache.org/dist/cassandra/1.2.3/apache-cassandra-1.2.3-bin.tar.gz.md5";>MD5]
-[http://www.apache.org/dist/cassandra/1.2.3/apache-cassandra-1.2.3-bin.tar.gz.sha1";>SHA1]
+[http://www.apache.org/dist/cassandra/1.2.4/apache-cassandra-1.2.4-bin.tar.gz.asc";>PGP]
+[http://www.apache.org/dist/cassandra/1.2.4/apache-cassandra-1.2.4-bin.tar.gz.md5";>MD5]
+[http://www.apache.org/dist/cassandra/1.2.4/apache-cassandra-1.2.4-bin.tar.gz.sha1";>SHA1]
 
 
 http://wiki.apache.org/cassandra/DebianPackaging";>Debian 
installation instructions
@@ -153,13 +153,13 @@
   
 
 http://www.apache.org/dyn/closer.cgi?path=/cassandra/1.2.3/apache-cassandra-1.2.3-src.tar.gz";
+   
href="http://www.apache.org/dyn/closer.cgi?path=/cassandra/1.2.4/apache-cassandra-1.2.4-src.tar.gz";
onclick="javascript: 
pageTracker._trackPageview('/clicks/source_download');">
-  apache-cassandra-1.2.3-src.tar.gz
+  apache-cassandra-1.2.4-src.tar.gz
 
-[http://www.apache.org/dist/cassandra/1.2.3/apache-cassandra-1.2.3-src.tar.gz.asc";>PGP]
-[http://www.apache.org/dist/cassandra/1.2.3/apache-cassandra-1.2.3-src.tar.gz.md5";>MD5]
-[http://www.apache.org/dist/cassandra/1.2.3/apache-cassandra-1.2.3-src.tar.gz.sha1";>SHA1]
+[http://www.apache.org/dist/cassandra/1.2.4/apache-cassandra-1.2.4-src.tar.gz.asc";>PGP]
+[http://www.apache.org/dist/cassandra/1.2.4/apache-cassandra-1.2.4-src.tar.gz.md5";>MD5]
+[http://www.apache.org/dist/cassandra/1.2.4/apache-cassandra-1.2.4-src.tar.gz.sha1";>SHA1]
 
   
 

Modified: cassandra/site/publish/index.html
URL: 
http://svn.apache.org/viewvc/cassandra/site/publish/index.html?rev=1466930&r1=1466929&r2=1466930&view=diff
==
--- cassandra/site/publish/index.html (original)
+++ cassandra/site/publish/index.html Thu Apr 11 15:49:41 2013
@@ -75,8 +75,8 @@
   Download
   
 
-The latest release is 1.2.3
-(http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=blob_plain;f=CHANGES.txt;hb=refs/tags/cassandra-1.2.3";>Changes)
+The latest release is 1.2.4
+(http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=blob_plain;f=CHANGES.txt;hb=refs/tags/cassandra-1.2.4";>Changes)
 
 
 Download options

Modified: cassandra/site/src/settings.py
URL: 
http://svn.apache.org/viewvc/cassandra/site/src/settings.py?rev=1466930&r1=1466929&r2=1466930&view=diff
==
--- cassandra/site/src/settings.py (original)
+++ cassandra/site/src/settings.py Thu Apr 11 15:49:41 2013
@@ -98,8 +98,8 @@ class CassandraDef(object):
 veryoldstable_version = '1.0.12'
 veryoldstable_release_date = '2012-10-04'
 veryoldstable_exists = True
-stable_version = '1.2.3'
-stable_release_date = '2013-03-18'
+stable_version = '1.2.4'
+stable_release_date = '2013-04-11'
 devel_version = '1.2.0-rc2'
 devel_release_date = '2012-12-21'
 devel_exists = False




Git Push Summary

2013-04-11 Thread slebresne
Updated Tags:  refs/tags/1.2.4-tentative [deleted] 2e96d0711


Git Push Summary

2013-04-11 Thread slebresne
Updated Tags:  refs/tags/cassandra-1.2.4 [created] 487b6a708


[jira] [Updated] (CASSANDRA-5418) streaming fails

2013-04-11 Thread Yuki Morishita (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yuki Morishita updated CASSANDRA-5418:
--

Attachment: 0001-add-RangeTombstone-transfer-test.patch

So I created unit test to stream RangeTombstones between column index 
boundaries. (Patch attached)
It fails with the same stack trace here on current 1.2 branch, but it passes 
with 5418-v4.
So I will commit v4 and test.

> streaming fails
> ---
>
> Key: CASSANDRA-5418
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5418
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.2, 1.2.3
> Environment: 5 nodes, vnodes enabled, encryption disabled, 
> compression enabled, RackInferring snitch, Centos 6, Oracle JVM with JNA 
> enabled.
>Reporter: Igor Ivanov
>Priority: Critical
> Attachments: 0001-add-RangeTombstone-transfer-test.patch, 
> 5418-1.2.txt, 5418-1.2-v2.txt, 5418-1.2-v3.txt, 5418-v4.txt
>
>
> When I run *nodetool repair* on cas01 node it get's stuck at some point.
> I see following exceptions in cas01 system.log:
> {quote}
> ERROR [Streaming to /10.10.45.60:28] 2013-04-02 09:03:55,353 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.60:28,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>   at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   ... 3 more
> ERROR [Thread-2076] 2013-04-02 09:07:12,261 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-2076,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-3660-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> {quote}
> On other machines there are some exceptions too:
> {quote}
> ERROR [Thread-1424] 2013-04-02 09:07:12,248 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-1424,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-2268-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> ERROR [Streaming to /10.10.45.58:55] 2013-04-02 09:07:12,263 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.58:55,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> o

git commit: remove CompactSerializerTest; it doesn't tell us anything useful

2013-04-11 Thread jbellis
Updated Branches:
  refs/heads/trunk 3fdd46476 -> f5ec4c7c3


remove CompactSerializerTest; it doesn't tell us  anything useful


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

Branch: refs/heads/trunk
Commit: f5ec4c7c3620d453db39d44217cfeca4f775b47b
Parents: 3fdd464
Author: Jonathan Ellis 
Authored: Thu Apr 11 09:32:41 2013 -0500
Committer: Jonathan Ellis 
Committed: Thu Apr 11 09:32:51 2013 -0500

--
 .../apache/cassandra/io/CompactSerializerTest.java |  163 ---
 1 files changed, 0 insertions(+), 163 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f5ec4c7c/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
--
diff --git a/test/unit/org/apache/cassandra/io/CompactSerializerTest.java 
b/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
deleted file mode 100644
index a436702..000
--- a/test/unit/org/apache/cassandra/io/CompactSerializerTest.java
+++ /dev/null
@@ -1,163 +0,0 @@
-package org.apache.cassandra.io;
-/*
- *
- * 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.apache.cassandra.SchemaLoader;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-public class CompactSerializerTest extends SchemaLoader
-{
-private static Set expectedClassNames;
-private static List discoveredClassNames;
-
-@BeforeClass
-public static void scanClasspath()
-{
-expectedClassNames = new HashSet();
-expectedClassNames.add("RangeSliceCommandSerializer");
-expectedClassNames.add("IndexScanCommandSerializer");
-expectedClassNames.add("ReadCommandSerializer");
-expectedClassNames.add("ReadResponseSerializer");
-expectedClassNames.add("RowSerializer");
-expectedClassNames.add("RowMutationSerializer");
-expectedClassNames.add("SliceByNamesReadCommandSerializer");
-expectedClassNames.add("SliceFromReadCommandSerializer");
-expectedClassNames.add("TruncateResponseSerializer");
-expectedClassNames.add("TruncationSerializer");
-expectedClassNames.add("WriteResponseSerializer");
-expectedClassNames.add("EndpointStateSerializer");
-expectedClassNames.add("GossipDigestSerializer");
-expectedClassNames.add("GossipDigestAck2Serializer");
-expectedClassNames.add("GossipDigestAckSerializer");
-expectedClassNames.add("GossipDigestSynSerializer");
-expectedClassNames.add("HeartBeatStateSerializer");
-expectedClassNames.add("VersionedValueSerializer");
-expectedClassNames.add("PendingFileSerializer");
-expectedClassNames.add("StreamHeaderSerializer");
-expectedClassNames.add("FileStatusSerializer");
-expectedClassNames.add("StreamRequestSerializer");
-expectedClassNames.add("CounterMutationSerializer");
-expectedClassNames.add("HashableSerializer");
-expectedClassNames.add("StreamingRepairTaskSerializer");
-expectedClassNames.add("AbstractBoundsSerializer");
-expectedClassNames.add("SnapshotCommandSerializer");
-expectedClassNames.add("RangeSliceReplySerializer");
-expectedClassNames.add("StringSerializer");
-expectedClassNames.add("CallbackDeterminedSerializer");
-expectedClassNames.add("TreeRequestSerializer");
-expectedClassNames.add("ValidatorSerializer");
-expectedClassNames.add("MigrationsSerializer");
-expectedClassNames.add("InnerSerializer");
-expectedClassNames.add("LeafSerializer");
-expectedClassNames.add("MerkleTreeSerializer");
-expectedClassNames.ad

[jira] [Commented] (CASSANDRA-5436) Binary protocol versioning

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628968#comment-13628968
 ] 

Sylvain Lebresne commented on CASSANDRA-5436:
-

bq. sets a version on the incoming message, to be able to serialize for an old 
client

Looks mostly good but in the case an exception is raised, the version won't be 
set correctly. We should set it in the catch block of 
Dispatcher.messageReceived (like we do for StreamId).

bq. the client sends a StartupMessage, and if that has the wrong version, we 
just disconnect

I think we at least send an error message before disconnecting. There is that 
at least :)

bq. would perhaps be nice if the server ignored this version if it is higher 
than what it supports and replies with a Ready/Authenticate-message with 
CURRENT_VERSION

That would work I think.

Though I'll mention another option that would be to ignore the protocol version 
for the OPTIONS message instead, and respond with a SUPPORTED message that have 
the same version than the message received, but that includes which protocol 
versions the server support.  I'm suggesting that because I feel it might be 
simpler for clients if we always respond to their message with the same version 
they sent us, but also because it feels that this kind of 'whats supported' 
business is why the OPTIONS/SUPPORTED messages exists in the first place.

I do note however that it's probably a tad too late for 1.2. So clients of the 
v2 protocol will probably have to handle the case where they get an exception 
on startup and special case to try v1 in that case.

Otherwise, I've forked the protocol spec, so we can document change we make for 
v2 while we make it. I've added a 'Changes since v1' to the v2 spec, so client 
implementors have an easy changelog to upgrade to v2 (without needing to diff 
the spec that is).


> Binary protocol versioning
> --
>
> Key: CASSANDRA-5436
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5436
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Eriksson
> Attachments: 0001-CASSANDRA-5436.patch
>
>
> There should be a way to version the binary protocol, CASSANDRA-5349 adds a 
> message that breaks compatibility (QueryMessage) and the ticket half-handles 
> versioning - it allows old clients to connect and execute queries (by always 
> replying in the same version as the message sent by the client), it does not 
> allow new clients to connect to old servers though.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5417) Push composites support in the storage engine

2013-04-11 Thread T Jake Luciani (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628921#comment-13628921
 ] 

T Jake Luciani commented on CASSANDRA-5417:
---

Sorry still looking at this.  I've been thinking about the names and I think 
isPacked should become isSimple (since simple is the opposite of composite).  I 
see it's used in CType and Composites.

The whole packed and composite thing needs to be fixed in a later ticket :)

> Push composites support in the storage engine
> -
>
> Key: CASSANDRA-5417
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5417
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
> Fix For: 2.0
>
>
> CompositeType happens to be very useful and is now widely used: CQL3 heavily 
> rely on it, and super columns are now using it too internally. Besides, 
> CompositeType has been advised as a replacement of super columns on the 
> thrift side for a while, so it's safe to assume that it's generally used 
> there too.
> CompositeType has initially been introduced as just another AbstractType.  
> Meaning that the storage engine has no nothing whatsoever of composites 
> being, well, composite. This has the following drawbacks:
> * Because internally a composite value is handled as just a ByteBuffer, we 
> end up doing a lot of extra work. Typically, each time we compare 2 composite 
> value, we end up "deserializing" the components (which, while it doesn't copy 
> data per-se because we just slice the global ByteBuffer, still waste some cpu 
> cycles and allocate a bunch of ByteBuffer objects). And since compare can be 
> called *a lot*, this is likely not negligible.
> * This make CQL3 code uglier than necessary. Basically, CQL3 makes extensive 
> use of composites, and since it gets backs ByteBuffer from the internal 
> columns, it always have to check if it's actually a compositeType or not, and 
> then split it and pick the different parts it needs. It's only an API 
> problem, but having things exposed as composites directly would definitively 
> make thinks cleaner. In particular, in most cases, CQL3 don't care whether it 
> has a composite with only one component or a non-really-composite value, but 
> we still always distinguishes both cases.  Lastly, if we do expose composites 
> more directly internally, it's not a lot more work to "internalize" better 
> the different parts of the cell name that CQL3 uses (what's the clustering 
> key, what's the actuall CQL3 column name, what's the collection element), 
> making things cleaner. Last but not least, there is currently a bunch of 
> places where methods take a ByteBuffer as argument and it's hard to know 
> whether it expects a cell name or a CQL3 column name. This is pretty error 
> prone.
> * It makes it hard (or impossible) to do a number of performance 
> improvements.  Consider CASSANDRA-4175, I'm not really sure how you can do it 
> properly (in memory) if cell names are just ByteBuffer (since CQL3 column 
> names are just one of the component in general). But we also miss 
> oportunities of sharing prefixes. If we were able to share prefixes of 
> composite names in memory we would 1) lower the memory footprint and 2) 
> potentially speed-up comparison (of the prefixes) by checking reference 
> equality first (also, doing prefix sharing on-disk, which is a separate 
> concern btw, might be easier to do if we do prefix sharing in memory).
> So I suggest pushing CompositeType support inside the storage engine. What I 
> mean by that concretely would be change the internal {{Column.name}} from 
> ByteBuffer to some CellName type. A CellName would API-wise just be a list of 
> ByteBuffer. But in practice, we'd have a specific CellName implementation for 
> not-really-composite names, and the truly composite implementation will allow 
> some prefix sharing. From an external API however, nothing would change, we 
> would pack the composite as usual before sending it back to the client, but 
> at least internally, comparison won't have to deserialize the components 
> every time, and CQL3 code will be cleaner.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5418) streaming fails

2013-04-11 Thread Sylvain Lebresne (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-5418:


Attachment: 5418-v4.txt

I agree on the source of the problem. On the patch however, since the goal 
should be to write only what we get from the stream (since we've used the 
dataSize from the stream), it would feel more natural to me to just skip 
tombstoneTracker.writeOpenedMarker (in which case we really can skip the 
tombstone tracker completely and save a few CPU cycles). I'm attaching a v5 
patch that implement this (imo simpler) alternative.

Now as was noted above, this fix (whatever version of the patch we use) has the 
small downside that if the source and destination don't have the same 
column_index_size_in_kb, we'll be screwed. This is definitively a much less 
problem that this issue and so we should still fix this, but for 2.0, once 
CASSANRA-4180 gets in, then we should more or less revert this fix because it 
won't be necessary anymore. I've create CASSANRA-5454 so we don't forget about 
it.


> streaming fails
> ---
>
> Key: CASSANDRA-5418
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5418
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.2, 1.2.3
> Environment: 5 nodes, vnodes enabled, encryption disabled, 
> compression enabled, RackInferring snitch, Centos 6, Oracle JVM with JNA 
> enabled.
>Reporter: Igor Ivanov
>Priority: Critical
> Attachments: 5418-1.2.txt, 5418-1.2-v2.txt, 5418-1.2-v3.txt, 
> 5418-v4.txt
>
>
> When I run *nodetool repair* on cas01 node it get's stuck at some point.
> I see following exceptions in cas01 system.log:
> {quote}
> ERROR [Streaming to /10.10.45.60:28] 2013-04-02 09:03:55,353 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.60:28,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>   at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   ... 3 more
> ERROR [Thread-2076] 2013-04-02 09:07:12,261 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-2076,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-3660-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> {quote}
> On other machines there are some exceptions too:
> {quote}
> ERROR [Thread-1424] 2013-04-02 09:07:12,248 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-1424,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-2268-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> ERROR [Streaming to /10.10.45.58:55] 2013-04-02 09:07:12,263 
> CassandraDaemon.java (line 

[jira] [Created] (CASSANDRA-5454) Changing column_index_size_in_kb on different nodes might corrupt files

2013-04-11 Thread Sylvain Lebresne (JIRA)
Sylvain Lebresne created CASSANDRA-5454:
---

 Summary: Changing column_index_size_in_kb on different nodes might 
corrupt files
 Key: CASSANDRA-5454
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5454
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.2.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
 Fix For: 2.0


RangeTombstones requires that we sometimes repeat a few markers in the data 
file at index boundaries. Meaning that the same row with different 
column_index_size_in_kb will not have the same data size.

This is a problem for streaming, because if the column_index_size_in_kb is 
different in the source and the destination, the resulting row should have a 
different size on the destination, but streaming rely on the data size not 
changing in 1.2.

Now, while having different column_index_size on different nodes is probably 
not extremely useful in the long run, you may still have temporal discrepancies 
because there is no real way to change the setting on all node atomically. 
Besides, it's not to hard to get different setting on different nodes due to 
human error. And currently, the result is that if a file is stream while the 
setting is not consistent, then we'll end up corrupting the received file (due 
to the fix from CASSANDRA-5418 to be precise).

I don't see a good way to fix this in 1.2, so users will have to be careful not 
to have streaming happening while they change the column_index_size_in_kb 
setting. But in 2.0, once CASSANDRA-4180 is committed, we won't have the 
problem of having to respect the dataSize from the source on the destination 
anymore. So basically we should revert the fix from CASSANDRA-5418 (though we 
may still want to avoid repeating unneeded marker, but the tombstoneTracker can 
give us that easily).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5418) streaming fails

2013-04-11 Thread Igor Ivanov (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Igor Ivanov updated CASSANDRA-5418:
---

Attachment: 5418-1.2-v3.txt

v3 includes assertion, maybe will catch if column_index_size_in_kb is changed.

> streaming fails
> ---
>
> Key: CASSANDRA-5418
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5418
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.2, 1.2.3
> Environment: 5 nodes, vnodes enabled, encryption disabled, 
> compression enabled, RackInferring snitch, Centos 6, Oracle JVM with JNA 
> enabled.
>Reporter: Igor Ivanov
>Priority: Critical
> Attachments: 5418-1.2.txt, 5418-1.2-v2.txt, 5418-1.2-v3.txt
>
>
> When I run *nodetool repair* on cas01 node it get's stuck at some point.
> I see following exceptions in cas01 system.log:
> {quote}
> ERROR [Streaming to /10.10.45.60:28] 2013-04-02 09:03:55,353 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.60:28,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>   at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   ... 3 more
> ERROR [Thread-2076] 2013-04-02 09:07:12,261 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-2076,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-3660-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> {quote}
> On other machines there are some exceptions too:
> {quote}
> ERROR [Thread-1424] 2013-04-02 09:07:12,248 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-1424,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-2268-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> ERROR [Streaming to /10.10.45.58:55] 2013-04-02 09:07:12,263 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.58:55,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>   at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>   at 
> org.apache.cassandra

[jira] [Updated] (CASSANDRA-5418) streaming fails

2013-04-11 Thread Igor Ivanov (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Igor Ivanov updated CASSANDRA-5418:
---

Attachment: 5418-1.2-v2.txt

I've looked over the ColumnIndex.Builder code again and saw that it can build 
incorrect index (endPosition updated twice). So, added fromStream flag and skip 
logic to ColumnIndex.Builder.

> streaming fails
> ---
>
> Key: CASSANDRA-5418
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5418
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.2.2, 1.2.3
> Environment: 5 nodes, vnodes enabled, encryption disabled, 
> compression enabled, RackInferring snitch, Centos 6, Oracle JVM with JNA 
> enabled.
>Reporter: Igor Ivanov
>Priority: Critical
> Attachments: 5418-1.2.txt, 5418-1.2-v2.txt
>
>
> When I run *nodetool repair* on cas01 node it get's stuck at some point.
> I see following exceptions in cas01 system.log:
> {quote}
> ERROR [Streaming to /10.10.45.60:28] 2013-04-02 09:03:55,353 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.60:28,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>   at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   ... 3 more
> ERROR [Thread-2076] 2013-04-02 09:07:12,261 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-2076,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-3660-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> {quote}
> On other machines there are some exceptions too:
> {quote}
> ERROR [Thread-1424] 2013-04-02 09:07:12,248 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-1424,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-2268-Data.db;
>  correct is 131074
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>   at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>   at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> ERROR [Streaming to /10.10.45.58:55] 2013-04-02 09:07:12,263 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.58:55,5,main]
> java.lang.RuntimeException: java.io.EOFException
>   at com.google.common.base.Throwables.propagate(Throwables.java:160)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>   at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>   at java.io.DataInputStream.readInt(Unknown Source)
>   at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>   at 
> org.apache.cassandra.streaming.compress.Compres

[jira] [Comment Edited] (CASSANDRA-5062) Support CAS

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628800#comment-13628800
 ] 

Sylvain Lebresne edited comment on CASSANDRA-5062 at 4/11/13 9:56 AM:
--

Actually my bad, I should have ran the test before, but there is a few unit 
test failures. At least ColumnFamilyStoreTest is failing because it uses 
ColumnFamily.isEmpty that has been changed by the patch to include the 
deletionInfo. I'm fine just updating the test to use getColumnCount() instead, 
but it would be nice to do a quick check of all the current uses of 
ColumnFamily.isEmpty to make sure this don't break anything else (and we should 
update the comment of the isEmpty method).

I also get
{noformat}
[junit] Testcase: 
testRowMutationRead(org.apache.cassandra.db.SerializationsTest):  Caused an 
ERROR
[junit] addr is of illegal length
[junit] java.net.UnknownHostException: addr is of illegal length
[junit] at java.net.InetAddress.getByAddress(InetAddress.java:935)
[junit] at java.net.InetAddress.getByAddress(InetAddress.java:1318)
[junit] at 
org.apache.cassandra.net.CompactEndpointSerializationHelper.deserialize(CompactEndpointSerializationHelper.java:38)
[junit] at org.apache.cassandra.net.MessageIn.read(MessageIn.java:62)
[junit] at 
org.apache.cassandra.db.SerializationsTest.testRowMutationRead(SerializationsTest.java:263)
{noformat}
My initial reaction was to say that this is unrelated to this ticket but the 
test passes on trunk. Maybe the patch just need to be rebased?

  was (Author: slebresne):
Actually my bad, I should have ran the test before, but there is a few unit 
test failures. At least ColumnFamilyStoreTest is failing because it uses 
ColumnFamily.isEmpty that has been changed by the patch to include the 
deletionInfo. I'm fine just updating the test to use getColumnCount() instead, 
but it would be nice to do a quick check of all the current uses of 
ColumnFamily.isEmpty to make sure this don't break anything else (and we should 
update the comment of the isEmpty method).
  
> Support CAS
> ---
>
> Key: CASSANDRA-5062
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5062
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
> Attachments: half-baked commit 1.jpg, half-baked commit 2.jpg, 
> half-baked commit 3.jpg
>
>
> "Strong" consistency is not enough to prevent race conditions.  The classic 
> example is user account creation: we want to ensure usernames are unique, so 
> we only want to signal account creation success if nobody else has created 
> the account yet.  But naive read-then-write allows clients to race and both 
> think they have a green light to create.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5062) Support CAS

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628800#comment-13628800
 ] 

Sylvain Lebresne commented on CASSANDRA-5062:
-

Actually my bad, I should have ran the test before, but there is a few unit 
test failures. At least ColumnFamilyStoreTest is failing because it uses 
ColumnFamily.isEmpty that has been changed by the patch to include the 
deletionInfo. I'm fine just updating the test to use getColumnCount() instead, 
but it would be nice to do a quick check of all the current uses of 
ColumnFamily.isEmpty to make sure this don't break anything else (and we should 
update the comment of the isEmpty method).

> Support CAS
> ---
>
> Key: CASSANDRA-5062
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5062
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
> Attachments: half-baked commit 1.jpg, half-baked commit 2.jpg, 
> half-baked commit 3.jpg
>
>
> "Strong" consistency is not enough to prevent race conditions.  The classic 
> example is user account creation: we want to ensure usernames are unique, so 
> we only want to signal account creation success if nobody else has created 
> the account yet.  But naive read-then-write allows clients to race and both 
> think they have a green light to create.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5062) Support CAS

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628792#comment-13628792
 ] 

Sylvain Lebresne commented on CASSANDRA-5062:
-

I guess I'm out of nitpicks for this part, +1.

Well, maybe just one last nit for the road: the equals and hashcode in Commit 
are missing @Override.

> Support CAS
> ---
>
> Key: CASSANDRA-5062
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5062
> Project: Cassandra
>  Issue Type: New Feature
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
> Attachments: half-baked commit 1.jpg, half-baked commit 2.jpg, 
> half-baked commit 3.jpg
>
>
> "Strong" consistency is not enough to prevent race conditions.  The classic 
> example is user account creation: we want to ensure usernames are unique, so 
> we only want to signal account creation success if nobody else has created 
> the account yet.  But naive read-then-write allows clients to race and both 
> think they have a green light to create.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5441) Add support for read at CL.SERIAL

2013-04-11 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5441?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628791#comment-13628791
 ] 

Sylvain Lebresne commented on CASSANDRA-5441:
-

Agreed, I think that's about all we need.

But as a side note, it would be nice to use this ticket to also add proper 
validation of CL.SERIAL (refuse it in normal writes, in any type of counter 
operation, etc...).

> Add support for read at CL.SERIAL
> -
>
> Key: CASSANDRA-5441
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5441
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: API, Core
>Reporter: Jonathan Ellis
> Fix For: 2.0
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[1/2] git commit: Allow empty blob literals in CQL3

2013-04-11 Thread slebresne
Updated Branches:
  refs/heads/trunk 4ab0dacad -> be78b3a5b


Allow empty blob literals in CQL3

patch by slebresne; reviewed by iamaleksey for CASSANDRA-5452


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

Branch: refs/heads/trunk
Commit: 83ed1cbda98bec659fdf8ffb7e19789c04e36763
Parents: 225cb0e
Author: Sylvain Lebresne 
Authored: Thu Apr 11 10:48:35 2013 +0200
Committer: Sylvain Lebresne 
Committed: Thu Apr 11 10:49:50 2013 +0200

--
 CHANGES.txt  |1 +
 src/java/org/apache/cassandra/cql3/Cql.g |2 +-
 2 files changed, 2 insertions(+), 1 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/83ed1cbd/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 66279f0..5c26014 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,6 +2,7 @@
  * 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)
 Merged from 1.1:
  * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83ed1cbd/src/java/org/apache/cassandra/cql3/Cql.g
--
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g 
b/src/java/org/apache/cassandra/cql3/Cql.g
index 0c2113f..a91e529 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -1052,7 +1052,7 @@ IDENT
 ;
 
 HEXNUMBER
-: '0' X HEX+
+: '0' X HEX*
 ;
 
 UUID



git commit: Allow empty blob literals in CQL3

2013-04-11 Thread slebresne
Updated Branches:
  refs/heads/cassandra-1.2 225cb0e7f -> 83ed1cbda


Allow empty blob literals in CQL3

patch by slebresne; reviewed by iamaleksey for CASSANDRA-5452


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

Branch: refs/heads/cassandra-1.2
Commit: 83ed1cbda98bec659fdf8ffb7e19789c04e36763
Parents: 225cb0e
Author: Sylvain Lebresne 
Authored: Thu Apr 11 10:48:35 2013 +0200
Committer: Sylvain Lebresne 
Committed: Thu Apr 11 10:49:50 2013 +0200

--
 CHANGES.txt  |1 +
 src/java/org/apache/cassandra/cql3/Cql.g |2 +-
 2 files changed, 2 insertions(+), 1 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/83ed1cbd/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 66279f0..5c26014 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -2,6 +2,7 @@
  * 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)
 Merged from 1.1:
  * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/83ed1cbd/src/java/org/apache/cassandra/cql3/Cql.g
--
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g 
b/src/java/org/apache/cassandra/cql3/Cql.g
index 0c2113f..a91e529 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -1052,7 +1052,7 @@ IDENT
 ;
 
 HEXNUMBER
-: '0' X HEX+
+: '0' X HEX*
 ;
 
 UUID