[jira] [Commented] (CASSANDRA-2975) Upgrade MurmurHash to version 3

2011-11-16 Thread David Allsopp (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151092#comment-13151092
 ] 

David Allsopp commented on CASSANDRA-2975:
--

Having slept on it, I realized that the {{switch-case}} is slow because of the 
{{switch-case}} fallthrough in Java - _i.e. all 15 cases get tested every time_ 
(http://download.oracle.com/javase/tutorial/java/nutsandbolts/switch.html)

Adding a {{break}} to the end of each case fixes this without the verbosity of 
the binary search approach, i.e:

{noformat}
switch (length  15)
{
case 0: break;
case 1:
k1 ^= ((long) key.get(offset));
k1 *= c1;
k1 = rotl64(k1, 31);
k1 *= c2;
h1 ^= k1;
break;
case 2:
k1 ^= ((long) key.get(offset + 1))  8;
break;
...etc...
{noformat}

Combining this with the inlining almost doubles the speed of hashing for keys 
with lengths from 1-32 bytes!

 Upgrade MurmurHash to version 3
 ---

 Key: CASSANDRA-2975
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2975
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brian Lindauer
Assignee: Brian Lindauer
Priority: Trivial
  Labels: lhf
 Fix For: 1.1

 Attachments: 
 0001-Convert-BloomFilter-to-use-MurmurHash-v3-instead-of-.patch, 
 0002-Backwards-compatibility-with-files-using-Murmur2-blo.patch


 MurmurHash version 3 was finalized on June 3. It provides an enormous speedup 
 and increased robustness over version 2, which is implemented in Cassandra. 
 Information here:
 http://code.google.com/p/smhasher/
 The reference implementation is here:
 http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp?spec=svn136r=136
 I have already done the work to port the (public domain) reference 
 implementation to Java in the MurmurHash class and updated the BloomFilter 
 class to use the new implementation:
 https://github.com/lindauer/cassandra/commit/cea6068a4a3e5d7d9509335394f9ef3350d37e93
 Apart from the faster hash time, the new version only requires one call to 
 hash() rather than 2, since it returns 128 bits of hash instead of 64.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2893) Add row-level isolation

2011-11-16 Thread Sylvain Lebresne (Updated) (JIRA)

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

Sylvain Lebresne updated CASSANDRA-2893:


Attachment: snaptree-0.1-SNAPSHOT.jar

 Add row-level isolation
 ---

 Key: CASSANDRA-2893
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2893
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.1

 Attachments: 0001-Move-deletion-infos-into-ISortedColumns.patch, 
 0002-Make-memtable-use-CF.addAll.patch, 
 0003-Add-AtomicSortedColumn-and-snapTree.patch, snaptree-0.1-SNAPSHOT.jar


 This could be done using an the atomic ConcurrentMap operations from the 
 Memtable and something like http://code.google.com/p/pcollections/ to replace 
 the ConcurrentSkipListMap in ThreadSafeSortedColumns.  The trick is that 
 pcollections does not provide a SortedMap, so we probably need to write our 
 own.
 Googling [persistent sortedmap] I found 
 http://code.google.com/p/actord/source/browse/trunk/actord/src/main/scala/ff/collection
  (in scala) and http://clojure.org/data_structures#Data Structures-Maps.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2893) Add row-level isolation

2011-11-16 Thread Sylvain Lebresne (Updated) (JIRA)

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

Sylvain Lebresne updated CASSANDRA-2893:


Attachment: 0003-Add-AtomicSortedColumn-and-snapTree.patch
0002-Make-memtable-use-CF.addAll.patch
0001-Move-deletion-infos-into-ISortedColumns.patch

Attaching initial patches.

The basic idea is to make it so that applying a mutation to a memtable is 
atomic, or in other words, make it use CF.addAll() and have that last operation 
be atomic and isolated (adding to the row cache also needs to be atomic and 
isolated but it uses CF.addAll already so making CF.addAll atomic is the 
solution for that too).

To do that, addAll copies the initial cf, add the new columns and atomically 
compare and swap with the old one cf. To make this efficient, the patch uses 
the snapTree O(1) cloning (copy-on-write) facilities.

I'm attaching the snapTree jar, but note that it's modified from the original 
(https://github.com/nbronson/snaptree) because it has bug. The modified version 
with the small fix is at https://github.com/pcmanus/snaptree (I've issued a 
pull request). Btw, I don't know if the license of snapTree is compatible with 
the ASF one. Note that we only use the copy-on-write clone facility of 
snapTree, and not really the fact that it is thread-safe outside of that. So in 
particular a persistent sorted map could be used in place of snapTree if we 
wanted to, though the copy-on-write used by the latter is likely to generate 
less garbage overall.

I'm attaching 3 patches:
* The first patch pushes the CF deletion infos from the AbstractColumnContainer 
to the ISortedColums implementation. Reason being that we will want that both 
updates and deletes are atomic and isolated so we'll need to have those in the 
same structure.
* The second patch modifies Memtable.apply() to use CF.addAll directly.
* The third patch introduces AtomicSortedColumns using snapTree and uses it 
whenever thread-safety/isolation is needed. Note that it fully replace 
ThreadSafeSortedColumns that is removed, and also that the patch tries to limit 
the use of AtomicSortedColumns to concurrent context, making 
TreeMapBackedSortedColumns the default for other non-concurrent context.

There is two gray areas with this patch that I know of:
* It would be easy to break isolation for super columns. If cf is an 
AtomicSortedColumns backed (super) column family and you do a {{sc = 
cf.getColumn(someSCname)}} and then do {{sc.addAll(cols)}}, then that last 
operation won't be in isolation. I don't think we do that in any context where 
it matters, but still something to be aware of.
* Iterator based removal is not thread-safe. Basically, if you do an iteration, 
doing removes using the iterator remove() method and there is a concurrent 
mutation on the cf, the remove may well just be ignored.  I think the main 
place where we do iterator based removes is during CFS.removeDeleted(). But 
it's mostly done during queries/compaction so not in a concurrent context. We 
do a removeDeleted on cachedRow sometimes during compaction but in that case it 
won't be the end of the world if that remove is ignored because of a concurrent 
mutation. Still, not very beautiful but I don't see a simple solution (outside 
of not using iterator based removes that is).

Overall, I think the patch is ready for benchmarking (all unit tests are 
passing). I did a very quick stress test on my localhost and I didn't see any 
noticeable difference with or without the patch (neither writes nor reads).  
But 1) that was not a very scientific benchmark and 2) it was a short 
benchmark. I don't think raw performance will a problem with this patch, the 
problem is that it generates more garbage, which itself may degrade performance 
on the long run. That's probably what we'll want to benchmark.

 Add row-level isolation
 ---

 Key: CASSANDRA-2893
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2893
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.1

 Attachments: 0001-Move-deletion-infos-into-ISortedColumns.patch, 
 0002-Make-memtable-use-CF.addAll.patch, 
 0003-Add-AtomicSortedColumn-and-snapTree.patch, snaptree-0.1-SNAPSHOT.jar


 This could be done using an the atomic ConcurrentMap operations from the 
 Memtable and something like http://code.google.com/p/pcollections/ to replace 
 the ConcurrentSkipListMap in ThreadSafeSortedColumns.  The trick is that 
 pcollections does not provide a SortedMap, so we probably need to write our 
 own.
 Googling [persistent sortedmap] I found 
 http://code.google.com/p/actord/source/browse/trunk/actord/src/main/scala/ff/collection
  (in scala) and 

[jira] [Issue Comment Edited] (CASSANDRA-2975) Upgrade MurmurHash to version 3

2011-11-16 Thread David Allsopp (Issue Comment Edited) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151092#comment-13151092
 ] 

David Allsopp edited comment on CASSANDRA-2975 at 11/16/11 9:31 AM:


