[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev commented on CASSANDRA-7872:


No, Phattom ref proc does not require STW FullGC. 
On Java 6, 7, and 8 it gets processed during CMS remark phase (if you enable 
PrintGCDetails you'll see timing of this stage in logs, like for example:
{code}
[Rescan (parallel) , 0.0159280 secs]
[weak refs processing, 0.0032600 secs] --- here
[class unloading, 0.0082150 secs][scrub symbol table, 0.0078220 secs][scrub 
string table, 0.0013500 secs] [1 CMS-remark: 4480383K(24117248K)] 
4506007K(28835840K), 0.1030490 secs] [Times: user=1.00 sys=0.01, real=0.10 
secs] 
{code}
This is how it works since 1.6_24 and all more modern JDKs, including all java 
7 and on. jdk 6_24 is 3 years old now, so this way of work can be considered 
stable.

Could you provide details on which java or java build number it did not work 
this way ?


 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev edited comment on CASSANDRA-7872 at 10/5/14 6:58 AM:
-

No, Phattom ref proc does not require STW FullGC. 
On Java 6, 7, and 8 it gets processed during CMS remark phase (if you enable 
PrintGCDetails you'll see timing of this stage in logs, like for example:
{code}
[Rescan (parallel) , 0.0159280 secs]
[weak refs processing, 0.0032600 secs] --- here
[class unloading, 0.0082150 secs][scrub symbol table, 0.0078220 secs][scrub 
string table, 0.0013500 secs] [1 CMS-remark: 4480383K(24117248K)] 
4506007K(28835840K), 0.1030490 secs] [Times: user=1.00 sys=0.01, real=0.10 
secs] 
{code}
This is how it works since 1.6_24 and all more modern JDKs, including all java 
7 and on. jdk 6_24 is 3 years old now, so this way of work can be considered 
stable.
(AFAIK before 6_24 it also worked this exact way, but implementation of it was 
less performant)

Could you provide details on which java or java build number it did not work 
this way ?



was (Author: m0nstermind):
No, Phattom ref proc does not require STW FullGC. 
On Java 6, 7, and 8 it gets processed during CMS remark phase (if you enable 
PrintGCDetails you'll see timing of this stage in logs, like for example:
{code}
[Rescan (parallel) , 0.0159280 secs]
[weak refs processing, 0.0032600 secs] --- here
[class unloading, 0.0082150 secs][scrub symbol table, 0.0078220 secs][scrub 
string table, 0.0013500 secs] [1 CMS-remark: 4480383K(24117248K)] 
4506007K(28835840K), 0.1030490 secs] [Times: user=1.00 sys=0.01, real=0.10 
secs] 
{code}
This is how it works since 1.6_24 and all more modern JDKs, including all java 
7 and on. jdk 6_24 is 3 years old now, so this way of work can be considered 
stable.

Could you provide details on which java or java build number it did not work 
this way ?


 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-7872:


I'm not saying that STW is required to cleanup weak refs, but rather that it 
was (and probably still is) not guaranteed to happen at any given phrase and is 
an implementation detail, I think we've tried multiple versions of OpenJDK and 
some versions of Oracle JDK for 1.7 last year, although I can't recon which 
ones. We tried to make it work for the memory mapped mode couple of times 
before and I'm coming from the experience where it never performed what was 
expected reliably and that's why we do call Cleaner explicitly once reference 
count for SSTable reaches 0 instead of relaying on weak ref cleanup, personally 
I don't think it's a good idea to rely on this especially as we have 
CASSANDRA-7705 for the future versions so I would be very much happy to have 1 
and 2 but not #3.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev commented on CASSANDRA-7872:


I suggest youre talking not about STW but about that the moment of cleanup 
cannot be predicted and guaranteed to happen. Yes, if you build your entire 
critical resource cleanup system on ref queues, it is a problem. 

But this is not this case in this particular ticket. Delete of much of 
resources happen normally when sstable ref count reaches 0, i.e. as predictable 
as before. Cleanup on Ref queue is used only for those, which miscount their 
refs, so they will never be deleted, until restart. Having it deleted on next 
CMS gc (i.e. at some not exactly predictable point in near future) is IMO 
better than having them never deleted. 
Some of our c* clusters are running for several months and even years with no 
reboot, so not cleaning up resources until restart is an operational pain.

As I can see CASSANDRA-7705 is planned for 3.0, which is far from being ready. 
So i suggest it is better to have some resources cleanup code now, in 2.0 and 
2.1. It could be removed later on 3.0 release. 

Another benefit of having #3 is that it could help to catch bugs reference 
miscounts not cleaning up resources, b/c it notes to logs when it detected the 
miscount. Without #3 miscounts pass unnoticed. This could be used debugging 
CASSANDRA-7705  as well.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-7872:


bq. Having it deleted on next CMS gc (i.e. at some not exactly predictable 
point in near future) is IMO better than having them never deleted. 

If you can back quoted statement up with any document (JMM or other) that 
explicitly guarantees cleanup behavior you described (being, it's guaranteed to 
happen on CMS or minor collection for phantom references) I would be fine 
adding #3, meanwhile there is no real point of arguing about adding code that 
relies on the implementation detail and may never actually trigger.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev commented on CASSANDRA-7872:


Well this is part of PhRef contract.

http://docs.oracle.com/javase/6/docs/api/java/lang/ref/PhantomReference.html

{quote}
If the garbage collector determines at a certain point in time that the 
referent of a phantom reference is phantom reachable, then at that time or at 
some later time it will enqueue the reference
{quote}

The key point here not about CMS, but about some certain time while running the 
program. This is enough for cleaning up resources which cannot be cleaned other 
way due to bugs, etc.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev edited comment on CASSANDRA-7872 at 10/5/14 8:34 AM:
-

Well this is part of PhRef contract.

http://docs.oracle.com/javase/6/docs/api/java/lang/ref/PhantomReference.html

{quote}
If the garbage collector determines at a certain point in time that the 
referent of a phantom reference is phantom reachable, then at that time or at 
some later time it will enqueue the reference
{quote}

The key point here not about CMS, but about some certain time while running the 
program. This is enough for cleaning up resources which cannot be cleaned 
other, more predictable, way due to bugs, etc.


was (Author: m0nstermind):
Well this is part of PhRef contract.

http://docs.oracle.com/javase/6/docs/api/java/lang/ref/PhantomReference.html

{quote}
If the garbage collector determines at a certain point in time that the 
referent of a phantom reference is phantom reachable, then at that time or at 
some later time it will enqueue the reference
{quote}

The key point here not about CMS, but about some certain time while running the 
program. This is enough for cleaning up resources which cannot be cleaned other 
way due to bugs, etc.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-7872:


That's exactly my point, certain point in time could be only when Full GC is 
performed as an fallback for CMS (exactly what we have observed previously) and 
that point could take weeks to happen which means that system will, most 
luckily, run out of disk space even before phantom references are determined to 
be cleaned, which brings us back to the point on relying on implementation 
detail (in our cases that it's going to happen on minor collection or CMS) with 
no execution guarantees which gives everybody a false impression of safely 
where there is none.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev commented on CASSANDRA-7872:


Dont you mixing Cleaner of MappedBuffer, which relies on finalizers with 
Phantom ref Q? You may observed a behavior of MappedByteBuffers dont release 
their mapped memory to OS until FullGC, and this is well known 7 year 
bug-o-feature of JDK. The source of this behavior is how finalize() methods 
are processed in JDK.

But PhantomRef processing is a different story, there are no finalizers there, 
so their proc happens on each CMS cycle.  

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-7872:


I'm not trying to mix terms here, I just gave a concrete example for reference 
but we've multiple things. My point is - there is no guaranteed phrase or time 
when phantom or any other reference cleanup is going to happen, if #3 is 
intended as Plan B in case things go wrong with SSTable reference counting 
then by relying on phantom references we rely purely on implementation detail 
that in some JVMs it happens on CMS or minor collection which would not be true 
for all GC collector implementations and/or JVM implementations, by doing so we 
create false sense of safely for everybody which is a bad thing, so I'm 
strongly -1 on doing so. 

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Oleg Anastasyev (JIRA)

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

Oleg Anastasyev commented on CASSANDRA-7872:


I cannot agree with you.
Having Plan B is better for reliability, than not having it at all. This whole 
ticket is about Plan B.
C* is recommended to run on Oracle's Java, it would not likely run on anything 
else (just b/c it uses undocumented classes, like sun.misc.Cleaner for 
example). Oracle's Java has contract for PhRefs and the way how it is 
implemented. This way is stable and will be supported by JDK team in future to 
do not break backward compatibility, this is how they do JDK development. It 
fits purpose of cleaning things out which cannot be cleaned other way.

So not using a feature of JDK and leave garbage on disk, because of possible 
existence of some abstract JDK out there with different behavior (on which C* 
will not likely work anyway) is just to sacrifice reliability for nothing.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread mck (JIRA)

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

mck commented on CASSANDRA-8032:


[~iamaleksey] I was unaware originally that request_scheduler was only 
implemented against the thrift api, and never for cql.
So  guess the question, ie CASSANDRA-8059, is whether to implement 
request_scheduler for cql, or to remove it along with thrift?

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.
 I haven't look into the feasibility of this within the code, but it comes to 
 mind as rather simple, and i would be interested in offering a patch if the 
 idea carries validity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread mck (JIRA)

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

mck edited comment on CASSANDRA-8032 at 10/5/14 9:33 AM:
-

[~iamaleksey] I was unaware originally that request_scheduler was only 
implemented against the thrift api, and never for cql.
So i guess the question, ie CASSANDRA-8059, is whether to implement 
request_scheduler for cql, or to remove it along with thrift?


was (Author: michaelsembwever):
[~iamaleksey] I was unaware originally that request_scheduler was only 
implemented against the thrift api, and never for cql.
So  guess the question, ie CASSANDRA-8059, is whether to implement 
request_scheduler for cql, or to remove it along with thrift?

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.
 I haven't look into the feasibility of this within the code, but it comes to 
 mind as rather simple, and i would be interested in offering a patch if the 
 idea carries validity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8059) CQL requests need to go through the request scheduler

2014-10-05 Thread mck (JIRA)

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

mck updated CASSANDRA-8059:
---
Description: 
Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
request scheduler. I'm guessing they never have?

If this is to be fixed/implemented then it looks like that there's a number of 
methods in `CassandraServer` that need to call `schedule(timeout)`.

The alternative is to remove the request scheduler along with thrift?

  was:
Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
request scheduler. I'm guessing they never have?

If this is to be fixed/implemented then it looks like that there's a number of 
methods in `CassandraServer` that need to call `schedule(timeout)`.


 CQL requests need to go through the request scheduler
 -

 Key: CASSANDRA-8059
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8059
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: mck

 Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
 request scheduler. I'm guessing they never have?
 If this is to be fixed/implemented then it looks like that there's a number 
 of methods in `CassandraServer` that need to call `schedule(timeout)`.
 The alternative is to remove the request scheduler along with thrift?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8059) CQL requests need to go through the request scheduler