Having slept on it, I realized that the {{switch-case}} is slow because of the 
{{switch-case}} fallthrough in Java - _i.e. all 15 cases get tested every time_ 
(http://download.oracle.com/javase/tutorial/java/nutsandbolts/switch.html)

Adding a {{break}} to the end of each case fixes this without the verbosity of 
the binary search approach, i.e:

{noformat}
switch (length  15)
{
case 0: break;
case 1:
k1 ^= ((long) key.get(offset));
k1 *= c1;
k1 = rotl64(k1, 31);
k1 *= c2;
h1 ^= k1;
break;
case 2:
k1 ^= ((long) key.get(offset + 1))  8;
break;
...etc...
{noformat}

Combining this with the inlining almost doubles the speed of hashing for keys 
with lengths from 1-32 bytes!

UPDATE: Argh, this breaks something - the hash output is different. Can't see 
why yet...

  was (Author: dallsopp):
Having slept on it, I realized that the {{switch-case}} is slow because of 
the {{switch-case}} fallthrough in Java - _i.e. all 15 cases get tested every 
time_ (http://download.oracle.com/javase/tutorial/java/nutsandbolts/switch.html)

Adding a {{break}} to the end of each case fixes this without the verbosity of 
the binary search approach, i.e:

{noformat}
switch (length  15)
{
case 0: break;
case 1:
k1 ^= ((long) key.get(offset));
k1 *= c1;
k1 = rotl64(k1, 31);
k1 *= c2;
h1 ^= k1;
break;
case 2:
k1 ^= ((long) key.get(offset + 1))  8;
break;
...etc...
{noformat}

Combining this with the inlining almost doubles the speed of hashing for keys 
with lengths from 1-32 bytes!
  
 Upgrade MurmurHash to version 3
 ---

 Key: CASSANDRA-2975
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2975
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brian Lindauer
Assignee: Brian Lindauer
Priority: Trivial
  Labels: lhf
 Fix For: 1.1

 Attachments: 
 0001-Convert-BloomFilter-to-use-MurmurHash-v3-instead-of-.patch, 
 0002-Backwards-compatibility-with-files-using-Murmur2-blo.patch


 MurmurHash version 3 was finalized on June 3. It provides an enormous speedup 
 and increased robustness over version 2, which is implemented in Cassandra. 
 Information here:
 http://code.google.com/p/smhasher/
 The reference implementation is here:
 http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp?spec=svn136r=136
 I have already done the work to port the (public domain) reference 
 implementation to Java in the MurmurHash class and updated the BloomFilter 
 class to use the new implementation:
 https://github.com/lindauer/cassandra/commit/cea6068a4a3e5d7d9509335394f9ef3350d37e93
 Apart from the faster hash time, the new version only requires one call to 
 hash() rather than 2, since it returns 128 bits of hash instead of 64.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2975) Upgrade MurmurHash to version 3

2011-11-16 Thread David Allsopp (Updated) (JIRA)

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

David Allsopp updated CASSANDRA-2975:
-

Comment: was deleted

(was: Having slept on it, I realized that the {{switch-case}} is slow because 
of the {{switch-case}} fallthrough in Java - _i.e. all 15 cases get tested 
every time_ 
(http://download.oracle.com/javase/tutorial/java/nutsandbolts/switch.html)

Adding a {{break}} to the end of each case fixes this without the verbosity of 
the binary search approach, i.e:

{noformat}
switch (length  15)
{
case 0: break;
case 1:
k1 ^= ((long) key.get(offset));
k1 *= c1;
k1 = rotl64(k1, 31);
k1 *= c2;
h1 ^= k1;
break;
case 2:
k1 ^= ((long) key.get(offset + 1))  8;
break;
...etc...
{noformat}

Combining this with the inlining almost doubles the speed of hashing for keys 
with lengths from 1-32 bytes!

UPDATE: Argh, this breaks something - the hash output is different. Can't see 
why yet...)

 Upgrade MurmurHash to version 3
 ---

 Key: CASSANDRA-2975
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2975
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brian Lindauer
Assignee: Brian Lindauer
Priority: Trivial
  Labels: lhf
 Fix For: 1.1

 Attachments: 
 0001-Convert-BloomFilter-to-use-MurmurHash-v3-instead-of-.patch, 
 0002-Backwards-compatibility-with-files-using-Murmur2-blo.patch


 MurmurHash version 3 was finalized on June 3. It provides an enormous speedup 
 and increased robustness over version 2, which is implemented in Cassandra. 
 Information here:
 http://code.google.com/p/smhasher/
 The reference implementation is here:
 http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp?spec=svn136r=136
 I have already done the work to port the (public domain) reference 
 implementation to Java in the MurmurHash class and updated the BloomFilter 
 class to use the new implementation:
 https://github.com/lindauer/cassandra/commit/cea6068a4a3e5d7d9509335394f9ef3350d37e93
 Apart from the faster hash time, the new version only requires one call to 
 hash() rather than 2, since it returns 128 bits of hash instead of 64.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2975) Upgrade MurmurHash to version 3

2011-11-16 Thread David Allsopp (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151121#comment-13151121
 ] 

David Allsopp commented on CASSANDRA-2975:
--

I really must learn not to post stuff late at night :-(.  The optimisation of 
the {{switch-case}} breaks the algorithm because it relies on the fall-through 
behaviour of {{switch-case}} in C and Java. Oh well. The inlining only speeds 
things up a few percent, but might be worthwhile if others see the same 
improvement on other hardware.

 Upgrade MurmurHash to version 3
 ---

 Key: CASSANDRA-2975
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2975
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brian Lindauer
Assignee: Brian Lindauer
Priority: Trivial
  Labels: lhf
 Fix For: 1.1

 Attachments: 
 0001-Convert-BloomFilter-to-use-MurmurHash-v3-instead-of-.patch, 
 0002-Backwards-compatibility-with-files-using-Murmur2-blo.patch


 MurmurHash version 3 was finalized on June 3. It provides an enormous speedup 
 and increased robustness over version 2, which is implemented in Cassandra. 
 Information here:
 http://code.google.com/p/smhasher/
 The reference implementation is here:
 http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp?spec=svn136r=136
 I have already done the work to port the (public domain) reference 
 implementation to Java in the MurmurHash class and updated the BloomFilter 
 class to use the new implementation:
 https://github.com/lindauer/cassandra/commit/cea6068a4a3e5d7d9509335394f9ef3350d37e93
 Apart from the faster hash time, the new version only requires one call to 
 hash() rather than 2, since it returns 128 bits of hash instead of 64.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Issue Comment Edited] (CASSANDRA-2975) Upgrade MurmurHash to version 3

2011-11-16 Thread David Allsopp (Issue Comment Edited) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13150931#comment-13150931
 ] 

David Allsopp edited comment on CASSANDRA-2975 at 11/16/11 10:20 AM:
-

Does anyone have any data on what the typical key size is (i.e. the average 
input size for the hash)?

I have a couple of optimisations for the MurmurHash3 implementation that I 
think give another 10-40% speedup, particularly for smaller values (e.g. 30% 
speedup for buffer lengths under 256 bytes) and no worse for large values (tens 
of KB). These results were on a AMD Phenom II X6 1055T @ 2.80 GHz, under 64-bit 
Windows 7, Java 1.6.0_27.

Firstly, inline the {{rotl64}} calls , e.g. so that
{noformat}
k1 = rotl64(k1, 31);
{noformat}
becomes
{noformat}
k1 = (k1  31) | (k1  33);
{noformat}

-Secondly, rather than a large {{switch-case}} to handle the 'tail', use nested 
{{if-else}} to form a simple binary search. Particularly for relatively small 
inputs, handling the 'tail' is a significant part of the computation. E.g:-

{noformat}
int ln = length  15;
if (ln  8)
  {
 if (ln  12)
   {
  // etc for cases 13 - 15
   }
 else
   {
  // cases 11 and 12
   }

  }
else
  {
 // etc for cases 1-7
  }
{noformat}

Will try to post a proper benchmark when I've tidied it up (run out of time 
today!) so anyone interested can try it on other hardware...

-This latter optimisation is pretty verbose and ugly to look at - it _might_ be 
just as fast, and much more concise, to lookup the offsets and shifts from an 
array, and deal with the special cases 1 and 9 as, well, special cases - but 
haven't benchmarked this alternative yet...-

  was (Author: dallsopp):
Does anyone have any data on what the typical key size is (i.e. the average 
input size for the hash)?

I have a couple of optimisations for the MurmurHash3 implementation that I 
think give another 10-40% speedup, particularly for smaller values (e.g. 30% 
speedup for buffer lengths under 256 bytes) and no worse for large values (tens 
of KB). These results were on a AMD Phenom II X6 1055T @ 2.80 GHz, under 64-bit 
Windows 7, Java 1.6.0_27.

Firstly, inline the {{rotl64}} calls , e.g. so that
{noformat}
k1 = rotl64(k1, 31);
{noformat}
becomes
{noformat}
k1 = (k1  31) | (k1  33);
{noformat}

Secondly, rather than a large {{switch-case}} to handle the 'tail', use nested 
{{if-else}} to form a simple binary search. Particularly for relatively small 
inputs, handling the 'tail' is a significant part of the computation. E.g:

{noformat}
int ln = length  15;
if (ln  8)
  {
 if (ln  12)
   {
  // etc for cases 13 - 15
   }
 else
   {
  // cases 11 and 12
   }

  }
else
  {
 // etc for cases 1-7
  }
{noformat}

Will try to post a proper benchmark when I've tidied it up (run out of time 
today!) so anyone interested can try it on other hardware...

This latter optimisation is pretty verbose and ugly to look at - it _might_ be 
just as fast, and much more concise, to lookup the offsets and shifts from an 
array, and deal with the special cases 1 and 9 as, well, special cases - but 
haven't benchmarked this alternative yet...
  
 Upgrade MurmurHash to version 3
 ---

 Key: CASSANDRA-2975
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2975
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brian Lindauer
Assignee: Brian Lindauer
Priority: Trivial
  Labels: lhf
 Fix For: 1.1

 Attachments: 
 0001-Convert-BloomFilter-to-use-MurmurHash-v3-instead-of-.patch, 
 0002-Backwards-compatibility-with-files-using-Murmur2-blo.patch


 MurmurHash version 3 was finalized on June 3. It provides an enormous speedup 
 and increased robustness over version 2, which is implemented in Cassandra. 
 Information here:
 http://code.google.com/p/smhasher/
 The reference implementation is here:
 http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp?spec=svn136r=136
 I have already done the work to port the (public domain) reference 
 implementation to Java in the MurmurHash class and updated the BloomFilter 
 class to use the new implementation:
 https://github.com/lindauer/cassandra/commit/cea6068a4a3e5d7d9509335394f9ef3350d37e93
 Apart from the faster hash time, the new version only requires one call to 
 hash() rather than 2, since it returns 128 bits of hash instead of 64.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3496) Load from `nodetool ring` does not update after cleanup.

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3496?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151127#comment-13151127
 ] 

Sylvain Lebresne commented on CASSANDRA-3496:
-

patch lgtm, +1

 Load from `nodetool ring` does not update after cleanup.
 

 Key: CASSANDRA-3496
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3496
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.0
Reporter: Benjamin Coverston
Assignee: Jonathan Ellis
 Fix For: 1.0.4

 Attachments: 3496.txt


 Repro:
 Bring up a node.
 Insert 1M rows:
 127.0.0.1   datacenter1 rack1   Up Normal  406.92 MB   
 100.00% 77747037169725419723056812679314618801
 (Already looks wrong, 406.92 is higher than I'm used to seeing from a single 
 run of stress)
 Bootstrap a second node into the cluster:
 162877269496252595336256012556853953561
 127.0.0.1   datacenter1 rack1   Up Normal  407.03 MB   49.96% 
  77747037169725419723056812679314618801
 127.0.0.2   datacenter1 rack1   Up Normal  157.91 MB   50.04% 
  162877269496252595336256012556853953561
 Cleanup
 162877269496252595336256012556853953561
 127.0.0.1   datacenter1 rack1   Up Normal  551.2 MB   49.96%  
 77747037169725419723056812679314618801
 127.0.0.2   datacenter1 rack1   Up Normal  157.91 MB   50.04% 
  162877269496252595336256012556853953561
 Looks like each operation that adds and removes SSTables only adds to the 
 total and doesn't remove the old sstables from the total size count.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3496) Load from `nodetool ring` does not update after cleanup.

2011-11-16 Thread Sylvain Lebresne (Updated) (JIRA)

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

Sylvain Lebresne updated CASSANDRA-3496:


 Reviewer: slebresne
Affects Version/s: (was: 1.0.2)
   1.0.0
Fix Version/s: 1.0.4
 Assignee: Jonathan Ellis  (was: Sylvain Lebresne)

 Load from `nodetool ring` does not update after cleanup.
 

 Key: CASSANDRA-3496
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3496
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.0
Reporter: Benjamin Coverston
Assignee: Jonathan Ellis
 Fix For: 1.0.4

 Attachments: 3496.txt


 Repro:
 Bring up a node.
 Insert 1M rows:
 127.0.0.1   datacenter1 rack1   Up Normal  406.92 MB   
 100.00% 77747037169725419723056812679314618801
 (Already looks wrong, 406.92 is higher than I'm used to seeing from a single 
 run of stress)
 Bootstrap a second node into the cluster:
 162877269496252595336256012556853953561
 127.0.0.1   datacenter1 rack1   Up Normal  407.03 MB   49.96% 
  77747037169725419723056812679314618801
 127.0.0.2   datacenter1 rack1   Up Normal  157.91 MB   50.04% 
  162877269496252595336256012556853953561
 Cleanup
 162877269496252595336256012556853953561
 127.0.0.1   datacenter1 rack1   Up Normal  551.2 MB   49.96%  
 77747037169725419723056812679314618801
 127.0.0.2   datacenter1 rack1   Up Normal  157.91 MB   50.04% 
  162877269496252595336256012556853953561
 Looks like each operation that adds and removes SSTables only adds to the 
 total and doesn't remove the old sstables from the total size count.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2506) Push read repair setting down to the DC-level

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2506?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151185#comment-13151185
 ] 

Sylvain Lebresne commented on CASSANDRA-2506:
-

I want to note that this v3 doesn't handle the case in the description of this 
ticket. Basically there has been 2 proposition of enhancements for RR on this 
ticket:
  # Being able to completely exlude a given DC of any RR
  # Have a different chance of repair for intra-DC and inter-DC RR

The initial patch from Vijay was basically handling both. I only regretted that 
this was resulting in something imo complicated for the user (but without 
having a much better initiave to suggest).

Jonathan then proposed to only add a list of DC that would be excluded from RR. 
This obviously only takle the 1st improvement above. The v3 chooses to only add 
a local RR chance, which only tackle the 2nd improvement above.

I'm fine saying RR is still useful and needs DC-related improvements, but we 
should first agree on what those are maybe.

On the patch itself:
* Still no handling of KW_DCLOCALREADREPAIRCHANCE in 
CreateColumnFamilyStatement.java
* If there is not enough node in the local DC for blockfor and if the first 
endpoints of the endpoint list are not (all) from the local DC (possible), then 
we could end up no read repairing all the node from the local DC even if the 
chance  DcLocalReadRepair.


 Push read repair setting down to the DC-level
 -

 Key: CASSANDRA-2506
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2506
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.1
Reporter: Brandon Williams
Assignee: Vijay
Priority: Minor
 Fix For: 1.0.4

 Attachments: 0001-dc-localized-read-repair-v2.patch, 
 0001-dc-localized-read-repair.patch, 0001-thrift-and-avro-changes-v3.patch, 
 0002-dc-localized-read-repair-v3.patch, 0002-thrift-and-avro-v2.patch, 
 0002-thrift-and-avro.patch, 0003-documentation-for-read_repair-v3.patch, 
 0003-documentation-for-read_repair_options-v2.patch, 
 0003-documentation-for-read_repair_options.patch


 Currently, read repair is a global setting.  However, when you have two DCs 
 and use one for analytics, it would be nice to turn it off only for that DC 
 so the live DC serving the application can still benefit from it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Resolved] (CASSANDRA-3481) During repair, incorrect data size Connection reset errors. Repair unable to complete.

2011-11-16 Thread Sylvain Lebresne (Resolved) (JIRA)

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

Sylvain Lebresne resolved CASSANDRA-3481.
-

   Resolution: Fixed
Fix Version/s: (was: 1.0.4)
   1.0.3
 Reviewer: jbellis

Forgot to close this one but it's been committed already.

 During repair, incorrect data size  Connection reset errors. Repair 
 unable to complete.
 

 Key: CASSANDRA-3481
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.2
Reporter: Eric Falcao
Assignee: Sylvain Lebresne
  Labels: connection, repair
 Fix For: 1.0.3

 Attachments: 3481-v2.patch, 3481.patch


 This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm 
 fairly certain repair was working ok. Repair worked decently for me in 0.8 
 (data bloat sucked). All my SSTables are version h.
 On one node:
 java.lang.AssertionError: incorrect row data size 596045 written to 
 /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
   at 
 org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
   at 
 org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
   at 
 org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
   at 
 org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
 On the other node:
 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 
 progress=0/1513497639 - 0%, 
 /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 
 progress=0/53400713 - 0%, 
 /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 
 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db 
 sections=550 progress=0/449498 - 0%, 
 /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 
 progress=0/316301 - 0%], 6 sstables.
  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 
 203) Streaming to /10.38.69.192
 ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java 
 (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
 java.lang.RuntimeException: java.net.SocketException: Connection reset
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
   at java.lang.Thread.run(Thread.java:619)
 Caused by: java.net.SocketException: Connection reset
   at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
   at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
   at 
 com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
   at 
 com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
   at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
   at 
 org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
   at 
 org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
   at 
 org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
   ... 3 more
 ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java 
 (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
 java.lang.RuntimeException: java.net.SocketException: Connection reset
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
   at java.lang.Thread.run(Thread.java:619)
 Caused by: java.net.SocketException: Connection reset
   at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
   at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
   at 
 com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
   at 
 com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
   at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
   at 
 org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
   at 
 

[jira] [Commented] (CASSANDRA-3362) allow sub-row repair

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151199#comment-13151199
 ] 

Sylvain Lebresne commented on CASSANDRA-3362:
-

Right now we're using tokens range to create the merkle tree, so we cannot 
repair less than a token without major changes to repair.
Besides, repair needs to use the same atoms on all the node it repairs, so I 
don't think the row index blocks would qualify since they differ from node to 
node.

Overall, I don't see how that can be done with the current repair.

 allow sub-row repair
 

 Key: CASSANDRA-3362
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3362
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
  Labels: repair

 With large rows, it would be nice to not have to send an entire row if a 
 small part is out of sync.  Could we use the row index blocks as repair atoms 
 instead of the full row?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2893) Add row-level isolation

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151206#comment-13151206
 ] 

Jonathan Ellis commented on CASSANDRA-2893:
---

SnapTree License looks like standard bsd-with-attribution, which is 
ASL-compatible.

 Add row-level isolation
 ---

 Key: CASSANDRA-2893
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2893
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.1

 Attachments: 0001-Move-deletion-infos-into-ISortedColumns.patch, 
 0002-Make-memtable-use-CF.addAll.patch, 
 0003-Add-AtomicSortedColumn-and-snapTree.patch, snaptree-0.1-SNAPSHOT.jar


 This could be done using an the atomic ConcurrentMap operations from the 
 Memtable and something like http://code.google.com/p/pcollections/ to replace 
 the ConcurrentSkipListMap in ThreadSafeSortedColumns.  The trick is that 
 pcollections does not provide a SortedMap, so we probably need to write our 
 own.
 Googling [persistent sortedmap] I found 
 http://code.google.com/p/actord/source/browse/trunk/actord/src/main/scala/ff/collection
  (in scala) and http://clojure.org/data_structures#Data Structures-Maps.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2893) Add row-level isolation

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151209#comment-13151209
 ] 

Jonathan Ellis commented on CASSANDRA-2893:
---

bq. I didn't see any noticeable difference

Do we even have an insert test mode that can generate same-row contention?

bq. it generates more garbage

but young-gen garbage, which is close to free.

 Add row-level isolation
 ---

 Key: CASSANDRA-2893
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2893
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.1

 Attachments: 0001-Move-deletion-infos-into-ISortedColumns.patch, 
 0002-Make-memtable-use-CF.addAll.patch, 
 0003-Add-AtomicSortedColumn-and-snapTree.patch, snaptree-0.1-SNAPSHOT.jar


 This could be done using an the atomic ConcurrentMap operations from the 
 Memtable and something like http://code.google.com/p/pcollections/ to replace 
 the ConcurrentSkipListMap in ThreadSafeSortedColumns.  The trick is that 
 pcollections does not provide a SortedMap, so we probably need to write our 
 own.
 Googling [persistent sortedmap] I found 
 http://code.google.com/p/actord/source/browse/trunk/actord/src/main/scala/ff/collection
  (in scala) and http://clojure.org/data_structures#Data Structures-Maps.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2893) Add row-level isolation

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151221#comment-13151221
 ] 

Sylvain Lebresne commented on CASSANDRA-2893:
-

bq. Do we even have an insert test mode that can generate same-row contention?

I don't think so, but that would be useful, even outside of testing this.

bq. but young-gen garbage, which is close to free

I suppose it depends but I don't think it will be necessarily young-gen, on the 
contrary even. The added garbage is that when we add new columns to a row 
already in the memtable, we'll copy a number of nodes of the underlying 
SnapTree. But those old nodes are live since their initial insertion and the 
new ones will be until the next insertion into this row (or longer, we don't do 
a full copy). So this roughly depends of the average between two updates to the 
same row in a memtable. Not totally sure it fits into a young-gen. That being 
I'm not particularly worried.

 Add row-level isolation
 ---

 Key: CASSANDRA-2893
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2893
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.1

 Attachments: 0001-Move-deletion-infos-into-ISortedColumns.patch, 
 0002-Make-memtable-use-CF.addAll.patch, 
 0003-Add-AtomicSortedColumn-and-snapTree.patch, snaptree-0.1-SNAPSHOT.jar


 This could be done using an the atomic ConcurrentMap operations from the 
 Memtable and something like http://code.google.com/p/pcollections/ to replace 
 the ConcurrentSkipListMap in ThreadSafeSortedColumns.  The trick is that 
 pcollections does not provide a SortedMap, so we probably need to write our 
 own.
 Googling [persistent sortedmap] I found 
 http://code.google.com/p/actord/source/browse/trunk/actord/src/main/scala/ff/collection
  (in scala) and http://clojure.org/data_structures#Data Structures-Maps.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-1740) Nodetool commands to query and stop compaction, repair, cleanup and scrub

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151251#comment-13151251
 ] 

Jonathan Ellis commented on CASSANDRA-1740:
---

I'm not a fan of the instanceof UIE in the default exception handler.  Feels 
spaghetti-ish to me.

What about checking for UIE in the compactionexecutor afterExecute?  That would 
address the DRY problem without violating encapsulation so badly.

Nit: would prefer mutable fields to be private and exposed via getter if 
necessary, e.g. would use Holder.isStopped() here.

 Nodetool commands to query and stop compaction, repair, cleanup and scrub
 -

 Key: CASSANDRA-1740
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1740
 Project: Cassandra
  Issue Type: Improvement
  Components: Tools
Reporter: Chip Salzenberg
Assignee: Vijay
Priority: Minor
  Labels: compaction
 Fix For: 1.0.4

 Attachments: 0001-Patch-to-Stop-compactions-v2.patch, 
 0001-Patch-to-Stop-compactions.patch, CASSANDRA-1740.patch

   Original Estimate: 24h
  Remaining Estimate: 24h

 The only way to stop compaction, repair, cleanup, or scrub in progress is to 
 stop and restart the entire Cassandra server.  Please provide nodetool 
 commands to query whether such things are running, and stop them if they are.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3433) Describe ring is broken

2011-11-16 Thread Jonathan Ellis (Updated) (JIRA)

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

Jonathan Ellis updated CASSANDRA-3433:
--

Component/s: Tools
 API
 Labels: thrift  (was: )

 Describe ring is broken
 ---

 Key: CASSANDRA-3433
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3433
 Project: Cassandra
  Issue Type: Bug
  Components: API, Tools
Affects Versions: 1.0.1
Reporter: Nick Bailey
Assignee: Nick Bailey
  Labels: thrift
 Fix For: 1.0.2

 Attachments: 
 0001-Don-t-use-rpc-address-for-endpoints-field-of-describ.patch


 CASSANDRA-2882 broke describe_ring by causing the 'endpoints' field to 
 contain the rpc address rather than the listen address. the rpc_address 
 belongs in the 'rpc_endpoints' field. Hence the name.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Sylvain Lebresne (Updated) (JIRA)

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

Sylvain Lebresne updated CASSANDRA-3406:


Attachment: 3406.patch

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 3406.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Created] (CASSANDRA-3498) Add same-row contention mode to stress tool

2011-11-16 Thread Jonathan Ellis (Created) (JIRA)
Add same-row contention mode to stress tool
---

 Key: CASSANDRA-3498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3498
 Project: Cassandra
  Issue Type: New Feature
  Components: Tools
Reporter: Jonathan Ellis
Assignee: Pavel Yaskevich
Priority: Minor
 Fix For: 1.1


For CASSANDRA-2893 and other scenarios we'd like to generate non-unique rows to 
insert.  (Maybe we can re-use the same pseudorandom distribution code we 
already have for reads.)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2893) Add row-level isolation

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151310#comment-13151310
 ] 

Jonathan Ellis commented on CASSANDRA-2893:
---

bq. Do we even have an insert test mode that can generate same-row contention?

CASSANDRA-3498

 Add row-level isolation
 ---

 Key: CASSANDRA-2893
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2893
 Project: Cassandra
  Issue Type: Improvement
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
 Fix For: 1.1

 Attachments: 0001-Move-deletion-infos-into-ISortedColumns.patch, 
 0002-Make-memtable-use-CF.addAll.patch, 
 0003-Add-AtomicSortedColumn-and-snapTree.patch, snaptree-0.1-SNAPSHOT.jar


 This could be done using an the atomic ConcurrentMap operations from the 
 Memtable and something like http://code.google.com/p/pcollections/ to replace 
 the ConcurrentSkipListMap in ThreadSafeSortedColumns.  The trick is that 
 pcollections does not provide a SortedMap, so we probably need to write our 
 own.
 Googling [persistent sortedmap] I found 
 http://code.google.com/p/actord/source/browse/trunk/actord/src/main/scala/ff/collection
  (in scala) and http://clojure.org/data_structures#Data Structures-Maps.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151317#comment-13151317
 ] 

Jonathan Ellis commented on CASSANDRA-3406:
---

Honestly I'm not finding the benefits here very compelling in exchange for the 
additional complexity.  We're still doing the same amount of i/o, and 99% as 
much CPU (most of the scrub corruption detection only kicks in if there's an 
exception trying to rewrite).

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 3406.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3362) allow sub-row repair

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151320#comment-13151320
 ] 

Jonathan Ellis commented on CASSANDRA-3362:
---

That's a pretty big ouch for wide-row data models.  If you're doing tens of 
appends per second to one of those, the odds are pretty good that your merkle 
trees will be out of sync at any given instant, and you end up streaming the 
entire row.

 allow sub-row repair
 

 Key: CASSANDRA-3362
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3362
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
  Labels: repair

 With large rows, it would be nice to not have to send an entire row if a 
 small part is out of sync.  Could we use the row index blocks as repair atoms 
 instead of the full row?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151348#comment-13151348
 ] 