2014-10-05 Thread mck (JIRA)

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

mck updated CASSANDRA-8059:
---
Description: 
Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
request scheduler. I'm guessing they never have?

If this is to be fixed/implemented then it looks like that there's a number of 
methods in `CassandraServer` that need to call `schedule(timeout)`.

The alternative is to remove the request scheduler along with thrift?

Regarding that choice see my opinion from CASSANDRA-8032…
{quote}
… … The request scheduler is a useful concept but i think that setups with 
enough nodes often favour separate clusters rather than either creating 
separate virtual datacenters or using the request scheduler. To give the 
request scheduler another, and more flexible, implementation could especially 
help those users that don't yet have enough nodes to warrant separate clusters, 
or even separate virtual datacenters. On such smaller clusters cassandra can 
still be seen as an unstable technology because poor consumers/schemas can 
easily affect, even bring down, a whole cluster. … …
{quote}

  was:
Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
request scheduler. I'm guessing they never have?

If this is to be fixed/implemented then it looks like that there's a number of 
methods in `CassandraServer` that need to call `schedule(timeout)`.

The alternative is to remove the request scheduler along with thrift?


 CQL requests need to go through the request scheduler
 -

 Key: CASSANDRA-8059
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8059
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: mck

 Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
 request scheduler. I'm guessing they never have?
 If this is to be fixed/implemented then it looks like that there's a number 
 of methods in `CassandraServer` that need to call `schedule(timeout)`.
 The alternative is to remove the request scheduler along with thrift?
 Regarding that choice see my opinion from CASSANDRA-8032…
 {quote}
 … … The request scheduler is a useful concept but i think that setups with 
 enough nodes often favour separate clusters rather than either creating 
 separate virtual datacenters or using the request scheduler. To give the 
 request scheduler another, and more flexible, implementation could especially 
 help those users that don't yet have enough nodes to warrant separate 
 clusters, or even separate virtual datacenters. On such smaller clusters 
 cassandra can still be seen as an unstable technology because poor 
 consumers/schemas can easily affect, even bring down, a whole cluster. … …
 {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-7872:


bq. Having Plan B is better for reliability, than not having it at all. This 
whole ticket is about Plan B. C* is recommended to run on Oracle's Java, it 
would not likely run on anything else (just b/c it uses undocumented classes, 
like sun.misc.Cleaner for example). Oracle's Java has contract for PhRefs and 
the way how it is implemented.

Having Plan B that could potentially never trigger and gives people force 
perspective on things is worse then not having one at all, also recommended 
doesn't mean required (and OpenJDK has sun.misc.Cleaner for that matter, but 
agent support is weaker comparing to Oracle JDK). You claim that there is a 
contract but the reality of things is that you failed to provide any proof 
that such contact exists even in Oracle JDK which explicitly mentions how/when 
phantom or any type of reference is supposed to be cleaned up except If the 
garbage collector determines at a certain point in time which is a broad 
definition and could mean anything e.g. only at Full GC time, as that behavior 
is left to be an implementation detail by JVM there is no guarantee that it's 
not going to change across releases or even that all of the garbage collector 
implementations are going to yield the same behavior, so things can go south 
and crash way before Plan B would trigger.