Sylvain Lebresne commented on CASSANDRA-3406:
-

Parts of the patch is just refactoring slightly CompactionManager to avoid some 
existing code duplication between performCleanup and performScrub. I don't 
claim such refactoring is a priority or anything, but I would venture that this 
is a good thing in itself. Once that refactoring is done, the new operation is 
literally 4 lines. Then there is the cruft to make it callable from nodetool, 
but overall it doesn't sound like much complexity to me.

Now for the benefits, it is clearly *not* for saving i/o or CPU. The goal is:
* to avoid having an operation called 'scrub' part of the normal upgrade path 
because it's a scary name. Yes, it's just a naming thing (but names are 
important) and yes nobody came complaining about that name but let's be honest, 
scrub was not created for the action of rewriting sstables post-upgrade and the 
name is not adapted.
* scrub does an automatic snapshot. It's totally reasonable for scrub initial 
purpose given the fact it can discard data (albeit corrupted ones), but it's 
just annoying when you've already snapshotted (and maybe move the snapshot in 
some safe place) everything just before your upgrade because you're a good guy.
* scrub can discard data. I think this is something that should never go 
unnoticed. By pushing the use of scrub for case where there is absolutely no 
reason to suspect corruption, it makes it more likely to have it be unnoticed, 
at least at first.

So yes, all of this is mostly details, and sorry to be so verbose for such a 
minor issue but I happen to think that such details are important and that this 
ticket would be an improvement.

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 3406.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151352#comment-13151352
 ] 

Jonathan Ellis commented on CASSANDRA-3406:
---

Can you split it into refactor + newscrub patches?

(Incidently I think having snapshot on the upgrade path is a Very Good Thing 
Indeed, although scrub isn't quite the best way to do that.)

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 3406.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3362) allow sub-row repair

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151353#comment-13151353
 ] 

Sylvain Lebresne commented on CASSANDRA-3362:
-

For the record, I certainly don't pretend it's a good thing. I would even add 
that it will also be a problem in the hypothesis of CASSANDRA-1684.

 allow sub-row repair
 

 Key: CASSANDRA-3362
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3362
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
  Labels: repair

 With large rows, it would be nice to not have to send an entire row if a 
 small part is out of sync.  Could we use the row index blocks as repair atoms 
 instead of the full row?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151359#comment-13151359
 ] 

Sylvain Lebresne commented on CASSANDRA-3406:
-

bq. Can you split it into refactor + newscrub patches?

Will do.

bq. (Incidently I think having snapshot on the upgrade path is a Very Good 
Thing Indeed, although scrub isn't quite the best way to do that.)

I couldn't agree more but it must be before the upgrade, scrub is run after, so 
not the good place at all.

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 3406.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3418) Counter decrements require a space around the minus sign but not around the plus sign

2011-11-16 Thread paul cannon (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151383#comment-13151383
 ] 

paul cannon commented on CASSANDRA-3418:


+1 for the fix

 Counter decrements require a space around the minus sign but not around the 
 plus sign
 -

 Key: CASSANDRA-3418
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3418
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.0.0
Reporter: Kelley Reynolds
Assignee: Pavel Yaskevich
Priority: Minor
  Labels: cql
 Fix For: 1.0.3

 Attachments: CASSANDRA-3418-fix.patch, CASSANDRA-3418.patch


 UPDATE validation_cf_counter SET test=test+1 WHERE id='test_key' = Success
 UPDATE validation_cf_counter SET test=test + 1 WHERE id='test_key' = Success
 UPDATE validation_cf_counter SET test=test - 1 WHERE id='test_key' = Success
 UPDATE validation_cf_counter SET test=test-1 WHERE id='test_key' = Failure 
 (line 1:38 no viable alternative at input 'test')

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1202818 - in /cassandra/branches/cassandra-1.0: CHANGES.txt src/java/org/apache/cassandra/db/DataTracker.java

2011-11-16 Thread jbellis
Author: jbellis
Date: Wed Nov 16 18:07:39 2011
New Revision: 1202818

URL: http://svn.apache.org/viewvc?rev=1202818view=rev
Log:
fix liveSize stat when sstables are removed
patch by Jackson Chung; reviewed by jbellis and slebresne for CASSANDRA-3496

Modified:
cassandra/branches/cassandra-1.0/CHANGES.txt

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/DataTracker.java

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1202818r1=1202817r2=1202818view=diff
==
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Wed Nov 16 18:07:39 2011
@@ -1,3 +1,7 @@
+1.0.4
+ * fix liveSize stat when sstables are removed (CASSANDRA-3496)
+
+
 1.0.3
  * revert name-based query defragmentation aka CASSANDRA-2503 (CASSANDRA-3491)
  * fix invalidate-related test failures (CASSANDRA-3437)
@@ -27,6 +31,7 @@ Merged from 0.8:
  * `describe ring` command for CLI (CASSANDRA-3220)
  * (Hadoop) skip empty rows when entire row is requested, redux 
(CASSANDRA-2855)
 
+
 1.0.2
  * defragment rows for name-based queries under STCS (CASSANDRA-2503)
  * cleanup usage of StorageService.setMode() (CASANDRA-3388)

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/DataTracker.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/DataTracker.java?rev=1202818r1=1202817r2=1202818view=diff
==
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/DataTracker.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/DataTracker.java
 Wed Nov 16 18:07:39 2011
@@ -298,9 +298,9 @@ public class DataTracker
 if (logger.isDebugEnabled())
 logger.debug(String.format(removing %s from list of files 
tracked for %s.%s,
 sstable.descriptor, cfstore.table.name, 
cfstore.getColumnFamilyName()));
+liveSize.addAndGet(-sstable.bytesOnDisk());
 sstable.markCompacted();
 sstable.releaseReference();
-liveSize.addAndGet(-sstable.bytesOnDisk());
 }
 }
 




[jira] [Resolved] (CASSANDRA-3496) Load from `nodetool ring` does not update after cleanup.

2011-11-16 Thread Jonathan Ellis (Resolved) (JIRA)

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

Jonathan Ellis resolved CASSANDRA-3496.
---

Resolution: Fixed
  Assignee: Jackson Chung  (was: Jonathan Ellis)

Committed.

(Tried to catch the bug w/ a check on live size in RecoveryManagerTruncateTest, 
but I couldn't reproduce the race without adding sleeps inside the deletion 
task.  So I just committed the patch as above.)

 Load from `nodetool ring` does not update after cleanup.
 

 Key: CASSANDRA-3496
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3496
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.0
Reporter: Benjamin Coverston
Assignee: Jackson Chung
 Fix For: 1.0.4

 Attachments: 3496.txt


 Repro:
 Bring up a node.
 Insert 1M rows:
 127.0.0.1   datacenter1 rack1   Up Normal  406.92 MB   
 100.00% 77747037169725419723056812679314618801
 (Already looks wrong, 406.92 is higher than I'm used to seeing from a single 
 run of stress)
 Bootstrap a second node into the cluster:
 162877269496252595336256012556853953561
 127.0.0.1   datacenter1 rack1   Up Normal  407.03 MB   49.96% 
  77747037169725419723056812679314618801
 127.0.0.2   datacenter1 rack1   Up Normal  157.91 MB   50.04% 
  162877269496252595336256012556853953561
 Cleanup
 162877269496252595336256012556853953561
 127.0.0.1   datacenter1 rack1   Up Normal  551.2 MB   49.96%  
 77747037169725419723056812679314618801
 127.0.0.2   datacenter1 rack1   Up Normal  157.91 MB   50.04% 
  162877269496252595336256012556853953561
 Looks like each operation that adds and removes SSTables only adds to the 
 total and doesn't remove the old sstables from the total size count.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2975) Upgrade MurmurHash to version 3

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151392#comment-13151392
 ] 

Jonathan Ellis commented on CASSANDRA-2975:
---

Really, JIT doesn't inline the rot164?  Did you warm up the JVM before timing 
things?  
http://stackoverflow.com/questions/504103/how-do-i-write-a-correct-micro-benchmark-in-java

 Upgrade MurmurHash to version 3
 ---

 Key: CASSANDRA-2975
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2975
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brian Lindauer
Assignee: Brian Lindauer
Priority: Trivial
  Labels: lhf
 Fix For: 1.1

 Attachments: 
 0001-Convert-BloomFilter-to-use-MurmurHash-v3-instead-of-.patch, 
 0002-Backwards-compatibility-with-files-using-Murmur2-blo.patch


 MurmurHash version 3 was finalized on June 3. It provides an enormous speedup 
 and increased robustness over version 2, which is implemented in Cassandra. 
 Information here:
 http://code.google.com/p/smhasher/
 The reference implementation is here:
 http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp?spec=svn136r=136
 I have already done the work to port the (public domain) reference 
 implementation to Java in the MurmurHash class and updated the BloomFilter 
 class to use the new implementation:
 https://github.com/lindauer/cassandra/commit/cea6068a4a3e5d7d9509335394f9ef3350d37e93
 Apart from the faster hash time, the new version only requires one call to 
 hash() rather than 2, since it returns 128 bits of hash instead of 64.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1202821 - in /cassandra/branches/cassandra-1.0: CHANGES.txt src/java/org/apache/cassandra/tools/NodeCmd.java

2011-11-16 Thread jbellis
Author: jbellis
Date: Wed Nov 16 18:22:32 2011
New Revision: 1202821

URL: http://svn.apache.org/viewvc?rev=1202821view=rev
Log:
add bloom filter FP rates to nodetool cfstats
patch by Vijay; reviewed by jbellis for CASSANDRA-3347

Modified:
cassandra/branches/cassandra-1.0/CHANGES.txt

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/NodeCmd.java

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1202821r1=1202820r2=1202821view=diff
==
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Wed Nov 16 18:22:32 2011
@@ -1,5 +1,6 @@
 1.0.4
  * fix liveSize stat when sstables are removed (CASSANDRA-3496)
+ * add bloom filter FP rates to nodetool cfstats (CASSANDRA-3347)
 
 
 1.0.3

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/NodeCmd.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/NodeCmd.java?rev=1202821r1=1202820r2=1202821view=diff
==
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/NodeCmd.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/NodeCmd.java
 Wed Nov 16 18:22:32 2011
@@ -461,6 +461,8 @@ public class NodeCmd
 outs.println(\t\tWrite Count:  + cfstore.getWriteCount());
 outs.println(\t\tWrite Latency:  + String.format(%01.3f, 
cfstore.getRecentWriteLatencyMicros() / 1000) +  ms.);
 outs.println(\t\tPending Tasks:  + 
cfstore.getPendingTasks());
+outs.println(\t\tBloom Filter False Postives:  + 
cfstore.getBloomFilterFalsePositives());
+outs.println(\t\tBloom Filter False Ratio:  + 
String.format(%01.5f, cfstore.getRecentBloomFilterFalseRatio()));
 
 InstrumentingCacheMBean keyCacheMBean = 
probe.getKeyCacheMBean(tableName, cfstore.getColumnFamilyName());
 if (keyCacheMBean.getCapacity()  0)




[jira] [Created] (CASSANDRA-3499) Make the 'load' interface pluggable

2011-11-16 Thread Chris Goffinet (Created) (JIRA)
Make the 'load' interface pluggable
---

 Key: CASSANDRA-3499
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3499
 Project: Cassandra
  Issue Type: Bug
Reporter: Chris Goffinet
Priority: Minor


We should make the 'Load' attribute of the cluster at least pluggable. One use 
case we had was we could build a plugin that was specific to us, that could be 
tied directly into our time series database we have for all of our 
infrastructure. This would allow us to populate and expose more data per node 
instead of needing to gossip this data around (CPU/Network/Memory/etc)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2503) Eagerly re-write data at read time (superseding)

2011-11-16 Thread Jonathan Ellis (Updated) (JIRA)

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

Jonathan Ellis updated CASSANDRA-2503:
--

Attachment: 2503-v3.txt

v3 adds boolean updateIndexes to Table.apply; this is safe to turn off for 
the defragment write, since we're updating w/ exactly the existing data, 
timestamp and all.

Also adds a check for {{cfs.getMinimumCompactionThreshold()  0}}.

 Eagerly re-write data at read time (superseding)
 --

 Key: CASSANDRA-2503
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2503
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Stu Hood
Assignee: Jonathan Ellis
  Labels: compaction, performance
 Fix For: 1.1

 Attachments: 2503-v2.txt, 2503-v3.txt, 2503.txt


 Once CASSANDRA-2498 is implemented, it will be possible to implement an 
 optimization to eagerly rewrite (supersede) data at read time. If a 
 successful read needed to hit more than a certain threshold of sstables, we 
 can eagerly rewrite it in a new sstable, and 2498 will allow only that file 
 to be accessed. This basic approach would improve read performance 
 considerably, but would cause a lot of duplicate data to be written, and 
 would make compaction's work more necessary.
 Augmenting the basic idea, if when we superseded data in a file we marked it 
 as superseded somehow, the next compaction that touched that file could 
 remove the data. Since our file format is immutable, the values that a 
 particular sstable superseded could be recorded in a component of that 
 sstable. If we always supersede at the block level (as defined by 
 CASSANDRA-674 or CASSANDRA-47), then the list of superseded blocks could be 
 represented using a generation number and a bitmap of block numbers. Since 
 2498 would already allow for sstables to be eliminated due to timestamps, 
 this information would probably only be used at compaction time (by loading 
 all superseding information in the system for the sstables that are being 
 compacted).
 Initially described on 
 [1608|https://issues.apache.org/jira/secure/EditComment!default.jspa?id=12477095commentId=12920353].

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1202834 - in /cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql: Cql.g Term.java UpdateStatement.java

2011-11-16 Thread xedin
Author: xedin
Date: Wed Nov 16 18:50:44 2011
New Revision: 1202834

URL: http://svn.apache.org/viewvc?rev=1202834view=rev
Log:
Fix to properly support update of the counter with expression like 'field = 
field - -num'
patch by Pavel Yaskevich; reviewed by Paul Canon for CASSANDRA-3418

Modified:
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Cql.g
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Term.java

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/UpdateStatement.java

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Cql.g
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Cql.g?rev=1202834r1=1202833r2=1202834view=diff
==
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Cql.g 
(original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Cql.g 
Wed Nov 16 18:50:44 2011
@@ -470,6 +470,7 @@ termPairWithOperation[MapTerm, Operatio
   | c=term ( '+'  v=term { columns.put(key, new 
Operation(c, org.apache.cassandra.cql.Operation.OperationType.PLUS, v)); }
 | op='-'? v=intTerm
   { validateMinusSupplied(op, v, input);
+if (op != null) v = new 
Term(-(Long.valueOf(v.getText())), v.getType());
 columns.put(key, new Operation(c, 
org.apache.cassandra.cql.Operation.OperationType.MINUS, v)); } ))
 ;
 

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Term.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Term.java?rev=1202834r1=1202833r2=1202834view=diff
==
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Term.java 
(original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/Term.java 
Wed Nov 16 18:50:44 2011
@@ -54,6 +54,12 @@ public class Term
 this.text = text == null ?  : text;
 this.type = type;
 }
+
+public Term(long value, TermType type)
+{
+this.text = String.valueOf(value);
+this.type = type;
+}
 
 protected Term()
 {

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/UpdateStatement.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/UpdateStatement.java?rev=1202834r1=1202833r2=1202834view=diff
==
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/UpdateStatement.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/cql/UpdateStatement.java
 Wed Nov 16 18:50:44 2011
@@ -217,11 +217,6 @@ public class UpdateStatement extends Abs
 try
 {
 value = Long.parseLong(op.b.getText());
-
-if (op.type == OperationType.MINUS)
-{
-if (value  0) value *= -1;
-}
 }
 catch (NumberFormatException e)
 {




[jira] [Commented] (CASSANDRA-3418) Counter decrements require a space around the minus sign but not around the plus sign

2011-11-16 Thread Pavel Yaskevich (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3418?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151417#comment-13151417
 ] 

Pavel Yaskevich commented on CASSANDRA-3418:


Committed.

 Counter decrements require a space around the minus sign but not around the 
 plus sign
 -

 Key: CASSANDRA-3418
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3418
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.0.0
Reporter: Kelley Reynolds
Assignee: Pavel Yaskevich
Priority: Minor
  Labels: cql
 Fix For: 1.0.3

 Attachments: CASSANDRA-3418-fix.patch, CASSANDRA-3418.patch


 UPDATE validation_cf_counter SET test=test+1 WHERE id='test_key' = Success
 UPDATE validation_cf_counter SET test=test + 1 WHERE id='test_key' = Success
 UPDATE validation_cf_counter SET test=test - 1 WHERE id='test_key' = Success
 UPDATE validation_cf_counter SET test=test-1 WHERE id='test_key' = Failure 
 (line 1:38 no viable alternative at input 'test')

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Sylvain Lebresne (Updated) (JIRA)

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

Sylvain Lebresne updated CASSANDRA-3406:


Attachment: (was: 3406.patch)

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 0001-Refactor-to-avoid-code-duplication.patch, 
 0002-Add-upgradesstables-command.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Sylvain Lebresne (Updated) (JIRA)

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

Sylvain Lebresne updated CASSANDRA-3406:


Attachment: 0002-Add-upgradesstables-command.patch
0001-Refactor-to-avoid-code-duplication.patch

Patches attached with the refactor in its own patch.

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 0001-Refactor-to-avoid-code-duplication.patch, 
 0002-Add-upgradesstables-command.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3458) Add cqlsh to deb and rpm packaging

2011-11-16 Thread paul cannon (Updated) (JIRA)

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

paul cannon updated CASSANDRA-3458:
---

Attachment: 3458.patch-debian.txt

This adds cqlsh to the Debian packaging, and adds the necessary dependencies on 
python and python-cql for cassandra.

NOTE that the next time we build and distribute official cassandra debs, we 
should now make python-cql and python-thrift available in the same area.

 Add cqlsh to deb and rpm packaging
 --

 Key: CASSANDRA-3458
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3458
 Project: Cassandra
  Issue Type: Improvement
  Components: Packaging
Reporter: paul cannon
Assignee: paul cannon
Priority: Minor
 Attachments: 3458.patch-debian.txt


 Once (if?) CASSANDRA-3188 is committed, cqlsh will be distributed with the 
 cassandra tarballs, but not in the debs or rpms. (Actually, it looks like the 
 cqlsh script will get put in the rpm by accident, but not its associated 
 libraries).
 We might even want to break cqlsh out into a separate package from the same 
 source, so that it can be installed on machines only intended to be used as 
 cassandra clients, not servers.
 Maybe that doesn't make sense without including nodetool and cassandra-cli 
 too, and then we'd need yet another package to hold the jars that are common 
 between cassandra and cassandra-client... maybe it's not worth it for now.
 Either way, make sure installing cassandra debs and rpms ends up with a 
 working cqlsh.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3458) Add cqlsh to deb and rpm packaging

2011-11-16 Thread paul cannon (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3458?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151428#comment-13151428
 ] 

paul cannon commented on CASSANDRA-3458:


I'm going to just punt on adding cqlsh to the rpm for now. The existing spec 
file is already way out of date and probably unworking, and it doesn't seem 
right to put in the work to fix it up under the rubric of this ticket. Can we 
just commit the addition to the deb for now?

 Add cqlsh to deb and rpm packaging
 --

 Key: CASSANDRA-3458
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3458
 Project: Cassandra
  Issue Type: Improvement
  Components: Packaging
Reporter: paul cannon
Assignee: paul cannon
Priority: Minor
 Attachments: 3458.patch-debian.txt


 Once (if?) CASSANDRA-3188 is committed, cqlsh will be distributed with the 
 cassandra tarballs, but not in the debs or rpms. (Actually, it looks like the 
 cqlsh script will get put in the rpm by accident, but not its associated 
 libraries).
 We might even want to break cqlsh out into a separate package from the same 
 source, so that it can be installed on machines only intended to be used as 
 cassandra clients, not servers.
 Maybe that doesn't make sense without including nodetool and cassandra-cli 
 too, and then we'd need yet another package to hold the jars that are common 
 between cassandra and cassandra-client... maybe it's not worth it for now.
 Either way, make sure installing cassandra debs and rpms ends up with a 
 working cqlsh.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3371) Cassandra inferred schema and actual data don't match

2011-11-16 Thread Pete Warden (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151476#comment-13151476
 ] 

Pete Warden commented on CASSANDRA-3371:


That makes sense, and looks very approachable, thanks. I'll keep an eye on the 
comments here so I can give it a try as soon as a prototype is available. 
Thanks again Brandon, Jacob and Jeremy for your hard work on this one.

 Cassandra inferred schema and actual data don't match
 -

 Key: CASSANDRA-3371
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3371
 Project: Cassandra
  Issue Type: Bug
  Components: Hadoop
Affects Versions: 0.8.7
Reporter: Pete Warden
Assignee: Brandon Williams
 Attachments: 3371-v2.txt, 3371-v3.txt, pig.diff


 It's looking like there may be a mismatch between the schema that's being 
 reported by the latest CassandraStorage.java, and the data that's actually 
 returned. Here's an example:
 rows = LOAD 'cassandra://Frap/PhotoVotes' USING CassandraStorage();
 DESCRIBE rows;
 rows: {key: chararray,columns: {(name: chararray,value: 
 bytearray,photo_owner: chararray,value_photo_owner: bytearray,pid: 
 chararray,value_pid: bytearray,matched_string: 
 chararray,value_matched_string: bytearray,src_big: chararray,value_src_big: 
 bytearray,time: chararray,value_time: bytearray,vote_type: 
 chararray,value_vote_type: bytearray,voter: chararray,value_voter: 
 bytearray)}}
 DUMP rows;
 (691831038_1317937188.48955,{(photo_owner,1596090180),(pid,6855155124568798560),(matched_string,),(src_big,),(time,Thu
  Oct 06 14:39:48 -0700 2011),(vote_type,album_dislike),(voter,691831038)})
 getSchema() is reporting the columns as an inner bag of tuples, each of which 
 contains 16 values. In fact, getNext() seems to return an inner bag 
 containing 7 tuples, each of which contains two values. 
 It appears that things got out of sync with this change:
 http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/contrib/pig/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java?r1=1177083r2=1177082pathrev=1177083
 See more discussion at:
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/pig-cassandra-problem-quot-Incompatible-field-schema-quot-error-tc6882703.html
 Here's a patch I ended up creating for my own use, which gives the results I 
 need (though it doesn't handle super-columns):
 DESCRIBE rows;
 rows: {cassandra_key: chararray,photo_owner: bytearray,pid: 
 bytearray,place_matched_string: bytearray,src_big: bytearray,time: 
 bytearray,vote_type: bytearray,voter: bytearray}
 Index: 
 contrib/pig/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
 ===
 --- 
 contrib/pig/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
 (revision 1185044)
 +++ 
 contrib/pig/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
 (working copy)
 @@ -26,7 +26,7 @@
  import org.apache.cassandra.db.marshal.IntegerType;
  import org.apache.cassandra.db.marshal.TypeParser;
  import org.apache.cassandra.thrift.*;
 -import org.apache.cassandra.utils.Hex;
 +import org.apache.cassandra.utils.FBUtilities;
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
  
 @@ -122,15 +122,15 @@
  assert key != null  cf != null;
  
  // and wrap it in a tuple
 - Tuple tuple = TupleFactory.getInstance().newTuple(2);
 + Tuple tuple = TupleFactory.getInstance().newTuple(cf.size()+1);
  ArrayListTuple columns = new ArrayListTuple();
 -tuple.set(0, new DataByteArray(key.array(), 
 key.position()+key.arrayOffset(), key.limit()+key.arrayOffset()));
 +int tupleIndex = 0;
 +tuple.set(tupleIndex++, new DataByteArray(key.array(), 
 key.position()+key.arrayOffset(), key.limit()+key.arrayOffset()));
  for (Map.EntryByteBuffer, IColumn entry : cf.entrySet())
  {
 -columns.add(columnToTuple(entry.getKey(), entry.getValue(), 
 cfDef));
 +tuple.set(tupleIndex++, columnToTuple(entry.getKey(), 
 entry.getValue(), cfDef));
  }
  
 -tuple.set(1, new DefaultDataBag(columns));
  return tuple;
  }
  catch (InterruptedException e)
 @@ -139,30 +139,22 @@
  }
  }
  
 -private Tuple columnToTuple(ByteBuffer name, IColumn col, CfDef cfDef) 
 throws IOException
 +private Object columnToTuple(ByteBuffer name, IColumn col, CfDef cfDef) 
 throws IOException
  {
 -Tuple pair = TupleFactory.getInstance().newTuple(2);
  ListAbstractType marshallers = getDefaultMarshallers(cfDef);
  

[jira] [Commented] (CASSANDRA-3483) Support bringing up a new datacenter to existing cluster without repair

2011-11-16 Thread Sylvain Lebresne (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151499#comment-13151499
 ] 

Sylvain Lebresne commented on CASSANDRA-3483:
-

I think it wouldn't be crazy and actually very (very) simple to add a new 
nodetool command (rebuild?) that would basically have the node asks the other 
replicas to stream all there data to him (for the correct ranges obviously). In 
other words, a command that force the streaming part of bootstrap without all 
the join ring part. Or another way to say is to have the streaming part of a 
repair but without the validation part.

The method to add a new DC would be the same as today except that repair would 
be replaced by this new operation.

 Support bringing up a new datacenter to existing cluster without repair
 ---

 Key: CASSANDRA-3483
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3483
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.2
Reporter: Chris Goffinet

 Was talking to Brandon in irc, and we ran into a case where we want to bring 
 up a new DC to an existing cluster. He suggested from jbellis the way to do 
 it currently was set strategy options of dc2:0, then add the nodes. After the 
 nodes are up, change the RF of dc2, and run repair. 
 I'd like to avoid a repair as it runs AES and is a bit more intense than how 
 bootstrap works currently by just streaming ranges from the SSTables. Would 
 it be possible to improve this functionality (adding a new DC to existing 
 cluster) than the proposed method? We'd be happy to do a patch if we got some 
 input on the best way to go about it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3483) Support bringing up a new datacenter to existing cluster without repair