I made my opinion known and pretty much done arguing about this without any 
real arguments to support #3, so I will leave this to [~jbellis] to tie break 
and we'll go from there.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8032:
--

Right. This feature AFAIK, this feature isn't really being used by people.

As for finer granularity of control, from read-only vs read-write users on 
keyspaces - you can have table-level granularity for authorization there using 
the IAuthenticator+IAuthorizer APIs.

Using it for virtual datacenter bulding would be a hack, and on its own it's 
not enough to support true multitenancy - you need better resource 
isolation/usage limits support, deep and on all levels, to make that happen.

So I'd say let it go, for now, until more suitable times, and close 
CASSANDRA-8059 - you'll have a hard time convincing other committers are 
justified, given that this would not *really* bring us any closer to 
multitenancy, and on its own the feature request is very niche.

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.
 I haven't look into the feasibility of this within the code, but it comes to 
 mind as rather simple, and i would be interested in offering a patch if the 
 idea carries validity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko edited comment on CASSANDRA-8032 at 10/5/14 1:01 PM:
---

Right. AFAIK this feature isn't really being used by people.

As for finer granularity of control, from read-only vs read-write users on 
keyspaces - you can have table-level granularity for authorization there using 
the IAuthenticator+IAuthorizer APIs.

Using it for virtual datacenter bulding would be a hack, and on its own it's 
not enough to support true multitenancy - you need better resource 
isolation/usage limits support, deep and on all levels, to make that happen.

So I'd say let it go, for now, until more suitable times, and close 
CASSANDRA-8059 - you'll have a hard time convincing other committers that the 
changes to CQL Statement classes are justified, given that this would not 
*really* bring us any closer to multitenancy, and on its own the feature 
request is very niche.


was (Author: iamaleksey):
Right. This feature AFAIK, this feature isn't really being used by people.

As for finer granularity of control, from read-only vs read-write users on 
keyspaces - you can have table-level granularity for authorization there using 
the IAuthenticator+IAuthorizer APIs.

Using it for virtual datacenter bulding would be a hack, and on its own it's 
not enough to support true multitenancy - you need better resource 
isolation/usage limits support, deep and on all levels, to make that happen.

So I'd say let it go, for now, until more suitable times, and close 
CASSANDRA-8059 - you'll have a hard time convincing other committers are 
justified, given that this would not *really* bring us any closer to 
multitenancy, and on its own the feature request is very niche.

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.
 I haven't look into the feasibility of this within the code, but it comes to 
 mind as rather simple, and i would be interested in offering a patch if the 
 idea carries validity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8060) Geography-aware replication

2014-10-05 Thread Donald Smith (JIRA)
Donald Smith created CASSANDRA-8060:
---

 Summary: Geography-aware replication
 Key: CASSANDRA-8060
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8060
 Project: Cassandra
  Issue Type: Wish
Reporter: Donald Smith


We have three data centers in the US (CA in California, TX in Texas, and NJ in 
NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all our 
writing to CA.  That represents a bottleneck, since the coordinator nodes in CA 
are responsible for all the replication to every data center.

Far better if we had the option of setting things up so that CA replicated to 
TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
for replicating to UK, which should replicate to DE.  Etc, etc.

This could be controlled by the topology file.

It would have major ramifications for latency architecture but might be 
appropriate for some scenarios.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8060) Geography-aware, daisy-chaining replication

2014-10-05 Thread Donald Smith (JIRA)

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

Donald Smith updated CASSANDRA-8060:

Summary: Geography-aware, daisy-chaining replication  (was: Geography-aware 
replication)

 Geography-aware, daisy-chaining replication
 ---

 Key: CASSANDRA-8060
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8060
 Project: Cassandra
  Issue Type: Wish
Reporter: Donald Smith

 We have three data centers in the US (CA in California, TX in Texas, and NJ 
 in NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all 
 our writing to CA.  That represents a bottleneck, since the coordinator nodes 
 in CA are responsible for all the replication to every data center.
 Far better if we had the option of setting things up so that CA replicated to 
 TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
 for replicating to UK, which should replicate to DE.  Etc, etc.
 This could be controlled by the topology file.
 It would have major ramifications for latency architecture but might be 
 appropriate for some scenarios.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8060) Geography-aware, daisy-chaining replication