2011-11-16 Thread Brandon Williams (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151503#comment-13151503
 ] 

Brandon Williams commented on CASSANDRA-3483:
-

We would still need to put the nodes into a 'bootstrap' state to get incoming 
writes forwarded to them, otherwise you have to repair in the end anyway.

 Support bringing up a new datacenter to existing cluster without repair
 ---

 Key: CASSANDRA-3483
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3483
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.2
Reporter: Chris Goffinet

 Was talking to Brandon in irc, and we ran into a case where we want to bring 
 up a new DC to an existing cluster. He suggested from jbellis the way to do 
 it currently was set strategy options of dc2:0, then add the nodes. After the 
 nodes are up, change the RF of dc2, and run repair. 
 I'd like to avoid a repair as it runs AES and is a bit more intense than how 
 bootstrap works currently by just streaming ranges from the SSTables. Would 
 it be possible to improve this functionality (adding a new DC to existing 
 cluster) than the proposed method? We'd be happy to do a patch if we got some 
 input on the best way to go about it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3406) Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't intended to.

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151506#comment-13151506
 ] 

Jonathan Ellis commented on CASSANDRA-3406:
---

+1

 Create a nodetool upgrade_sstables to avoid using scrubs for tasks it wasn't 
 intended to.
 -

 Key: CASSANDRA-3406
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3406
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Affects Versions: 1.0.0
Reporter: Sylvain Lebresne
Assignee: Sylvain Lebresne
Priority: Trivial
 Fix For: 1.0.4

 Attachments: 0001-Refactor-to-avoid-code-duplication.patch, 
 0002-Add-upgradesstables-command.patch


 Scrub was intended to check a data file is not corrupted and to try to 
 correct some form of corruption and discard the data when it can't repair. 
 But we are now using it also for:
 * major upgrade, to have sstable in the new data format for streaming sake 
 (that one could be fixed independently by supporting old format during 
 streaming)
 * to force the compaction of existing sstables after changing the compression 
 algorithm
 We should probably provide a separate tool/command for those two last tasks 
 since:
 * we could have a better name, like upgrade_sstables or rewrite_sstables for 
 that operation
 * we could avoid the automatic snapshot that scrub does (and is not expected 
 by users for those operations)
 * make it slightly quicker/simpler by avoiding the corruption detection code

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-3186) nodetool should not NPE when rack/dc info is not yet available

2011-11-16 Thread Alex Araujo (Updated) (JIRA)

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

Alex Araujo updated CASSANDRA-3186:
---

Attachment: cassandra-0.8-3186.txt

Merges 1.0's Ec2Snitch which does the right thing

 nodetool should not NPE when rack/dc info is not yet available
 --

 Key: CASSANDRA-3186
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3186
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.8.1
Reporter: Brandon Williams
Assignee: Brandon Williams
Priority: Minor
  Labels: lhf
 Fix For: 0.8.8

 Attachments: cassandra-0.8-3186.txt


 As the title says.  What happens is the persisted ring is loaded, but if 
 those nodes are down and you're using a snitch like ec2 that gets rack/dc 
 info from gossip, nodetool NPEs instead of showing that the node is down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1202892 - /cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java

2011-11-16 Thread brandonwilliams
Author: brandonwilliams
Date: Wed Nov 16 21:44:54 2011
New Revision: 1202892

URL: http://svn.apache.org/viewvc?rev=1202892view=rev
Log:
Set default rack/dc in ec2snitch to avoid NPEs.
Patch by Alex Araujo, reviewed by brandonwilliams for CASSANDRA-3186.

Modified:

cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java

Modified: 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java?rev=1202892r1=1202891r2=1202892view=diff
==
--- 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java
 (original)
+++ 
cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java
 Wed Nov 16 21:44:54 2011
@@ -44,6 +44,8 @@ public class Ec2Snitch extends AbstractN
 {
 protected static Logger logger = LoggerFactory.getLogger(Ec2Snitch.class);
 protected static final String ZONE_NAME_QUERY_URL = 
http://169.254.169.254/latest/meta-data/placement/availability-zone;;
+private static final String DEFAULT_DC = UNKNOWN-DC;
+private static final String DEFAULT_RACK = UNKNOWN-RACK;
 protected String ec2zone;
 protected String ec2region;
 
@@ -83,14 +85,20 @@ public class Ec2Snitch extends AbstractN
 {
 if (endpoint.equals(FBUtilities.getLocalAddress()))
 return ec2zone;
-return 
Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RACK).value;
+EndpointState state = 
Gossiper.instance.getEndpointStateForEndpoint(endpoint);
+if (null == state || null == 
state.getApplicationState(ApplicationState.RACK))
+return DEFAULT_RACK;
+return state.getApplicationState(ApplicationState.RACK).value;
 }
 
 public String getDatacenter(InetAddress endpoint)
 {
 if (endpoint.equals(FBUtilities.getLocalAddress()))
 return ec2region;
-return 
Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.DC).value;
+EndpointState state = 
Gossiper.instance.getEndpointStateForEndpoint(endpoint);
+if (null == state || null == 
state.getApplicationState(ApplicationState.DC))
+return DEFAULT_DC;
+return state.getApplicationState(ApplicationState.DC).value;
 }
 
 @Override




[jira] [Resolved] (CASSANDRA-3114) After Choosing EC2Snitch you can't migrate off w/o a full cluster restart

2011-11-16 Thread Brandon Williams (Resolved) (JIRA)

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

Brandon Williams resolved CASSANDRA-3114.
-

Resolution: Fixed

Closing, see CASSANDRA-3186

 After Choosing EC2Snitch you can't migrate off w/o a full cluster restart
 -

 Key: CASSANDRA-3114
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3114
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 0.7.8, 0.8.4
Reporter: Benjamin Coverston

 Once you choose the Ec2Snitch the gossip messages will trigger this exception 
 if you try to move (for example) to the property file snitch:
 ERROR [pool-2-thread-11] 2011-08-30 16:38:06,935 Cassandra.java (line 3041) 
 Internal error processing get_slice 
 java.lang.NullPointerException 
 at org.apache.cassandra.locator.Ec2Snitch.getDatacenter(Ec2Snitch.java:84) 
 at 
 org.apache.cassandra.locator.DynamicEndpointSnitch.getDatacenter(DynamicEndpointSnitch.java:122)
  
 at 
 org.apache.cassandra.service.DatacenterReadCallback.assureSufficientLiveNodes(DatacenterReadCallback.java:77)
  
 at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:516) 
 at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:480) 
 at 
 org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:109)
  
 at 
 org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:263)
  
 at 
 org.apache.cassandra.thrift.CassandraServer.multigetSliceInternal(CassandraServer.java:345)
  
 at 
 org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:306)
  
 at 
 org.apache.cassandra.thrift.Cassandra$Processor$get_slice.process(Cassandra.java:3033)
  
 at 
 org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2889) 
 at 
 org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:187)
  
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
  
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
  
 at java.lang.Thread.run(Thread.java:662)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1202903 - /cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java

2011-11-16 Thread jbellis
Author: jbellis
Date: Wed Nov 16 22:02:03 2011
New Revision: 1202903

URL: http://svn.apache.org/viewvc?rev=1202903view=rev
Log:
actually test persistent-ness in testPeristentStatistics

Modified:

cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java

Modified: 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java?rev=1202903r1=1202902r2=1202903view=diff
==
--- 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
 Wed Nov 16 22:02:03 2011
@@ -148,6 +148,9 @@ public class SSTableReaderTest extends C
 rm.apply();
 }
 store.forceBlockingFlush();
+
+store.clearUnsafe();
+store.loadNewSSTables();
 assert store.getMaxRowSize() != 0;
 }
 




svn commit: r1202904 - in /cassandra/branches/cassandra-1.0: ./ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/sstable/

2011-11-16 Thread jbellis
Author: jbellis
Date: Wed Nov 16 22:07:35 2011
New Revision: 1202904

URL: http://svn.apache.org/viewvc?rev=1202904view=rev
Log:
record partitioner in sstable metadata component
patch by Yuki Morishita and jbellis for CASSANDRA-3393

Modified:
cassandra/branches/cassandra-1.0/CHANGES.txt

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java

cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableReader.java

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1202904r1=1202903r2=1202904view=diff
==
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Wed Nov 16 22:07:35 2011
@@ -1,6 +1,7 @@
 1.0.4
  * fix liveSize stat when sstables are removed (CASSANDRA-3496)
  * add bloom filter FP rates to nodetool cfstats (CASSANDRA-3347)
+ * record partitioner in sstable metadata component (CASSANDRA-3393)
 
 
 1.0.3

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java?rev=1202904r1=1202903r2=1202904view=diff
==
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java
 Wed Nov 16 22:07:35 2011
@@ -62,7 +62,6 @@ public class SystemTable
 private static final ByteBuffer TOKEN = ByteBufferUtil.bytes(Token);
 private static final ByteBuffer GENERATION = 
ByteBufferUtil.bytes(Generation);
 private static final ByteBuffer CLUSTERNAME = 