2014-10-05 Thread Donald Smith (JIRA)

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

Donald Smith updated CASSANDRA-8060:

Description: 
We have three data centers in the US (CA in California, TX in Texas, and NJ in 
NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all our 
writing to CA.  That represents a bottleneck, since the coordinator nodes in CA 
are responsible for all the replication to every data center.

Far better if we had the option of setting things up so that CA replicated to 
TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
for replicating to UK, which should replicate to DE.  Etc, etc.

This could be controlled by the topology file.

It would require architectural changes and would have major ramifications for 
latency but might be appropriate for some scenarios.

  was:
We have three data centers in the US (CA in California, TX in Texas, and NJ in 
NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all our 
writing to CA.  That represents a bottleneck, since the coordinator nodes in CA 
are responsible for all the replication to every data center.

Far better if we had the option of setting things up so that CA replicated to 
TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
for replicating to UK, which should replicate to DE.  Etc, etc.

This could be controlled by the topology file.

It would have major ramifications for latency architecture but might be 
appropriate for some scenarios.


 Geography-aware, daisy-chaining replication
 ---

 Key: CASSANDRA-8060
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8060
 Project: Cassandra
  Issue Type: Wish
Reporter: Donald Smith

 We have three data centers in the US (CA in California, TX in Texas, and NJ 
 in NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all 
 our writing to CA.  That represents a bottleneck, since the coordinator nodes 
 in CA are responsible for all the replication to every data center.
 Far better if we had the option of setting things up so that CA replicated to 
 TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
 for replicating to UK, which should replicate to DE.  Etc, etc.
 This could be controlled by the topology file.
 It would require architectural changes and would have major ramifications for 
 latency but might be appropriate for some scenarios.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8060) Geography-aware, distributed replication

2014-10-05 Thread Donald Smith (JIRA)

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

Donald Smith updated CASSANDRA-8060:

Summary: Geography-aware, distributed replication  (was: Geography-aware, 
daisy-chaining replication)

 Geography-aware, distributed replication
 

 Key: CASSANDRA-8060
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8060
 Project: Cassandra
  Issue Type: Wish
Reporter: Donald Smith

 We have three data centers in the US (CA in California, TX in Texas, and NJ 
 in NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all 
 our writing to CA.  That represents a bottleneck, since the coordinator nodes 
 in CA are responsible for all the replication to every data center.
 Far better if we had the option of setting things up so that CA replicated to 
 TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
 for replicating to UK, which should replicate to DE.  Etc, etc.
 This could be controlled by the topology file.
 The replication could be organized in a tree-like structure instead of a 
 daisy-chain.
 It would require architectural changes and would have major ramifications for 
 latency but might be appropriate for some scenarios.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8060) Geography-aware, daisy-chaining replication

2014-10-05 Thread Donald Smith (JIRA)

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

Donald Smith updated CASSANDRA-8060:

Description: 
We have three data centers in the US (CA in California, TX in Texas, and NJ in 
NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all our 
writing to CA.  That represents a bottleneck, since the coordinator nodes in CA 
are responsible for all the replication to every data center.

Far better if we had the option of setting things up so that CA replicated to 
TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
for replicating to UK, which should replicate to DE.  Etc, etc.

This could be controlled by the topology file.

The replication could be organized in a tree-like structure instead of a 
daisy-chain.

It would require architectural changes and would have major ramifications for 
latency but might be appropriate for some scenarios.

  was:
We have three data centers in the US (CA in California, TX in Texas, and NJ in 
NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all our 
writing to CA.  That represents a bottleneck, since the coordinator nodes in CA 
are responsible for all the replication to every data center.

Far better if we had the option of setting things up so that CA replicated to 
TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
for replicating to UK, which should replicate to DE.  Etc, etc.

This could be controlled by the topology file.

It would require architectural changes and would have major ramifications for 
latency but might be appropriate for some scenarios.


 Geography-aware, daisy-chaining replication
 ---

 Key: CASSANDRA-8060
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8060
 Project: Cassandra
  Issue Type: Wish
Reporter: Donald Smith

 We have three data centers in the US (CA in California, TX in Texas, and NJ 
 in NJ), two in Europe (UK  and DE), and two in Asia (JP and CH1).  We do all 
 our writing to CA.  That represents a bottleneck, since the coordinator nodes 
 in CA are responsible for all the replication to every data center.
 Far better if we had the option of setting things up so that CA replicated to 
 TX , which replicated to NJ. NJ is closer to UK, so NJ should be responsible 
 for replicating to UK, which should replicate to DE.  Etc, etc.
 This could be controlled by the topology file.
 The replication could be organized in a tree-like structure instead of a 
 daisy-chain.
 It would require architectural changes and would have major ramifications for 
 latency but might be appropriate for some scenarios.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8061) tmplink files are not removed

2014-10-05 Thread Gianluca Borello (JIRA)
Gianluca Borello created CASSANDRA-8061:
---

 Summary: tmplink files are not removed
 Key: CASSANDRA-8061
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8061
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Linux
Reporter: Gianluca Borello


After installing 2.1.0, I'm experiencing a bunch of tmplink files that are 
filling my disk. I found https://issues.apache.org/jira/browse/CASSANDRA-7803 
and that is very similar, and I confirm it happens both on 2.1.0 as well as 
from the latest commit on the cassandra-2.1 branch 
(https://github.com/apache/cassandra/commit/aca80da38c3d86a40cc63d9a122f7d45258e4685
 from the cassandra-2.1)

Even starting with a clean keyspace, after a few hours I get:

$ sudo find /raid0 | grep tmplink | xargs du -hs
2.7G
/raid0/cassandra/data/draios/protobuf1-ccc6dce04beb11e4abf997b38fbf920b/draios-protobuf1-tmplink-ka-4515-Data.db
13M 
/raid0/cassandra/data/draios/protobuf1-ccc6dce04beb11e4abf997b38fbf920b/draios-protobuf1-tmplink-ka-4515-Index.db
1.8G
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-1788-Data.db
12M 
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-1788-Index.db
5.2M
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-2678-Index.db
822M
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-2678-Data.db
7.3M
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-3283-Index.db
1.2G
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-3283-Data.db
6.7M
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-3951-Index.db
1.1G
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-3951-Data.db
11M 
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-4799-Index.db
1.7G
/raid0/cassandra/data/draios/protobuf_by_agent1-cd071a304beb11e4abf997b38fbf920b/draios-protobuf_by_agent1-tmplink-ka-4799-Data.db
812K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-234-Index.db
122M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-208-Data.db
744K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-739-Index.db
660K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-193-Index.db
796K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-230-Index.db
137M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-230-Data.db
161M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-269-Data.db
139M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-234-Data.db
940K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-786-Index.db
936K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-269-Index.db
161M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-786-Data.db
672K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-197-Index.db
113M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-193-Data.db
116M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-197-Data.db
712K
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-208-Index.db
127M
/raid0/cassandra/data/draios/mounted_fs_by_agent1-d7bf3e304beb11e4abf997b38fbf920b/draios-mounted_fs_by_agent1-tmplink-ka-739-Data.db
776K

[jira] [Commented] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread mck (JIRA)

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

mck commented on CASSANDRA-8032:


{quote}AFAIK this feature isn't really being used by people.{quote}

Then i agree the right decision sounds like to remove it altogether. 
I'm definitely not interested in providing a patch that isn't inline with 
future plans for multitenancy :-)
Is the removal of thrift definite for cassandra-3.0 ? if so i'd be happy to 
provide the simple patch that removes the request_scheduler altogether.

Ok, then everything below is just for clarity's sake…
{quote}finer granularity of control, from read-only vs read-write users on 
keyspaces … Using it for virtual datacenter bulding would be a hack…{quote}

No no, I didn't mean the control of reads vs writes, but rather just the 
scheduler being more flexible by being based upon either user or keyspace.

{quote}… and on its own it's not enough to support true multitenancy…{quote}

This came up for us recently because another c* user recommended using separate 
clusters per application domain.
And given that it's a) all too easy for one keyspace to bring down the whole 
cluster, and b) the difficulty around upgrading c* when it comes to the matrix 
of testing required between keyspaces, use-cases, and all the different clients 
in use, (not to mention potential bugs popping up during the actual rolling 
upgrade process), the idea of using separate clusters certainly carries a lot 
of weight. 

But the problem is that, apart from not working with the theoretical ability of 
C* in that it can provide the same isolation with just separate 
virtual-datacenters, i reckon this can cause a hurdle for the adoption of C*. 
When people are trying out C* if such robustness can only be provided given 
enough nodes to warrant separate clusters you are going to discourage them, 
people naturally believe in the robustness they experience, rather than what 
you promise them. Put this up against the the current trend of microservices 
and devOps-within-each-team, if the cost to each team to use C* each time is 3 
new stand-alone nodes, it isn't a cost-effective solution.

Having to go for separate clusters to achieve this robustness also doesn't 
sell to the analytics advantages of C* – having your analytics in a separate 
virtual-dc is a really sweet feature, whenever you want to do analytics on 
something new just add a replica of it into the analytics dc, wait for the 
streaming to finish, and bang you're off and running… not so easy with separate 
clusters.

{quote}…you need better resource isolation/usage limits support, deep and on 
all levels, to make that happen.{quote}

And indeed that was my next improvement to file (and look into providing a 
patch for), at a minimum i believe one does need to bring compaction into the 
scheduler. 
I think this could provide a welcome start to multitenancy suitable for 
microservices-based enterprises, and a robustness and isolation for smaller 
users that don't yet have enough traffic to warrant separate clusters for each 
keyspace.

Rather long-winded, and just my 2¢ and one user's perspective, but i hope at 
least i've explained it well enough that it's understood. Maybe i can build 
some community support for multitenancy in the CQL world… and maybe there's 
smarter ways of achieving it than the previous request_scheduler approach…

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.

[jira] [Resolved] (CASSANDRA-8059) CQL requests need to go through the request scheduler

2014-10-05 Thread mck (JIRA)

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

mck resolved CASSANDRA-8059.

Resolution: Won't Fix

See the discussion in CASSANDRA-8032

 CQL requests need to go through the request scheduler
 -

 Key: CASSANDRA-8059
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8059
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: mck

 Implementing CASSANDRA-8032 it seems that no cql/cql3 requests go through the 
 request scheduler. I'm guessing they never have?
 If this is to be fixed/implemented then it looks like that there's a number 
 of methods in `CassandraServer` that need to call `schedule(timeout)`.
 The alternative is to remove the request scheduler along with thrift?
 Regarding that choice see my opinion from CASSANDRA-8032…
 {quote}
 … … The request scheduler is a useful concept but i think that setups with 
 enough nodes often favour separate clusters rather than either creating 
 separate virtual datacenters or using the request scheduler. To give the 
 request scheduler another, and more flexible, implementation could especially 
 help those users that don't yet have enough nodes to warrant separate 
 clusters, or even separate virtual datacenters. On such smaller clusters 
 cassandra can still be seen as an unstable technology because poor 
 consumers/schemas can easily affect, even bring down, a whole cluster. … …
 {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-6246) EPaxos

2014-10-05 Thread Blake Eggleston (JIRA)

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

Blake Eggleston commented on CASSANDRA-6246:


I’ve been thinking through how epaxos would be affected by repair, read repair, 
and hints.

Since both the read and write parts of an epaxos instance are executed both 
locally and asynchronously, it’s possible that a repair could write the result 
of an instance to a node before that instance is executed on that node. This 
would cause the decision of an epaxos instance to be different on the node 
being repaired, which could create an inconsistency between nodes. Although 
it’s difficult to imagine an instance taking more time to execute than a 
repair, I don’t think it’s impossible, and would introduce inconsistencies 
during normal operation.

Something that would be more likely to cause problems would be someone 
performing a quorum read on a key that has instances in flight, and triggering 
a read repair on that key. Hints would have a similar problem, but it would 
also mean that people are mixing serialized and unserialized writes 
concurrently.

Having the node sending the repair message include some metadata about the most 
recent executed instance(s) it's aware of is the best solution I've come up 
with so far. If the receiving node is behind, it could work with the sending 
node to catch up before performing the repair.

 EPaxos
 --

 Key: CASSANDRA-6246
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6246
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Blake Eggleston
Priority: Minor

 One reason we haven't optimized our Paxos implementation with Multi-paxos is 
 that Multi-paxos requires leader election and hence, a period of 
 unavailability when the leader dies.
 EPaxos is a Paxos variant that requires (1) less messages than multi-paxos, 
 (2) is particularly useful across multiple datacenters, and (3) allows any 
 node to act as coordinator: 
 http://sigops.org/sosp/sosp13/papers/p358-moraru.pdf
 However, there is substantial additional complexity involved if we choose to 
 implement it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8032:
--

bq. Is the removal of thrift definite for cassandra-3.0 ?

It's not being removed (yet). But Thrift is considered 'frozen' now (see the 
announcement somewhere in the dev mailing list, a few months ago). That is, we 
won't accept any patches for new Thrift-related functionality, or improvements 
to existing Thrift-related features. Only major bug fixes.

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.
 I haven't look into the feasibility of this within the code, but it comes to 
 mind as rather simple, and i would be interested in offering a patch if the 
 idea carries validity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8062) IllegalArgumentException passing blob as tuple value element in list

2014-10-05 Thread Bill Mitchell (JIRA)
Bill Mitchell created CASSANDRA-8062:


 Summary: IllegalArgumentException passing blob as tuple value 
element in list
 Key: CASSANDRA-8062
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8062
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Windows 7, DataStax 2.1.0 Cassandra server, Java 
cassandra-driver-2.1.1 
Reporter: Bill Mitchell


I am using the same table schema as described in earlier reports, e.g., 
CASSANDRA-7105:
CREATE TABLE sr (siteid uuid, listid bigint, partition int, createdate 
timestamp, emailcrypt blob, emailaddr text, properties text, removedate 
timestamp. removeimportid bigint,
PRIMARY KEY ((siteid, listid, partition), createdate, emailcrypt)
) WITH CLUSTERING ORDER BY (createdate DESC, emailcrypt DESC);

I am trying to take advantage of the new Tuple support to issue a query to 
request multiple rows in a single wide row by (createdate,emailcrypt) pair.  I 
declare a new TupleType that covers the clustering columns and then issue an IN 
predicate against a list of these values:
private static final TupleType dateEmailTupleType = 
TupleType.of(DataType.timestamp(), DataType.blob());
...
ListTupleValue partitionKeys = new ArrayList(recipKeys.size());
...
BoundStatement boundStatement = new BoundStatement(preparedStatement);
boundStatement = boundStatement.bind(siteID, partition, listID);
boundStatement.setList(3, partitionKeys);