ByteBufferUtil.bytes(ClusterName);
-private static final ByteBuffer PARTITIONER = 
ByteBufferUtil.bytes(Partioner);
 private static final ByteBuffer CURRENT_LOCAL_NODE_ID_KEY = 
ByteBufferUtil.bytes(CurrentLocal);
 private static final ByteBuffer ALL_LOCAL_NODE_ID_KEY = 
ByteBufferUtil.bytes(Local);
 
@@ -247,7 +246,7 @@ public class SystemTable
  * One of three things will happen if you try to read the system table:
  * 1. files are present and you can read them: great
  * 2. no files are there: great (new node is assumed)
- * 3. files are present but you can't read them: bad (suspect that the 
partitioner was changed).
+ * 3. files are present but you can't read them: bad
  * @throws ConfigurationException
  */
 public static void checkHealth() throws ConfigurationException, IOException
@@ -260,28 +259,26 @@ public class SystemTable
 catch (AssertionError err)
 {
 // this happens when a user switches from OPP to RP.
-ConfigurationException ex = new ConfigurationException(Could not 
read system table. Did you change partitioners?);
+ConfigurationException ex = new ConfigurationException(Could not 
read system table!);
 ex.initCause(err);
 throw ex;
 }
 
 SortedSetByteBuffer cols = new 
TreeSetByteBuffer(BytesType.instance);
-cols.add(PARTITIONER);
 cols.add(CLUSTERNAME);
 QueryFilter filter = 
QueryFilter.getNamesFilter(decorate(LOCATION_KEY), new QueryPath(STATUS_CF), 
cols);
 ColumnFamily cf = 
table.getColumnFamilyStore(STATUS_CF).getColumnFamily(filter);
 
 if (cf == null)
 {
-// this is either a brand new node (there will be no files), or 
the partitioner was changed from RP to OPP.
+// this is a brand new node
 ColumnFamilyStore cfs = table.getColumnFamilyStore(STATUS_CF);
 if (!cfs.getSSTables().isEmpty())
-throw new ConfigurationException(Found system table files, 
but they couldn't be loaded. Did you change the partitioner?);
+throw new ConfigurationException(Found system table files, 
but they couldn't be loaded!);
 
 // no system files.  this is a new node.
 RowMutation rm = new RowMutation(Table.SYSTEM_TABLE, LOCATION_KEY);
 cf = ColumnFamily.create(Table.SYSTEM_TABLE, 
SystemTable.STATUS_CF);
-cf.addColumn(new Column(PARTITIONER, 
ByteBufferUtil.bytes(DatabaseDescriptor.getPartitioner().getClass().getName()), 
FBUtilities.timestampMicros()));
 cf.addColumn(new Column(CLUSTERNAME, 
ByteBufferUtil.bytes(DatabaseDescriptor.getClusterName()), 
FBUtilities.timestampMicros()));
 rm.add(cf);
 rm.apply();
@@ -290,12 +287,8 @@ public class SystemTable
 }
 
 
- 

svn commit: r1202905 - /cassandra/branches/cassandra-1.0/CHANGES.txt

2011-11-16 Thread jbellis
Author: jbellis
Date: Wed Nov 16 22:08:09 2011
New Revision: 1202905

URL: http://svn.apache.org/viewvc?rev=1202905view=rev
Log:
fix ticket number in CHANGES

Modified:
cassandra/branches/cassandra-1.0/CHANGES.txt

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1202905r1=1202904r2=1202905view=diff
==
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Wed Nov 16 22:08:09 2011
@@ -1,7 +1,7 @@
 1.0.4
  * fix liveSize stat when sstables are removed (CASSANDRA-3496)
  * add bloom filter FP rates to nodetool cfstats (CASSANDRA-3347)
- * record partitioner in sstable metadata component (CASSANDRA-3393)
+ * record partitioner in sstable metadata component (CASSANDRA-3407)
 
 
 1.0.3




[jira] [Commented] (CASSANDRA-2434) range movements can violate consistency

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151570#comment-13151570
 ] 

Jonathan Ellis commented on CASSANDRA-2434:
---

Coming back to this after the 1.0 scramble.

It sounds like you have a reasonable solution here, is there any reason not to 
implement it for 1.1?

 range movements can violate consistency
 ---

 Key: CASSANDRA-2434
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2434
 Project: Cassandra
  Issue Type: Bug
Reporter: Peter Schuller
Assignee: paul cannon
 Fix For: 1.0.4

 Attachments: 2434-3.patch.txt, 2434-testery.patch.txt


 My reading (a while ago) of the code indicates that there is no logic 
 involved during bootstrapping that avoids consistency level violations. If I 
 recall correctly it just grabs neighbors that are currently up.
 There are at least two issues I have with this behavior:
 * If I have a cluster where I have applications relying on QUORUM with RF=3, 
 and bootstrapping complete based on only one node, I have just violated the 
 supposedly guaranteed consistency semantics of the cluster.
 * Nodes can flap up and down at any time, so even if a human takes care to 
 look at which nodes are up and things about it carefully before 
 bootstrapping, there's no guarantee.
 A complication is that not only does it depend on use-case where this is an 
 issue (if all you ever do you do at CL.ONE, it's fine); even in a cluster 
 which is otherwise used for QUORUM operations you may wish to accept 
 less-than-quorum nodes during bootstrap in various emergency situations.
 A potential easy fix is to have bootstrap take an argument which is the 
 number of hosts to bootstrap from, or to assume QUORUM if none is given.
 (A related concern is bootstrapping across data centers. You may *want* to 
 bootstrap to a local node and then do a repair to avoid sending loads of data 
 across DC:s while still achieving consistency. Or even if you don't care 
 about the consistency issues, I don't think there is currently a way to 
 bootstrap from local nodes only.)
 Thoughts?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




svn commit: r1202918 - in /cassandra/trunk: ./ contrib/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/cql/ src/java/org/apache/cassandra/db/ src/java/org/apache/

2011-11-16 Thread jbellis
Author: jbellis
Date: Wed Nov 16 22:49:20 2011
New Revision: 1202918

URL: http://svn.apache.org/viewvc?rev=1202918view=rev
Log:
merge from 1.0

Modified:
cassandra/trunk/   (props changed)
cassandra/trunk/CHANGES.txt
cassandra/trunk/contrib/   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java
   (props changed)

cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
   (props changed)
cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g
cassandra/trunk/src/java/org/apache/cassandra/cql/Term.java
cassandra/trunk/src/java/org/apache/cassandra/cql/UpdateStatement.java
cassandra/trunk/src/java/org/apache/cassandra/db/DataTracker.java
cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java

cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java

cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java

Propchange: cassandra/trunk/
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Nov 16 22:49:20 2011
@@ -4,7 +4,7 @@
 
/cassandra/branches/cassandra-0.8:1090934-1125013,1125019-1198724,1198726-1199259,1199284,1200226,1200471
 /cassandra/branches/cassandra-0.8.0:1125021-1130369
 /cassandra/branches/cassandra-0.8.1:1101014-1125018
-/cassandra/branches/cassandra-1.0:1167085-1202429
+/cassandra/branches/cassandra-1.0:1167085-1202912
 
/cassandra/branches/cassandra-1.0.0:1167104-1167229,1167232-1181093,1181741,1181816,1181820,1182951,1183243
 /cassandra/tags/cassandra-0.7.0-rc3:1051699-1053689
 /cassandra/tags/cassandra-0.8.0-rc1:1102511-1125020

Modified: cassandra/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1202918r1=1202917r2=1202918view=diff
==
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Wed Nov 16 22:49:20 2011
@@ -8,6 +8,12 @@
  * Use faster bytes comparison (CASSANDRA-3434)
 
 
+1.0.4
+ * fix liveSize stat when sstables are removed (CASSANDRA-3496)
+ * add bloom filter FP rates to nodetool cfstats (CASSANDRA-3347)
+ * record partitioner in sstable metadata component (CASSANDRA-3407)
+
+
 1.0.3
  * revert name-based query defragmentation aka CASSANDRA-2503 (CASSANDRA-3491)
  * fix invalidate-related test failures (CASSANDRA-3437)
@@ -37,6 +43,7 @@ Merged from 0.8:
  * `describe ring` command for CLI (CASSANDRA-3220)
  * (Hadoop) skip empty rows when entire row is requested, redux 
(CASSANDRA-2855)
 
+
 1.0.2
  * defragment rows for name-based queries under STCS (CASSANDRA-2503)
  * Add timing information to cassandra-cli GET/SET/LIST queries 
(CASSANDRA-3326)

Propchange: cassandra/trunk/contrib/
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Nov 16 22:49:20 2011
@@ -4,7 +4,7 @@
 
/cassandra/branches/cassandra-0.8/contrib:1090934-1125013,1125019-1198724,1198726-1199259,1199284,1200226,1200471
 /cassandra/branches/cassandra-0.8.0/contrib:1125021-1130369
 /cassandra/branches/cassandra-0.8.1/contrib:1101014-1125018
-/cassandra/branches/cassandra-1.0/contrib:1167085-1202429
+/cassandra/branches/cassandra-1.0/contrib:1167085-1202912
 
/cassandra/branches/cassandra-1.0.0/contrib:1167104-1167229,1167232-1181093,1181741,1181816,1181820,1182951,1183243
 /cassandra/tags/cassandra-0.7.0-rc3/contrib:1051699-1053689
 /cassandra/tags/cassandra-0.8.0-rc1/contrib:1102511-1125020

Propchange: 
cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
--
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Nov 16 22:49:20 2011
@@ -4,7 +4,7 @@
 
/cassandra/branches/cassandra-0.8/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1090934-1125013,1125019-1198724,1198726-1199259,1199284,1200226,1200471
 
/cassandra/branches/cassandra-0.8.0/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1125021-1130369
 
/cassandra/branches/cassandra-0.8.1/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1101014-1125018
-/cassandra/branches/cassandra-1.0/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:1167085-1202429

[jira] [Updated] (CASSANDRA-2407) Compaction thread should try to empty a bucket before moving on

2011-11-16 Thread Jonathan Ellis (Updated) (JIRA)

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

Jonathan Ellis updated CASSANDRA-2407:
--

Attachment: 2407-v3.txt

rebased

 Compaction thread should try to empty a bucket before moving on
 ---

 Key: CASSANDRA-2407
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2407
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Stu Hood
Assignee: Jonathan Ellis
Priority: Minor
  Labels: compaction
 Fix For: 1.1

 Attachments: 2407-v2.txt, 2407-v3.txt, 2407.txt


 As suggested by Aaron Morton 
 [(1)|https://issues.apache.org/jira/browse/CASSANDRA-2191?focusedCommentId=13010077page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13010077],
  a compaction thread should attempt to empty a bucket before moving on to a 
 larger bucket. This would change the submitMinorIfNeeded {{for}} loop into a 
 while loop that regenerated the buckets and started from the bottom after 
 each successful compaction.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2819) Split rpc timeout for read and write ops

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2819?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151611#comment-13151611
 ] 

Jonathan Ellis commented on CASSANDRA-2819:
---

bq. wanna modify the getRpcTimeout in DD so that it returns the max of the 
other get*RpcTimeout()?

I think we need to split it into getGenericRpcTimeout 
(getMiscellaneousRpcTimeout?) and getMaxRpcTimeout.

bq. Not a so elegant solution in this patch. It uses the corresponding 
get*RpcTimeout in different callbacks.

looks good to me.

 Split rpc timeout for read and write ops
 

 Key: CASSANDRA-2819
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2819
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Stu Hood
Assignee: Melvin Wang
 Fix For: 1.1

 Attachments: 2819-v4.txt, 2819-v5-rebased.txt, c2819-v6, c2819.patch, 
 rpc-jira.patch


 Given the vastly different latency characteristics of reads and writes, it 
 makes sense for them to have independent rpc timeouts internally.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2434) range movements can violate consistency

2011-11-16 Thread paul cannon (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151620#comment-13151620
 ] 