When I issue a SELECT against this table, the server fails apparently trying to 
break apart the list values:
DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,312 Message.java:420 - 
Received: PREPARE SELECT emailCrypt, emailAddr, removeDate, removeImportID, 
properties FROM sr WHERE siteID = ? AND partition = ? AND listID = ? AND ( 
createDate, emailCrypt ) IN ? ;, v=2
DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,323 Tracing.java:157 - request 
complete
DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,323 Message.java:433 - 
Responding: RESULT PREPARED a18ff9151e8bd3b13b48a0ba56ecb784 
[siteid(testdb_1412536748414, sr), 
org.apache.cassandra.db.marshal.UUIDType][partition(testdb_1412536748414, sr), 
org.apache.cassandra.db.marshal.Int32Type][listid(testdb_1412536748414, sr), 
org.apache.cassandra.db.marshal.LongType][in(createdate,emailcrypt)(testdb_1412536748414,
 sr), 
org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.TupleType(org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.TimestampType),org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.BytesType)))]
 (resultMetadata=[emailcrypt(testdb_1412536748414, sr), 
org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.BytesType)][emailaddr(testdb_1412536748414,
 sr), 
org.apache.cassandra.db.marshal.UTF8Type][removedate(testdb_1412536748414, sr), 
org.apache.cassandra.db.marshal.TimestampType][removeimportid(testdb_1412536748414,
 sr), 
org.apache.cassandra.db.marshal.LongType][properties(testdb_1412536748414, sr), 
org.apache.cassandra.db.marshal.UTF8Type]), v=2
DEBUG [SharedPool-Worker-1] 2014-10-05 14:20:15,363 Message.java:420 - 
Received: EXECUTE a18ff9151e8bd3b13b48a0ba56ecb784 with 4 values at consistency 
QUORUM, v=2
DEBUG [SharedPool-Worker-2] 2014-10-05 14:20:15,380 Message.java:420 - 
Received: EXECUTE a18ff9151e8bd3b13b48a0ba56ecb784 with 4 values at consistency 
QUORUM, v=2
DEBUG [SharedPool-Worker-5] 2014-10-05 14:20:15,402 Message.java:420 - 
Received: EXECUTE a18ff9151e8bd3b13b48a0ba56ecb784 with 4 values at consistency 
QUORUM, v=2
ERROR [SharedPool-Worker-5] 2014-10-05 14:20:16,125 ErrorMessage.java:218 - 
Unexpected exception during request
java.lang.IllegalArgumentException: null
at java.nio.Buffer.limit(Unknown Source) ~[na:1.7.0_25]
at 
org.apache.cassandra.utils.ByteBufferUtil.readBytes(ByteBufferUtil.java:539) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.serializers.CollectionSerializer.readValue(CollectionSerializer.java:122)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.serializers.ListSerializer.deserializeForNativeProtocol(ListSerializer.java:87)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.serializers.ListSerializer.deserializeForNativeProtocol(ListSerializer.java:27)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.serializers.CollectionSerializer.deserialize(CollectionSerializer.java:48)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.marshal.AbstractType.compose(AbstractType.java:66) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.cql3.Tuples$InValue.fromSerialized(Tuples.java:249) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at org.apache.cassandra.cql3.Tuples$InMarker.bind(Tuples.java:394) 

[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-7872:
--

I'd go for some more explicit way, if possible, but ultimately don't see an 
issue with using a feature that's guaranteed to work well on Oracle and Open 
JVMs, and work somehow elsewhere (after all, we don't have an issue w/ using 
Unsafe).

FWIW I feel like CASSANDRA-7705 is also relevant to this issue.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7872) ensure compacted obsolete sstables are not open on node restart and nodetool refresh, even on sstable reference miscounting or deletion tasks are failed.

2014-10-05 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-7872:


The thing is it's not guaranteed it just *happens* to work this way, pretty 
much what I was saying all the way.

 ensure compacted obsolete sstables are not open on node restart and nodetool 
 refresh, even on sstable reference miscounting or deletion tasks are failed.
 -

 Key: CASSANDRA-7872
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7872
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Oleg Anastasyev
Assignee: Oleg Anastasyev
 Fix For: 2.0.11

 Attachments: EnsureNoObsoleteSSTables-7872-v2.0.txt


 Since CASSANDRA-4436 compacted sstables are no more marked with 
 COMPACTED_MARKER file. Instead after they are compacted, DataTracker calls 
 SSTableReader.markObsolete(), but the actual deletion is happening later on 
 SSTableReader.releaseReference().
 This reference counting is very fragile, it is very easy to introduce a 
 hard-to-catch and rare bug, so this reference count never reaches 0 ( like 
 CASSANDRA-6503 for example )
 This means, that very rarely obsolete sstable files are not removed from disk 
 (but are not used anymore by cassandra to read data).
 If more than gc grace time has passed since sstable file was not removed from 
 disk and operator issues either nodetool refresh or just reboots a node, 
 these obsolete files are being discovered and open for read by a node. So 
 deleted data is resurrected, being quickly spread by RR to whole cluster.
 Because consequences are very serious (even a single not removed obsolete 
 sstable file could render your data useless) this patch makes sure no 
 obsolete sstable file can be open for read by:
 1. Removing sstables on CFS init analyzing sstable generations (sstable is 
 removed, if there are another sstable, listing this as ancestor)
 2. Reimplementing COMPACTED_MARKER file for sstable. This marker is created 
 as soon as markObsolete is called. This is neccessary b/c generation info can 
 be lost (when sstables compact to none)
 3. To remove sstables sooner then restart - reimplemented the good old GC 
 phantom reference queue as well. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8021) Improve cqlsh autocomplete for alter keyspace

2014-10-05 Thread Philip Thompson (JIRA)

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

Philip Thompson updated CASSANDRA-8021:
---
Assignee: (was: Philip Thompson)

 Improve cqlsh autocomplete for alter keyspace
 -

 Key: CASSANDRA-8021
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8021
 Project: Cassandra
  Issue Type: Improvement
Reporter: Philip Thompson
Priority: Minor
  Labels: cqlsh, lhf
 Fix For: 2.0.11


 Cqlsh autocomplete stops giving suggestions for the statement
 {code}ALTER KEYSPACE k WITH REPLICATION { 'class' : 'SimpleStrategy', 
 'replication_factor' : 1'};{code} after the word WITH.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8032) User based request scheduler

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8032:
--

As for multitenancy support - you aren't alone here, the request for it comes 
up relatively often. But I believe that it's something that should be designed 
from top to bottom, and it's going to be a major change encapsulating a lot of 
layers, and not from bottom up, via half measures.

I think we'll have it, eventually, but likely not in the 3.x line - there is 
some technical debt to be paid off first, and some more pressing features. That 
said, again, I do agree with you that it would be nice to have, definitely, so 
no need to sell me on it - we are on the same side here. It's just that the 
time for it hasn't come, yet.

 User based request scheduler
 

 Key: CASSANDRA-8032
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8032
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: mck
Assignee: mck
Priority: Minor
  Labels: patch
 Attachments: v1-0001-CASSANDRA-8032-User-based-request-scheduler.txt


 Today only a keyspace based request scheduler exists.
 Post CASSANDRA-4898 it could be possible to implement a request_scheduler 
 based on users (from system_auth.credentials) rather than keyspaces. This 
 could offer a finer granularity of control, from read-only vs read-write 
 users on keyspaces, to application dedicated vs ad-hoc users. Alternatively 
 it could also offer a granularity larger and easier to work with than per 
 keyspace.
 The request scheduler is a useful concept but i think that setups with enough 
 nodes often favour separate clusters rather than either creating separate 
 virtual datacenters or using the request scheduler. To give the request 
 scheduler another, and more flexible, implementation could especially help 
 those users that don't yet have enough nodes to warrant separate clusters, or 
 even separate virtual datacenters. On such smaller clusters cassandra can 
 still be seen as an unstable technology because poor consumers/schemas can 
 easily affect, even bring down, a whole cluster.
 I haven't look into the feasibility of this within the code, but it comes to 
 mind as rather simple, and i would be interested in offering a patch if the 
 idea carries validity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-7256) Error when dropping keyspace.

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko updated CASSANDRA-7256:
-
Attachment: 7256-minimal.txt

 Error when dropping keyspace.  
 ---

 Key: CASSANDRA-7256
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7256
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: ubuntu 3 nodes (had 3 more in 2nd datacenter but removed 
 it)
Reporter: Steven Lowenthal
Assignee: Aleksey Yeschenko
 Attachments: 7256-complete.txt, 7256-minimal.txt


 created a 3 node datacenter  called existing.
 ran cassandra-stress:
 {{cassandra-stress -R NetworkTopologyStrategy -O existing:2 -d existing0 -n 
 200 -k}}
 Added a 2nd datacenter called new with 3 nodes started it with 
 {{auto_bootstrap: false}}
 {code}
 alter keyspace Keyspace1 with replication = 
 {'class':'NetworkTopologyStrategy','existing':2,'new':2};
 {code}
 I then discovered that cassandra-stress --operation=read failed with 
 LOCAL_QUORUM if a node was down in the local datacenter - this occured in 
 both, but should not have, so decided to try again.
 I shut down the new datacenter and removed all 3 nodes.  I then tried to drop 
 the Keyspace1 keyspace.  cqlsh disconnected, and the log shows the error 
 below.
 {code}
 ERROR [MigrationStage:1] 2014-05-16 23:57:03,085 CassandraDaemon.java (line 
 198) Exception in thread Thread[MigrationStage:1,5,main]
 java.lang.IllegalStateException: One row required, 0 found
 at org.apache.cassandra.cql3.UntypedResultSet.one(UntypedResultSet.java:53)
 at org.apache.cassandra.config.KSMetaData.fromSchema(KSMetaData.java:263)
 at org.apache.cassandra.db.DefsTables.mergeKeyspaces(DefsTables.java:227)
 at org.apache.cassandra.db.DefsTables.mergeSchema(DefsTables.java:182)
 at 
 org.apache.cassandra.service.MigrationManager$2.runMayThrow(MigrationManager.java:303)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
 at java.util.concurrent.FutureTask.run(FutureTask.java:262)
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
 at java.lang.Thread.run(Thread.java:744)
 {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-7256) Error when dropping keyspace.

2014-10-05 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko updated CASSANDRA-7256:
-
Attachment: 7256-complete.txt

 Error when dropping keyspace.  
 ---

 Key: CASSANDRA-7256
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7256
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: ubuntu 3 nodes (had 3 more in 2nd datacenter but removed 
 it)
Reporter: Steven Lowenthal
Assignee: Aleksey Yeschenko
 Attachments: 7256-complete.txt, 7256-minimal.txt


 created a 3 node datacenter  called existing.
 ran cassandra-stress:
 {{cassandra-stress -R NetworkTopologyStrategy -O existing:2 -d existing0 -n 
 200 -k}}
 Added a 2nd datacenter called new with 3 nodes started it with 
 {{auto_bootstrap: false}}
 {code}
 alter keyspace Keyspace1 with replication = 
 {'class':'NetworkTopologyStrategy','existing':2,'new':2};
 {code}
 I then discovered that cassandra-stress --operation=read failed with 
 LOCAL_QUORUM if a node was down in the local datacenter - this occured in 
 both, but should not have, so decided to try again.
 I shut down the new datacenter and removed all 3 nodes.  I then tried to drop 
 the Keyspace1 keyspace.  cqlsh disconnected, and the log shows the error 
 below.
 {code}
 ERROR [MigrationStage:1] 2014-05-16 23:57:03,085 CassandraDaemon.java (line 
 198) Exception in thread Thread[MigrationStage:1,5,main]
 java.lang.IllegalStateException: One row required, 0 found
 at org.apache.cassandra.cql3.UntypedResultSet.one(UntypedResultSet.java:53)
 at org.apache.cassandra.config.KSMetaData.fromSchema(KSMetaData.java:263)
 at org.apache.cassandra.db.DefsTables.mergeKeyspaces(DefsTables.java:227)
 at org.apache.cassandra.db.DefsTables.mergeSchema(DefsTables.java:182)
 at 
 org.apache.cassandra.service.MigrationManager$2.runMayThrow(MigrationManager.java:303)
 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
 at java.util.concurrent.FutureTask.run(FutureTask.java:262)
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
 at java.lang.Thread.run(Thread.java:744)
 {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-7676) bin/cassandra should complain if $JAVA is empty or not an executable

2014-10-05 Thread Michael Shuler (JIRA)

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

Michael Shuler updated CASSANDRA-7676:
--
Labels: qa-resolved  (was: )

 bin/cassandra should complain if $JAVA is empty or not an executable
 

 Key: CASSANDRA-7676
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7676
 Project: Cassandra
  Issue Type: Improvement
Reporter: Robert Stupp
Assignee: Robert Stupp
Priority: Trivial
  Labels: qa-resolved
 Fix For: 2.0.11, 2.1.1

 Attachments: 7676v2.txt, java_not_found-fix.txt


 If JAVA_HOME points to a non-existing directory, bin/cassandra just complains 
 with {{[node1 ERROR] /Users/snazy/.ccm/test/node1/bin/cassandra: line 151: 
 exec: : not found}} (via ccm).
 Patch checks if {{$JAVA}} is empty or does not point to an executable and 
 prints appropriate error message.
 (Patch against trunk)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7676) bin/cassandra should complain if $JAVA is empty or not an executable

2014-10-05 Thread Michael Shuler (JIRA)

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

Michael Shuler commented on CASSANDRA-7676:
---

Looks good to me :)

 bin/cassandra should complain if $JAVA is empty or not an executable
 

 Key: CASSANDRA-7676
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7676
 Project: Cassandra
  Issue Type: Improvement
Reporter: Robert Stupp
Assignee: Robert Stupp
Priority: Trivial
  Labels: qa-resolved
 Fix For: 2.0.11, 2.1.1

 Attachments: 7676v2.txt, java_not_found-fix.txt


 If JAVA_HOME points to a non-existing directory, bin/cassandra just complains 
 with {{[node1 ERROR] /Users/snazy/.ccm/test/node1/bin/cassandra: line 151: 
 exec: : not found}} (via ccm).
 Patch checks if {{$JAVA}} is empty or does not point to an executable and 
 prints appropriate error message.
 (Patch against trunk)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-7676) bin/cassandra should complain if $JAVA is empty or not an executable

2014-10-05 Thread Michael Shuler (JIRA)

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

Michael Shuler updated CASSANDRA-7676:
--
Fix Version/s: 2.1.1
   2.0.11

 bin/cassandra should complain if $JAVA is empty or not an executable
 

 Key: CASSANDRA-7676
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7676
 Project: Cassandra
  Issue Type: Improvement
Reporter: Robert Stupp
Assignee: Robert Stupp
Priority: Trivial
  Labels: qa-resolved
 Fix For: 2.0.11, 2.1.1

 Attachments: 7676v2.txt, java_not_found-fix.txt


 If JAVA_HOME points to a non-existing directory, bin/cassandra just complains 
 with {{[node1 ERROR] /Users/snazy/.ccm/test/node1/bin/cassandra: line 151: 
 exec: : not found}} (via ccm).
 Patch checks if {{$JAVA}} is empty or does not point to an executable and 
 prints appropriate error message.
 (Patch against trunk)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)