paul cannon commented on CASSANDRA-2434:


bq. It sounds like you have a reasonable solution here, is there any reason not 
to implement it for 1.1?

Just that it's quite a bit more complex than simply disallowing overlapping 
ring movements, and the extra problems that come with higher complexity. I 
think this feature is worth it, on its own, but when i think of how much pain 
Brandon seems to be going through dealing with streaming code, maybe it's not.

 range movements can violate consistency
 ---

 Key: CASSANDRA-2434
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2434
 Project: Cassandra
  Issue Type: Bug
Reporter: Peter Schuller
Assignee: paul cannon
 Fix For: 1.0.4

 Attachments: 2434-3.patch.txt, 2434-testery.patch.txt


 My reading (a while ago) of the code indicates that there is no logic 
 involved during bootstrapping that avoids consistency level violations. If I 
 recall correctly it just grabs neighbors that are currently up.
 There are at least two issues I have with this behavior:
 * If I have a cluster where I have applications relying on QUORUM with RF=3, 
 and bootstrapping complete based on only one node, I have just violated the 
 supposedly guaranteed consistency semantics of the cluster.
 * Nodes can flap up and down at any time, so even if a human takes care to 
 look at which nodes are up and things about it carefully before 
 bootstrapping, there's no guarantee.
 A complication is that not only does it depend on use-case where this is an 
 issue (if all you ever do you do at CL.ONE, it's fine); even in a cluster 
 which is otherwise used for QUORUM operations you may wish to accept 
 less-than-quorum nodes during bootstrap in various emergency situations.
 A potential easy fix is to have bootstrap take an argument which is the 
 number of hosts to bootstrap from, or to assume QUORUM if none is given.
 (A related concern is bootstrapping across data centers. You may *want* to 
 bootstrap to a local node and then do a repair to avoid sending loads of data 
 across DC:s while still achieving consistency. Or even if you don't care 
 about the consistency issues, I don't think there is currently a way to 
 bootstrap from local nodes only.)
 Thoughts?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2434) range movements can violate consistency

2011-11-16 Thread Jonathan Ellis (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151640#comment-13151640
 ] 

Jonathan Ellis commented on CASSANDRA-2434:
---

No longer very optimistic on the may even end up simpler overall than our 
current code front?

TBH this area of the code is fragile and hairy and maybe starting from a clean 
slate with a real plan instead of trying to patch things in haphazardly would 
be a good thing.

But, I'd be okay with re-imposing the no overlapping moves rule and fixing 
the stream source problem if that's going to be substantially simpler.

 range movements can violate consistency
 ---

 Key: CASSANDRA-2434
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2434
 Project: Cassandra
  Issue Type: Bug
Reporter: Peter Schuller
Assignee: paul cannon
 Fix For: 1.0.4

 Attachments: 2434-3.patch.txt, 2434-testery.patch.txt


 My reading (a while ago) of the code indicates that there is no logic 
 involved during bootstrapping that avoids consistency level violations. If I 
 recall correctly it just grabs neighbors that are currently up.
 There are at least two issues I have with this behavior:
 * If I have a cluster where I have applications relying on QUORUM with RF=3, 
 and bootstrapping complete based on only one node, I have just violated the 
 supposedly guaranteed consistency semantics of the cluster.
 * Nodes can flap up and down at any time, so even if a human takes care to 
 look at which nodes are up and things about it carefully before 
 bootstrapping, there's no guarantee.
 A complication is that not only does it depend on use-case where this is an 
 issue (if all you ever do you do at CL.ONE, it's fine); even in a cluster 
 which is otherwise used for QUORUM operations you may wish to accept 
 less-than-quorum nodes during bootstrap in various emergency situations.
 A potential easy fix is to have bootstrap take an argument which is the 
 number of hosts to bootstrap from, or to assume QUORUM if none is given.
 (A related concern is bootstrapping across data centers. You may *want* to 
 bootstrap to a local node and then do a repair to avoid sending loads of data 
 across DC:s while still achieving consistency. Or even if you don't care 
 about the consistency issues, I don't think there is currently a way to 
 bootstrap from local nodes only.)
 Thoughts?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3186) nodetool should not NPE when rack/dc info is not yet available

2011-11-16 Thread Hudson (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151657#comment-13151657
 ] 

Hudson commented on CASSANDRA-3186:
---

Integrated in Cassandra-0.8 #399 (See 
[https://builds.apache.org/job/Cassandra-0.8/399/])
Set default rack/dc in ec2snitch to avoid NPEs.
Patch by Alex Araujo, reviewed by brandonwilliams for CASSANDRA-3186.

brandonwilliams : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1202892
Files : 
* 
/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/locator/Ec2Snitch.java


 nodetool should not NPE when rack/dc info is not yet available
 --

 Key: CASSANDRA-3186
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3186
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.8.1
Reporter: Brandon Williams
Assignee: Brandon Williams
Priority: Minor
  Labels: lhf
 Fix For: 0.8.8

 Attachments: cassandra-0.8-3186.txt


 As the title says.  What happens is the persisted ring is loaded, but if 
 those nodes are down and you're using a snitch like ec2 that gets rack/dc 
 info from gossip, nodetool NPEs instead of showing that the node is down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2819) Split rpc timeout for read and write ops

2011-11-16 Thread Melvin Wang (Updated) (JIRA)

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

Melvin Wang updated CASSANDRA-2819:
---

Attachment: c2819-v7

Please take another look. Following changes made
* CassandraServer.java, set the timeout when we schedule individual Rpcs
* outboundtcpconnection uses per message timeout.

there are still couple of places undetermined.
* FailureDetector.java:249, we need a getGossipTimeout() here?
* MessagingService.java, in expiringMap we probably want getMaxRpcTimeout()
* 
StorageService.java:sendReplicationNotification()/truncateResponseHander/FileStreamTask,
 we use getGenericRpcTimeout()?
* StorageProxy.java, 3 waitOnFutures calls for repairs, they should bear the 
timeout of the enclosing call?
* StorageProxy.java, describeSchema and truncate should use 
getGenericRpcTimeout()?
* OutboundTcpConnection.java, in connect(), shall we have a getSocketTimeout() 
or use getGenericRpcTimeout()?



 Split rpc timeout for read and write ops
 

 Key: CASSANDRA-2819
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2819
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Stu Hood
Assignee: Melvin Wang
 Fix For: 1.1

 Attachments: 2819-v4.txt, 2819-v5-rebased.txt, c2819-v6, c2819-v7, 
 c2819.patch, rpc-jira.patch


 Given the vastly different latency characteristics of reads and writes, it 
 makes sense for them to have independent rpc timeouts internally.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-3483) Support bringing up a new datacenter to existing cluster without repair

2011-11-16 Thread Chris Goffinet (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151784#comment-13151784
 ] 

Chris Goffinet commented on CASSANDRA-3483:
---

Yeah as Brandon mentioned, we would still want to go into the bootstrap state 
to get those writes. This would also allow us to add capacity in the same way, 
if we manually pick tokens (auto bootstrap is kinda worthless IMO) to existing 
DC as well. We can just fire off the bootstrap command from nodetool as needed.

 Support bringing up a new datacenter to existing cluster without repair
 ---

 Key: CASSANDRA-3483
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3483
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 1.0.2
Reporter: Chris Goffinet

 Was talking to Brandon in irc, and we ran into a case where we want to bring 
 up a new DC to an existing cluster. He suggested from jbellis the way to do 
 it currently was set strategy options of dc2:0, then add the nodes. After the 
 nodes are up, change the RF of dc2, and run repair. 
 I'd like to avoid a repair as it runs AES and is a bit more intense than how 
 bootstrap works currently by just streaming ranges from the SSTables. Would 
 it be possible to improve this functionality (adding a new DC to existing 
 cluster) than the proposed method? We'd be happy to do a patch if we got some 
 input on the best way to go about it.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-2967) Only bind JMX to the same IP address that is being used in Cassandra

2011-11-16 Thread Alex Araujo (Updated) (JIRA)

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

Alex Araujo updated CASSANDRA-2967:
---

Attachment: cassandra-1.0-2967-v4.txt

v4 removes conflicting {{-Dcom.sun.management.jmxremote.port}} system property 
from cassandra-env.sh and preserves the current RMI registry default port value.

Without any changes to the default cassandra.yaml file the following will work:

{code:title=nodetool ring}
nodetool -h localhost ring
{code}
{code:title=JConsole Remote Process}
service:jmx:rmi://localhost:7299/jndi/rmi://localhost:7199/jmxrmi
{code}

 Only bind JMX to the same IP address that is being used in Cassandra
 

 Key: CASSANDRA-2967
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2967
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.2
Reporter: Joaquin Casares
Assignee: Alex Araujo
Priority: Minor
  Labels: lhf
 Attachments: cassandra-0.8-2967.txt, cassandra-1.0-2967-v2.txt, 
 cassandra-1.0-2967-v3.txt, cassandra-1.0-2967-v4.txt


 The setup is 5 nodes in each data center are all running on one physical test 
 machine and even though the repair was run against the correct IP the wrong 
 JMX port was used. As a result, instead of repairing all 5 nodes I was 
 repairing the same node 5 times.
 It would be nice if Cassandra's JMX would bind to only the IP address on 
 which its thrift/RPC services are listening on instead of binding to all IP's 
 on the box.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2967) Only bind JMX to the same IP address that is being used in Cassandra

2011-11-16 Thread Vijay (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151852#comment-13151852
 ] 

Vijay commented on CASSANDRA-2967:
--

Plz note: 
http://download.oracle.com/javase/6/docs/technotes/guides/management/agent.html 
enables the RMI server, which will in-turn do System.gc every 60 min. I have 
tried it before and reverted because of the same. 
http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6200091 you might want to 
use DisableExplicitGC gc option or you might want to increase the 
-Dsun.rmi.dgc.server.gcInterval=60 both of which might be kind of dangerous 
depnds on how people use it.


 Only bind JMX to the same IP address that is being used in Cassandra
 

 Key: CASSANDRA-2967
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2967
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.2
Reporter: Joaquin Casares
Assignee: Alex Araujo
Priority: Minor
  Labels: lhf
 Attachments: cassandra-0.8-2967.txt, cassandra-1.0-2967-v2.txt, 
 cassandra-1.0-2967-v3.txt, cassandra-1.0-2967-v4.txt


 The setup is 5 nodes in each data center are all running on one physical test 
 machine and even though the repair was run against the correct IP the wrong 
 JMX port was used. As a result, instead of repairing all 5 nodes I was 
 repairing the same node 5 times.
 It would be nice if Cassandra's JMX would bind to only the IP address on 
 which its thrift/RPC services are listening on instead of binding to all IP's 
 on the box.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-2967) Only bind JMX to the same IP address that is being used in Cassandra

2011-11-16 Thread Vijay (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13151876#comment-13151876
 ] 

Vijay commented on CASSANDRA-2967:
--

https://github.com/Vijay2win/jmxProxy is what i use to connect jconsole to the 
remote vm.

 Only bind JMX to the same IP address that is being used in Cassandra
 

 Key: CASSANDRA-2967
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2967
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Affects Versions: 0.8.2
Reporter: Joaquin Casares
Assignee: Alex Araujo
Priority: Minor
  Labels: lhf
 Attachments: cassandra-0.8-2967.txt, cassandra-1.0-2967-v2.txt, 
 cassandra-1.0-2967-v3.txt, cassandra-1.0-2967-v4.txt


 The setup is 5 nodes in each data center are all running on one physical test 
 machine and even though the repair was run against the correct IP the wrong 
 JMX port was used. As a result, instead of repairing all 5 nodes I was 
 repairing the same node 5 times.
 It would be nice if Cassandra's JMX would bind to only the IP address on 
 which its thrift/RPC services are listening on instead of binding to all IP's 
 on the box.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira