[jira] [Commented] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-8683:
-

I am struggling to reproduce this test locally, but by spinning on it I have 
produced:

{noformat}
[junit] Testcase: 
testValidationMultipleSSTablePerLevel(org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest):
  Caused an ERROR
[junit] java.lang.AssertionError: Memory was freed
[junit] java.util.concurrent.ExecutionException: java.lang.AssertionError: 
Memory was freed
[junit] at java.util.concurrent.FutureTask.report(FutureTask.java:122)
[junit] at java.util.concurrent.FutureTask.get(FutureTask.java:188)
[junit] at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest.testValidationMultipleSSTablePerLevel(LeveledCompactionStrategyTest.java:184)
[junit] Caused by: java.lang.AssertionError: Memory was freed
[junit] at 
org.apache.cassandra.io.util.Memory.checkPosition(Memory.java:281)
[junit] at org.apache.cassandra.io.util.Memory.getInt(Memory.java:233)
[junit] at 
org.apache.cassandra.io.sstable.IndexSummary.getPositionInSummary(IndexSummary.java:118)
[junit] at 
org.apache.cassandra.io.sstable.IndexSummary.getKey(IndexSummary.java:123)
[junit] at 
org.apache.cassandra.io.sstable.format.SSTableReader$9$1.next(SSTableReader.java:1249)
[junit] at 
org.apache.cassandra.io.sstable.format.SSTableReader$9$1.next(SSTableReader.java:1226)
[junit] at 
com.google.common.collect.Iterators$5.next(Iterators.java:553)
[junit] at 
org.apache.cassandra.repair.Validator.prepare(Validator.java:89)
[junit] at 
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1007)
[junit] at 
org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:95)
[junit] at 
org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:591)
[junit] at java.util.concurrent.FutureTask.run(FutureTask.java:262)
[junit] at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[junit] at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[junit] at java.lang.Thread.run(Thread.java:745)
{noformat}

Which may or may not be related, but is certainly problematic.

 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the repairing files for non-incremental repairs, but the bug should exist in 
 2.1 as well)



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


[2/3] cassandra git commit: rpc_interface and listen_interface generate NPE on startup when specified interface doesn't exist

2015-01-27 Thread snazy
rpc_interface and listen_interface generate NPE on startup when specified 
interface doesn't exist

Patch by Ariel Weisberg; reviewed by Robert Stupp for CASSANDRA-8677


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

Branch: refs/heads/trunk
Commit: 3e5edb82c73b7b7c6e1d1e970fb764c3e3158da6
Parents: 33297ba
Author: Ariel Weisberg ariel.weisb...@datastax.com
Authored: Tue Jan 27 13:30:47 2015 +0100
Committer: Robert Stupp sn...@snazy.de
Committed: Tue Jan 27 13:30:47 2015 +0100

--
 .../cassandra/config/DatabaseDescriptor.java| 46 ++--
 1 file changed, 23 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e5edb82/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
--
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index db33dcc..1dd1688 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -158,6 +158,27 @@ public class DatabaseDescriptor
 return loader.loadConfig();
 }
 
+private static InetAddress getNetworkInterfaceAddress(String intf, String 
configName) throws ConfigurationException
+{
+try
+{
+NetworkInterface ni = NetworkInterface.getByName(intf);
+if (ni == null)
+throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ could not be found);
+EnumerationInetAddress addrs = ni.getInetAddresses();
+if (!addrs.hasMoreElements())
+throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ was found, but had no addresses);
+InetAddress retval = listenAddress = addrs.nextElement();
+if (addrs.hasMoreElements())
+throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ can't have more than one address);
+return retval;
+}
+catch (SocketException e)
+{
+throw new ConfigurationException(Configured  + configName +  
\ + intf + \ caused an exception, e);
+}
+}
+
 private static void applyConfig(Config config) throws 
ConfigurationException
 {
 conf = config;
@@ -326,18 +347,7 @@ public class DatabaseDescriptor
 }
 else if (conf.listen_interface != null)
 {
-try
-{
-EnumerationInetAddress addrs = 
NetworkInterface.getByName(conf.listen_interface).getInetAddresses();
-listenAddress = addrs.nextElement();
-if (addrs.hasMoreElements())
-throw new ConfigurationException(Interface  + 
conf.listen_interface + can't have more than one address);
-}
-catch (SocketException e)
-{
-throw new ConfigurationException(Unknown network interface in 
listen_interface  + conf.listen_interface);
-}
-
+listenAddress = getNetworkInterfaceAddress(conf.listen_interface, 
listen_interface);
 }
 
 /* Gossip Address to broadcast */
@@ -374,17 +384,7 @@ public class DatabaseDescriptor
 }
 else if (conf.rpc_interface != null)
 {
-try
-{
-EnumerationInetAddress addrs = 
NetworkInterface.getByName(conf.rpc_interface).getInetAddresses();
-rpcAddress = addrs.nextElement();
-if (addrs.hasMoreElements())
-throw new ConfigurationException(Interface  + 
conf.rpc_interface + can't have more than one address);
-}
-catch (SocketException e)
-{
-throw new ConfigurationException(Unknown network interface in 
rpc_interface  + conf.rpc_interface);
-}
+listenAddress = getNetworkInterfaceAddress(conf.rpc_interface, 
rpc_interface);
 }
 else
 {



[1/3] cassandra git commit: rpc_interface and listen_interface generate NPE on startup when specified interface doesn't exist

2015-01-27 Thread snazy
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 33297bad0 - 3e5edb82c
  refs/heads/trunk 6068daea8 - dd959f0f5


rpc_interface and listen_interface generate NPE on startup when specified 
interface doesn't exist

Patch by Ariel Weisberg; reviewed by Robert Stupp for CASSANDRA-8677


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

Branch: refs/heads/cassandra-2.1
Commit: 3e5edb82c73b7b7c6e1d1e970fb764c3e3158da6
Parents: 33297ba
Author: Ariel Weisberg ariel.weisb...@datastax.com
Authored: Tue Jan 27 13:30:47 2015 +0100
Committer: Robert Stupp sn...@snazy.de
Committed: Tue Jan 27 13:30:47 2015 +0100

--
 .../cassandra/config/DatabaseDescriptor.java| 46 ++--
 1 file changed, 23 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e5edb82/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
--
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index db33dcc..1dd1688 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -158,6 +158,27 @@ public class DatabaseDescriptor
 return loader.loadConfig();
 }
 
+private static InetAddress getNetworkInterfaceAddress(String intf, String 
configName) throws ConfigurationException
+{
+try
+{
+NetworkInterface ni = NetworkInterface.getByName(intf);
+if (ni == null)
+throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ could not be found);
+EnumerationInetAddress addrs = ni.getInetAddresses();
+if (!addrs.hasMoreElements())
+throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ was found, but had no addresses);
+InetAddress retval = listenAddress = addrs.nextElement();
+if (addrs.hasMoreElements())
+throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ can't have more than one address);
+return retval;
+}
+catch (SocketException e)
+{
+throw new ConfigurationException(Configured  + configName +  
\ + intf + \ caused an exception, e);
+}
+}
+
 private static void applyConfig(Config config) throws 
ConfigurationException
 {
 conf = config;
@@ -326,18 +347,7 @@ public class DatabaseDescriptor
 }
 else if (conf.listen_interface != null)
 {
-try
-{
-EnumerationInetAddress addrs = 
NetworkInterface.getByName(conf.listen_interface).getInetAddresses();
-listenAddress = addrs.nextElement();
-if (addrs.hasMoreElements())
-throw new ConfigurationException(Interface  + 
conf.listen_interface + can't have more than one address);
-}
-catch (SocketException e)
-{
-throw new ConfigurationException(Unknown network interface in 
listen_interface  + conf.listen_interface);
-}
-
+listenAddress = getNetworkInterfaceAddress(conf.listen_interface, 
listen_interface);
 }
 
 /* Gossip Address to broadcast */
@@ -374,17 +384,7 @@ public class DatabaseDescriptor
 }
 else if (conf.rpc_interface != null)
 {
-try
-{
-EnumerationInetAddress addrs = 
NetworkInterface.getByName(conf.rpc_interface).getInetAddresses();
-rpcAddress = addrs.nextElement();
-if (addrs.hasMoreElements())
-throw new ConfigurationException(Interface  + 
conf.rpc_interface + can't have more than one address);
-}
-catch (SocketException e)
-{
-throw new ConfigurationException(Unknown network interface in 
rpc_interface  + conf.rpc_interface);
-}
+listenAddress = getNetworkInterfaceAddress(conf.rpc_interface, 
rpc_interface);
 }
 else
 {



[jira] [Commented] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8683:


When I debugged this, the reason that we fell through all the way to that 
return null was that the index only contained the last key, then we were at 
EOF and ended the loop

 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the repairing files for non-incremental repairs, but the bug should exist in 
 2.1 as well)



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


[jira] [Updated] (CASSANDRA-8623) sstablesplit fails *randomly* with Data component is missing

2015-01-27 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson updated CASSANDRA-8623:
---
Attachment: 8623-v2.patch

new patch that avoids submitting tasks if the executor is shut down

 sstablesplit fails *randomly* with Data component is missing
 

 Key: CASSANDRA-8623
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8623
 Project: Cassandra
  Issue Type: Bug
Reporter: Alan Boudreault
Assignee: Marcus Eriksson
 Attachments: 
 0001-make-sure-we-finish-compactions-before-waiting-for-d.patch, 
 8623-v2.patch, output.log, output2.log


 I'm experiencing an issue related to sstablesplit. I would like to understand 
 if I am doing something wrong or there is an issue in the split process. The 
 process fails randomly with the following exception:
 {code}
 ERROR 02:17:36 Error in ThreadPoolExecutor
 java.lang.AssertionError: Data component is missing for 
 sstable./tools/bin/../../data/data/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/system-compactions_in_progress-ka-16
 {code}
 See attached output.log file. The process never stops after this exception 
 and I've also seen the dataset growing indefinitely (number of sstables).  
 * I have not been able to reproduce the issue with a single sstablesplit 
 command. ie, specifying all files with glob matching.
 * I can reproduce the bug if I call multiple sstablesplit one file at the 
 time (the way ccm does)
 Here is the test case file to reproduce the bug:
 https://drive.google.com/file/d/0BwZ_GPM33j6KdVh0NTdkOWV2R1E/view?usp=sharing
 1. Download the split_issue.tar.gz file. It includes latest cassandra-2.1 
 branch binaries.
 2. Extract it
 3. CD inside the use case directory
 4. Download the dataset (2G) just to be sure we have the same thing, and 
 place it in the working directory.
https://docs.google.com/uc?id=0BwZ_GPM33j6KV3ViNnpPcVFndUUexport=download
 5. The first time, run ./test.sh. This will setup and run a test.
 6. The next times, you can only run ./test --no-setup . This will only reset 
 the dataset as its initial state and re-run the test. You might have to run 
 the tests some times before experiencing it... but I'm always able with only 
 2-3 runs.



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


[jira] [Commented] (CASSANDRA-8422) cassandra won't start up due to Unable to gossip with any seeds on the decommissioned node

2015-01-27 Thread kris (JIRA)

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

kris commented on CASSANDRA-8422:
-

we are using dse 4.6 and facing the same issue.

 cassandra won't start up due to Unable to gossip with any seeds on the 
 decommissioned node
 

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

 - 2-node
   * nodeA - seed
   * nodeB
 1. decommission nodeB from the cluster with nodetool
when it's finished, kill cassandra process on nodeB
 2. delete data from commit/cache/data directories on nodeB
 3. try to start cassandra on nodeB (first time)
= FAILED with Unable to gossip with any seeds
 4. try to start cassandra on nodeB (second time)
   = OK
 It was not a one-time shot. I tried it a several times and encountered the 
 same issue for some reason.
 {code}
 ERROR [main] 2014-11-27 18:44:55,017 CassandraDaemon.java (line 513) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
 at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1211)
 at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:445)
 at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:659)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:611)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:503)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
  INFO [StorageServiceShutdownHook] 2014-11-27 18:44:55,076 Gossiper.java 
 (line 1307) Announcing shutdown
 {code}



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


[jira] [Commented] (CASSANDRA-8688) Standalone sstableupgrade tool throws exception

2015-01-27 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8688:


+1

 Standalone sstableupgrade tool throws exception
 ---

 Key: CASSANDRA-8688
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8688
 Project: Cassandra
  Issue Type: Bug
Reporter: Yuki Morishita
Assignee: Yuki Morishita
Priority: Minor
 Fix For: 2.1.3

 Attachments: 
 0001-remove-size-calculation-from-standalone-sstbaleupgra.patch


 Standalone sstableupgrade throws following exception:
 {code}
 Exception in thread main java.lang.AssertionError: Incoherent new size -1 
 replacing 
 [SSTableReader(path='/cassandra/bin/./../data/data/ks/cf-0f578640a59211e4a5a2ef9f87394ca6/ks-cf-jb-53881-Data.db')]
  by [] in View(pending_count=0, sstables=[], compacting=[])
 at org.apache.cassandra.db.DataTracker$View.newSSTables(DataTracker.java:741)
 at org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:713)
 at 
 org.apache.cassandra.db.DataTracker.removeSSTablesFromTracker(DataTracker.java:418)
 at org.apache.cassandra.db.DataTracker.markCompactedSSTablesReplaced(
 DataTracker.java:273)
   at org.apache.cassandra.db.compaction.Upgrader.upgrade(Upgrader.java:96)
 {code}
 Upgrader is trying to calculate SSTables size dispite the tool is meant to be 
 used offline.
 This is introduced in CASSANDRA-7852 and only affects cassandra 2.1.2.
 Online upgrade (nodetool upgradesstables) is not affected.



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


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

2015-01-27 Thread Mark Cooper (JIRA)

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

Mark Cooper commented on CASSANDRA-8061:


Thanks [~benedict]

For the record this branch seems to be have different and unfortunately worse 
behavior as far as not deleting tmplink files.

-rw-r--r--  2 mcooper rmrgroup   30M Jan 26 20:19 
leanpixels-funnelpagesv2-tmplink-ka-45813-Data.db
-rw-r--r--  2 mcooper rmrgroup   49M Jan 26 20:19 
leanpixels-funnelpagesv2-tmplink-ka-45812-Data.db
-rw-r--r--  2 mcooper rmrgroup   30M Jan 26 20:19 
leanpixels-funnelpagesv2-tmplink-ka-45815-Data.db
-rw-r--r--  2 mcooper rmrgroup   49M Jan 26 20:19 
leanpixels-funnelpagesv2-tmplink-ka-45814-Data.db
...
-rw-r--r--  2 mcooper rmrgroup   30M Jan 27 18:01 
leanpixels-funnelpagesv2-tmplink-ka-46452-Data.db
-rw-r--r--  2 mcooper rmrgroup   49M Jan 27 18:01 
leanpixels-funnelpagesv2-tmplink-ka-46451-Data.db
-rw-r--r--  2 mcooper rmrgroup   30M Jan 27 18:06 
leanpixels-funnelpagesv2-tmplink-ka-46455-Data.db
-rw-r--r--  2 mcooper rmrgroup   49M Jan 27 18:06 
leanpixels-funnelpagesv2-tmplink-ka-46454-Data.db

...I'm not sure if this is useful but this problem appears to be worse as our 
the amount of data on disk grows.

 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
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: 8061_v1.txt, 8248-thread_dump.txt


 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:
 {noformat}
 $ 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  
 

[jira] [Commented] (CASSANDRA-8535) java.lang.RuntimeException: Failed to rename XXX to YYY

2015-01-27 Thread Joshua McKenzie (JIRA)

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

Joshua McKenzie commented on CASSANDRA-8535:


[~krummas] / [~benedict]: Either of you care to take review on this?  It should 
be pretty trivial.

I expect this code to morph into a check for platform + mmap'ed indexAccessMode 
for 3.1 when re-instating memory mapped I/O on Windows so the logic isn't 
throwaway thankfully.

 java.lang.RuntimeException: Failed to rename XXX to YYY
 ---

 Key: CASSANDRA-8535
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8535
 Project: Cassandra
  Issue Type: Bug
 Environment: Windows 2008 X64
Reporter: Leonid Shalupov
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: 8535_v1.txt


 {code}
 java.lang.RuntimeException: Failed to rename 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-tmp-ka-5-Index.db
  to 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-ka-5-Index.db
   at 
 org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:170) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:154) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.java:569) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.java:561) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.java:535) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.finish(SSTableWriter.java:470) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableRewriter.finishAndMaybeThrow(SSTableRewriter.java:349)
  ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewriter.java:324)
  ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewriter.java:304)
  ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:200)
  ~[main/:na]
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
 ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:75)
  ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
  ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:226)
  ~[main/:na]
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
 ~[na:1.7.0_45]
   at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
 ~[na:1.7.0_45]
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  ~[na:1.7.0_45]
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  [na:1.7.0_45]
   at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
 Caused by: java.nio.file.FileSystemException: 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-tmp-ka-5-Index.db
  - 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-ka-5-Index.db:
  The process cannot access the file because it is being used by another 
 process.
   at 
 sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86) 
 ~[na:1.7.0_45]
   at 
 sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) 
 ~[na:1.7.0_45]
   at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301) 
 ~[na:1.7.0_45]
   at 
 sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287) 
 ~[na:1.7.0_45]
   at java.nio.file.Files.move(Files.java:1345) ~[na:1.7.0_45]
   at 
 org.apache.cassandra.io.util.FileUtils.atomicMoveWithFallback(FileUtils.java:184)
  ~[main/:na]
   at 
 org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:166) 
 ~[main/:na]
   ... 18 common frames omitted
 {code}



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


[jira] [Commented] (CASSANDRA-8684) Replace usage of Adler32 with CRC32

2015-01-27 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg commented on CASSANDRA-8684:
---

And it is yet more complicated. Linux Haswell results track with OS X. Go 
figure! Tested on a c4.8xlarge.

!https://docs.google.com/spreadsheets/d/1cxf-V4b8dXdz1vLb5ySUNxK09bukDHHpq79a09xHw20/pubchart?oid=2070067916format=image!

If you are forward thinking you might stick with CRC32 as fast enough, and soon 
to be much faster. The xxhash API right now also doesn't handle direct byte 
buffers.

And what is the magic sauce that makes CRC32Ex outperform CRC32 even though 
CRC32Ex is just a derived class of CRC32?

 Replace usage of Adler32 with CRC32
 ---

 Key: CASSANDRA-8684
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8684
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Ariel Weisberg
Assignee: Ariel Weisberg
 Attachments: CRCBenchmark.java, PureJavaCrc32.java, Sample.java


 I could not find a situation in which Adler32 outperformed PureJavaCrc32 much 
 less the intrinsic from Java 8. For small allocations PureJavaCrc32 was much 
 faster probably due to the JNI overhead of invoking the native Adler32 
 implementation where the array has to be allocated and copied.
 I tested on a 65w Sandy Bridge i5 running Ubuntu 14.04 with JDK 1.7.0_71 as 
 well as a c3.8xlarge running Ubuntu 14.04.
 I think it makes sense to stop using Adler32 when generating new checksums.
 c3.8xlarge, results are time in milliseconds, lower is better
 ||Allocation size|Adler32|CRC32|PureJavaCrc32||
 |64|47636|46075|25782|
 |128|36755|36712|23782|
 |256|31194|32211|22731|
 |1024|27194|28792|22010|
 |1048576|25941|27807|21808|
 |536870912|25957|27840|21836|
 i5
 ||Allocation size|Adler32|CRC32|PureJavaCrc32||
 |64|50539|50466|26826|
 |128|37092|38533|24553|
 |256|30630|32938|23459|
 |1024|26064|29079|22592|
 |1048576|24357|27911|22481|
 |536870912|24838|28360|22853|
 Another fun fact. Performance of the CRC32 intrinsic appears to double from 
 Sandy Bridge - Haswell. Unless I am measuring something different when going 
 from Linux/Sandy to Haswell/OS X.
 The intrinsic/JDK 8 implementation also operates against DirectByteBuffers 
 better and coding against the wrapper will get that boost when run with Java 
 8.



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


[jira] [Commented] (CASSANDRA-8337) mmap underflow during validation compaction

2015-01-27 Thread Joshua McKenzie (JIRA)

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

Joshua McKenzie commented on CASSANDRA-8337:


[~sterligovak]: Any chance you'll be able to reproduce with the attached patch 
in the near future?  Unfortunately we haven't seen nor reproduced this on this 
side.

If not, I can archive this ticket until such time as you can revisit this with 
us.  Thanks.

 mmap underflow during validation compaction
 ---

 Key: CASSANDRA-8337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8337
 Project: Cassandra
  Issue Type: Bug
Reporter: Alexander Sterligov
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: 8337_v1.txt, thread_dump


 During full parallel repair I often get errors like the following
 {quote}
 [2014-11-19 01:02:39,355] Repair session 116beaf0-6f66-11e4-afbb-c1c082008cbe 
 for range (3074457345618263602,-9223372036854775808] failed with error 
 org.apache.cassandra.exceptions.RepairException: [repair 
 #116beaf0-6f66-11e4-afbb-c1c082008cbe on iss/target_state_history, 
 (3074457345618263602,-9223372036854775808]] Validation failed in 
 /95.108.242.19
 {quote}
 At the log of the node there are always same exceptions:
 {quote}
 ERROR [ValidationExecutor:2] 2014-11-19 01:02:10,847 
 JVMStabilityInspector.java:94 - JVM state determined to be unstable.  Exiting 
 forcefully due to:
 org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
 mmap segment underflow; remaining is 15 but 47 requested
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1518)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1385)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPositionsForRanges(SSTableReader.java:1315)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1706)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1694)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:276)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getScanners(WrappingCompactionStrategy.java:320)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:917)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:97)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:557)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
 ~[na:1.7.0_51]
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  ~[na:1.7.0_51]
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  [na:1.7.0_51]
 at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
 Caused by: java.io.IOException: mmap segment underflow; remaining is 15 but 
 47 requested
 at 
 org.apache.cassandra.io.util.MappedFileDataInput.readBytes(MappedFileDataInput.java:135)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:348) 
 ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:327)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1460)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 ... 13 common frames omitted
 {quote}
 Now i'm using die disk_failure_policy to determine such conditions faster, 
 but I get them even with stop policy.
 Streams related to host with such exception are hanged. Thread dump is 
 attached. Only restart helps.
 After retry I get errors from other nodes.
 scrub doesn't help and report that sstables are ok.
 Sequential repairs doesn't cause such exceptions.
 Load is about 1000 write rps and 50 read rps per node.



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


[jira] [Commented] (CASSANDRA-8535) java.lang.RuntimeException: Failed to rename XXX to YYY

2015-01-27 Thread Maxim Podkolzine (JIRA)

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

Maxim Podkolzine commented on CASSANDRA-8535:
-

Joshua, no, I haven't run a scrub. There were several more exceptions, like 
that one. Let's discuss in a separate issue.
But I did not see {{FileSystemException}}.

I think the second exception is not relevant now, I have an idea, what might 
have caused it. Seems it is not Cassandra fault.

 java.lang.RuntimeException: Failed to rename XXX to YYY
 ---

 Key: CASSANDRA-8535
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8535
 Project: Cassandra
  Issue Type: Bug
 Environment: Windows 2008 X64
Reporter: Leonid Shalupov
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: 8535_v1.txt


 {code}
 java.lang.RuntimeException: Failed to rename 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-tmp-ka-5-Index.db
  to 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-ka-5-Index.db
   at 
 org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:170) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:154) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.java:569) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.java:561) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.java:535) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.finish(SSTableWriter.java:470) 
 ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableRewriter.finishAndMaybeThrow(SSTableRewriter.java:349)
  ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewriter.java:324)
  ~[main/:na]
   at 
 org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewriter.java:304)
  ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:200)
  ~[main/:na]
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
 ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:75)
  ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
  ~[main/:na]
   at 
 org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:226)
  ~[main/:na]
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
 ~[na:1.7.0_45]
   at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
 ~[na:1.7.0_45]
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  ~[na:1.7.0_45]
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  [na:1.7.0_45]
   at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
 Caused by: java.nio.file.FileSystemException: 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-tmp-ka-5-Index.db
  - 
 build\test\cassandra\data;0\system\schema_keyspaces-b0f2235744583cdb9631c43e59ce3676\system-schema_keyspaces-ka-5-Index.db:
  The process cannot access the file because it is being used by another 
 process.
   at 
 sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86) 
 ~[na:1.7.0_45]
   at 
 sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) 
 ~[na:1.7.0_45]
   at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301) 
 ~[na:1.7.0_45]
   at 
 sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287) 
 ~[na:1.7.0_45]
   at java.nio.file.Files.move(Files.java:1345) ~[na:1.7.0_45]
   at 
 org.apache.cassandra.io.util.FileUtils.atomicMoveWithFallback(FileUtils.java:184)
  ~[main/:na]
   at 
 org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:166) 
 ~[main/:na]
   ... 18 common frames omitted
 {code}



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


[jira] [Resolved] (CASSANDRA-8390) The process cannot access the file because it is being used by another process

2015-01-27 Thread Joshua McKenzie (JIRA)

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

Joshua McKenzie resolved CASSANDRA-8390.

Resolution: Cannot Reproduce

Closing out as cannot reproduce for now.  [~alexander_radzin] / 
[~ilya.komolkin]: if either of you can get us more / new information about this 
we can revisit it, but thus far it looks like it's specific to your environment 
and possibly the file system drivers active on your machines.

 The process cannot access the file because it is being used by another process
 --

 Key: CASSANDRA-8390
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8390
 Project: Cassandra
  Issue Type: Bug
Reporter: Ilya Komolkin
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: NoHostAvailableLogs.zip


 {code}21:46:27.810 [NonPeriodicTasks:1] ERROR o.a.c.service.CassandraDaemon - 
 Exception in thread Thread[NonPeriodicTasks:1,5,main]
 org.apache.cassandra.io.FSWriteError: java.nio.file.FileSystemException: 
 E:\Upsource_12391\data\cassandra\data\kernel\filechangehistory_t-a277b560764611e48c8e4915424c75fe\kernel-filechangehistory_t-ka-33-Index.db:
  The process cannot access the file because it is being used by another 
 process.
  
 at 
 org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:135) 
 ~[cassandra-all-2.1.1.jar:2.1.1]
 at 
 org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:121) 
 ~[cassandra-all-2.1.1.jar:2.1.1]
 at 
 org.apache.cassandra.io.sstable.SSTable.delete(SSTable.java:113) 
 ~[cassandra-all-2.1.1.jar:2.1.1]
 at 
 org.apache.cassandra.io.sstable.SSTableDeletingTask.run(SSTableDeletingTask.java:94)
  ~[cassandra-all-2.1.1.jar:2.1.1]
 at 
 org.apache.cassandra.io.sstable.SSTableReader$6.run(SSTableReader.java:664) 
 ~[cassandra-all-2.1.1.jar:2.1.1]
 at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
 ~[na:1.7.0_71]
 at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
 ~[na:1.7.0_71]
 at 
 java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:178)
  ~[na:1.7.0_71]
 at 
 java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:292)
  ~[na:1.7.0_71]
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  ~[na:1.7.0_71]
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  [na:1.7.0_71]
 at java.lang.Thread.run(Thread.java:745) [na:1.7.0_71]
 Caused by: java.nio.file.FileSystemException: 
 E:\Upsource_12391\data\cassandra\data\kernel\filechangehistory_t-a277b560764611e48c8e4915424c75fe\kernel-filechangehistory_t-ka-33-Index.db:
  The process cannot access the file because it is being used by another 
 process.
  
 at 
 sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86) 
 ~[na:1.7.0_71]
 at 
 sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) 
 ~[na:1.7.0_71]
 at 
 sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102) 
 ~[na:1.7.0_71]
 at 
 sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
  ~[na:1.7.0_71]
 at 
 sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103)
  ~[na:1.7.0_71]
 at java.nio.file.Files.delete(Files.java:1079) ~[na:1.7.0_71]
 at 
 org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:131) 
 ~[cassandra-all-2.1.1.jar:2.1.1]
 ... 11 common frames omitted{code}



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


[jira] [Updated] (CASSANDRA-8657) long-test LongCompactionsTest fails

2015-01-27 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis updated CASSANDRA-8657:
--
Assignee: Carl Yeksigian

 long-test LongCompactionsTest fails
 ---

 Key: CASSANDRA-8657
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8657
 Project: Cassandra
  Issue Type: Test
  Components: Tests
Reporter: Michael Shuler
Assignee: Carl Yeksigian
Priority: Minor
 Fix For: 2.1.3, 2.0.13

 Attachments: system.log


 Same error on 3 of the 4 tests in this suite - failure is the same for 2.0 
 and 2.1 branch:
 {noformat}
 [junit] Testsuite: org.apache.cassandra.db.compaction.LongCompactionsTest
 [junit] Tests run: 4, Failures: 3, Errors: 0, Skipped: 0, Time elapsed: 
 27.294 sec
 [junit] 
 [junit] Testcase: 
 testCompactionMany(org.apache.cassandra.db.compaction.LongCompactionsTest):   
 FAILED
 [junit] 
 /tmp/Keyspace14247587528884809907Standard1/Keyspace1/Keyspace1-Standard1-jb-0-Data.db
  is not correctly marked compacting
 [junit] junit.framework.AssertionFailedError: 
 /tmp/Keyspace14247587528884809907Standard1/Keyspace1/Keyspace1-Standard1-jb-0-Data.db
  is not correctly marked compacting
 [junit] at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.init(AbstractCompactionTask.java:49)
 [junit] at 
 org.apache.cassandra.db.compaction.CompactionTask.init(CompactionTask.java:47)
 [junit] at 
 org.apache.cassandra.db.compaction.LongCompactionsTest.testCompaction(LongCompactionsTest.java:102)
 [junit] at 
 org.apache.cassandra.db.compaction.LongCompactionsTest.testCompactionMany(LongCompactionsTest.java:67)
 [junit] 
 [junit] 
 [junit] Testcase: 
 testCompactionSlim(org.apache.cassandra.db.compaction.LongCompactionsTest):   
 FAILED
 [junit] 
 /tmp/Keyspace13809058557206351042Standard1/Keyspace1/Keyspace1-Standard1-jb-0-Data.db
  is not correctly marked compacting
 [junit] junit.framework.AssertionFailedError: 
 /tmp/Keyspace13809058557206351042Standard1/Keyspace1/Keyspace1-Standard1-jb-0-Data.db
  is not correctly marked compacting
 [junit] at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.init(AbstractCompactionTask.java:49)
 [junit] at 
 org.apache.cassandra.db.compaction.CompactionTask.init(CompactionTask.java:47)
 [junit] at 
 org.apache.cassandra.db.compaction.LongCompactionsTest.testCompaction(LongCompactionsTest.java:102)
 [junit] at 
 org.apache.cassandra.db.compaction.LongCompactionsTest.testCompactionSlim(LongCompactionsTest.java:58)
 [junit] 
 [junit] 
 [junit] Testcase: 
 testCompactionWide(org.apache.cassandra.db.compaction.LongCompactionsTest):   
 FAILED
 [junit] 
 /tmp/Keyspace15276133158440321595Standard1/Keyspace1/Keyspace1-Standard1-jb-0-Data.db
  is not correctly marked compacting
 [junit] junit.framework.AssertionFailedError: 
 /tmp/Keyspace15276133158440321595Standard1/Keyspace1/Keyspace1-Standard1-jb-0-Data.db
  is not correctly marked compacting
 [junit] at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.init(AbstractCompactionTask.java:49)
 [junit] at 
 org.apache.cassandra.db.compaction.CompactionTask.init(CompactionTask.java:47)
 [junit] at 
 org.apache.cassandra.db.compaction.LongCompactionsTest.testCompaction(LongCompactionsTest.java:102)
 [junit] at 
 org.apache.cassandra.db.compaction.LongCompactionsTest.testCompactionWide(LongCompactionsTest.java:49)
 [junit] 
 [junit] 
 [junit] Test org.apache.cassandra.db.compaction.LongCompactionsTest FAILED
 {noformat}
 A system.log is attached from the above run on 2.0 HEAD.



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


[jira] [Updated] (CASSANDRA-8692) Coalesce intra-cluster network messages

2015-01-27 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg updated CASSANDRA-8692:
--
Description: 
While researching CASSANDRA-8457 we found that it is effective and can be done 
without introducing additional latency at low concurrency/throughput.

The patch from that was used and found to be useful in a real life scenario so 
I propose we implement this in 2.1 in addition to 3.0.

The change set is a single file and is small enough to be reviewable.

  was:
While researching CASSANDRA-8457 we found that it is effective and can be done 
without introducing additional latency at low concurrency/throughput.

The patch from that was used and found to be useful in a real life scenario so 
I propose we implement this in 2.1 in addition to 3.0.

The change set is a single file and is small enough to be reviewable.i


 Coalesce intra-cluster network messages
 ---

 Key: CASSANDRA-8692
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8692
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Ariel Weisberg
Assignee: Ariel Weisberg

 While researching CASSANDRA-8457 we found that it is effective and can be 
 done without introducing additional latency at low concurrency/throughput.
 The patch from that was used and found to be useful in a real life scenario 
 so I propose we implement this in 2.1 in addition to 3.0.
 The change set is a single file and is small enough to be reviewable.



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


[jira] [Commented] (CASSANDRA-8422) cassandra won't start up due to Unable to gossip with any seeds on the decommissioned node

2015-01-27 Thread kris (JIRA)

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

kris commented on CASSANDRA-8422:
-

the cassandra version is 2.0.11.83

 cassandra won't start up due to Unable to gossip with any seeds on the 
 decommissioned node
 

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

 - 2-node
   * nodeA - seed
   * nodeB
 1. decommission nodeB from the cluster with nodetool
when it's finished, kill cassandra process on nodeB
 2. delete data from commit/cache/data directories on nodeB
 3. try to start cassandra on nodeB (first time)
= FAILED with Unable to gossip with any seeds
 4. try to start cassandra on nodeB (second time)
   = OK
 It was not a one-time shot. I tried it a several times and encountered the 
 same issue for some reason.
 {code}
 ERROR [main] 2014-11-27 18:44:55,017 CassandraDaemon.java (line 513) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
 at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1211)
 at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:445)
 at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:659)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:611)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:503)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
  INFO [StorageServiceShutdownHook] 2014-11-27 18:44:55,076 Gossiper.java 
 (line 1307) Announcing shutdown
 {code}



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


[jira] [Commented] (CASSANDRA-7350) Decommissioning nodes borks the seed node - can't add additional nodes

2015-01-27 Thread kris (JIRA)

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

kris commented on CASSANDRA-7350:
-

we are facing the same issue too.

 Decommissioning nodes borks the seed node - can't add additional nodes
 --

 Key: CASSANDRA-7350
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7350
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Ubuntu using the auto-clustering AMI
Reporter: Steven Lowenthal
Assignee: Shawn Kumar
Priority: Minor
  Labels: qa-resolved
 Fix For: 2.0.9


 1) Launch a 4 node cluster - I used the auto-clustering AMI (you get nodes 
 0-3)
 2) decommission that last 2 nodes (nodes , leaving a 2 node cluster)
 3) wipe the data directories from node 2
 4) bootstrap node2 - it won't join unable to gossip with any seeds.
 If you bootstrap the node a second time, it will join.  However if you try to 
 bootstrap node 3, it will also fail.
 I discovered that bouncing the seed node fixes the problem.  I think it 
 cropped up in 2.0.7.
 Error:
 ERROR [main] 2014-06-03 21:52:46,649 CassandraDaemon.java (line 497) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
   at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1193)
   at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:447)
   at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:656)
   at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:612)
   at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:505)
   at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:362)
   at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:480)
   at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:569)
 ERROR [StorageServiceShutdownHook] 2014-06-03 21:52:46,741 
 CassandraDaemon.java (line 198) Exception in thread Thread[StorageServi



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


[jira] [Commented] (CASSANDRA-8381) CFStats should record keys of largest N requests for time interval

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8381:
--

[~mstump] how close is #7974 to what you want?

 CFStats should record keys of largest N requests for time interval
 --

 Key: CASSANDRA-8381
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8381
 Project: Cassandra
  Issue Type: Improvement
Reporter: Matt Stump
Priority: Critical

 Isolating the problem partition for a CF is right now incredibly difficult. 
 If we could keep the primary key of the largest N read or write requests for 
 the pervious interval or since counter has been cleared it would be extremely 
 useful.



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


[jira] [Commented] (CASSANDRA-8337) mmap underflow during validation compaction

2015-01-27 Thread Alexander Sterligov (JIRA)

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

Alexander Sterligov commented on CASSANDRA-8337:


Unfortunately now my team have several high priority tasks. For sure we will 
work on this problem in Q1 (resources are reserved).





 mmap underflow during validation compaction
 ---

 Key: CASSANDRA-8337
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8337
 Project: Cassandra
  Issue Type: Bug
Reporter: Alexander Sterligov
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: 8337_v1.txt, thread_dump


 During full parallel repair I often get errors like the following
 {quote}
 [2014-11-19 01:02:39,355] Repair session 116beaf0-6f66-11e4-afbb-c1c082008cbe 
 for range (3074457345618263602,-9223372036854775808] failed with error 
 org.apache.cassandra.exceptions.RepairException: [repair 
 #116beaf0-6f66-11e4-afbb-c1c082008cbe on iss/target_state_history, 
 (3074457345618263602,-9223372036854775808]] Validation failed in 
 /95.108.242.19
 {quote}
 At the log of the node there are always same exceptions:
 {quote}
 ERROR [ValidationExecutor:2] 2014-11-19 01:02:10,847 
 JVMStabilityInspector.java:94 - JVM state determined to be unstable.  Exiting 
 forcefully due to:
 org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
 mmap segment underflow; remaining is 15 but 47 requested
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1518)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1385)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPositionsForRanges(SSTableReader.java:1315)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1706)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1694)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:276)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getScanners(WrappingCompactionStrategy.java:320)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:917)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:97)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:557)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
 ~[na:1.7.0_51]
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  ~[na:1.7.0_51]
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  [na:1.7.0_51]
 at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
 Caused by: java.io.IOException: mmap segment underflow; remaining is 15 but 
 47 requested
 at 
 org.apache.cassandra.io.util.MappedFileDataInput.readBytes(MappedFileDataInput.java:135)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:348) 
 ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:327)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 at 
 org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1460)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 ... 13 common frames omitted
 {quote}
 Now i'm using die disk_failure_policy to determine such conditions faster, 
 but I get them even with stop policy.
 Streams related to host with such exception are hanged. Thread dump is 
 attached. Only restart helps.
 After retry I get errors from other nodes.
 scrub doesn't help and report that sstables are ok.
 Sequential repairs doesn't cause such exceptions.
 Load is about 1000 write rps and 50 read rps per node.



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


[jira] [Commented] (CASSANDRA-8678) CREATE TABLE accepts value for default_time_to_live on counter table

2015-01-27 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-8678:
---

Patch addresses trivial CREATE TABLE syntax, does not prevent ALTER TABLE after 
the fact. Can add that if necessary.


 CREATE TABLE accepts value for default_time_to_live on counter table
 

 Key: CASSANDRA-8678
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8678
 Project: Cassandra
  Issue Type: Wish
  Components: Tools
Reporter: Aaron Ploetz
Priority: Minor
 Fix For: 2.1.2

 Attachments: cassandra-8678.diff


 I can create a counter table (via cqlsh) with a default_time_to_live:
 CREATE TABLE IF NOT EXISTS metrics2(
   time timestamp,
   value counter,
   PRIMARY KEY ((time))
 ) WITH default_time_to_live=10;
 Upsert a row that increments the counter:
 {{UPDATE metrics2 SET value=value+1 WHERE timestamp='2015-01-24 10:48 
 -0600';}}
 Wait 10 seconds, and select, and the row is (of course) still there.  There 
 should probably be a warning or error preventing the creation of a table that 
 has both counter columns and a value set for default_time_to_live.



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


[jira] [Commented] (CASSANDRA-7281) SELECT on tuple relations are broken for mixed ASC/DESC clustering order

2015-01-27 Thread Benjamin Lerer (JIRA)

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

Benjamin Lerer commented on CASSANDRA-7281:
---

[~marcinszymaniuk] do you want to provide another patch or should I reassign 
the ticket?

 SELECT on tuple relations are broken for mixed ASC/DESC clustering order
 

 Key: CASSANDRA-7281
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7281
 Project: Cassandra
  Issue Type: Bug
Reporter: Sylvain Lebresne
Assignee: Marcin Szymaniuk
 Fix For: 2.1.3, 2.0.13

 Attachments: 
 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-.patch, 
 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v2.patch, 
 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v3.patch


 As noted on 
 [CASSANDRA-6875|https://issues.apache.org/jira/browse/CASSANDRA-6875?focusedCommentId=13992153page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13992153],
  the tuple notation is broken when the clustering order mixes ASC and DESC 
 directives because the range of data they describe don't correspond to a 
 single continuous slice internally. To copy the example from CASSANDRA-6875:
 {noformat}
 cqlsh:ks create table foo (a int, b int, c int, PRIMARY KEY (a, b, c)) WITH 
 CLUSTERING ORDER BY (b DESC, c ASC);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 2, 0);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 1, 0);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 1, 1);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 0, 0);
 cqlsh:ks SELECT * FROM foo WHERE a=0;
  a | b | c
 ---+---+---
  0 | 2 | 0
  0 | 1 | 0
  0 | 1 | 1
  0 | 0 | 0
 (4 rows)
 cqlsh:ks SELECT * FROM foo WHERE a=0 AND (b, c)  (1, 0);
  a | b | c
 ---+---+---
  0 | 2 | 0
 (1 rows)
 {noformat}
 The last query should really return {{(0, 2, 0)}} and {{(0, 1, 1)}}.
 For that specific example we should generate 2 internal slices, but I believe 
 that with more clustering columns we may have more slices.



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


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

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-8061:
-

Ok, I've uploaded a new version of the patch 
[here|https://github.com/belliottsmith/cassandra/commits/7705-2.1-x+]

It should fix the bug with cleanup on error, and also ensure that any such 
problem doesn't shadow the original cause anyway. This is a temporary patch 
until I fix that aspect properly in mainline. It's quite likely all those 
exceptions you were seeing were the cause of the tmplink files, since they were 
attempts to clean them up. Look forward to seeing the results!

 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
Assignee: Joshua McKenzie
 Fix For: 2.1.3

 Attachments: 8061_v1.txt, 8248-thread_dump.txt


 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:
 {noformat}
 $ 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  
 

[jira] [Commented] (CASSANDRA-8381) CFStats should record keys of largest N requests for time interval

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8381:
--

(if at all close)

 CFStats should record keys of largest N requests for time interval
 --

 Key: CASSANDRA-8381
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8381
 Project: Cassandra
  Issue Type: Improvement
Reporter: Matt Stump
Priority: Critical

 Isolating the problem partition for a CF is right now incredibly difficult. 
 If we could keep the primary key of the largest N read or write requests for 
 the pervious interval or since counter has been cleared it would be extremely 
 useful.



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


[jira] [Commented] (CASSANDRA-7447) New sstable format with support for columnar layout

2015-01-27 Thread Evan Chan (JIRA)

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

Evan Chan commented on CASSANDRA-7447:
--

Hey guys,

I'm working on a project to implement a columnar database on top of Cassandra, 
so I found this really interesting.  I have a repo comparing regular CQL layout 
with a columnar layout, and I can confirm [~benedict]'s comments -- for certain 
queries am seeing over two orders of magnitude speedup:

https://github.com/velvia/cassandra-gdelt

It would be super useful to have an option for a columnar layout, or to have 
that portion of the storage format be pluggable somehow.

I'm splitting out my research into a zero-serialization, high performance 
vector binary format into a separate repo, drop me a note if you guys are 
interested.

 New sstable format with support for columnar layout
 ---

 Key: CASSANDRA-7447
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7447
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Benedict
Assignee: Benedict
  Labels: performance, storage
 Fix For: 3.1

 Attachments: ngcc-storage.odp, storage_format.pdf


 h2. Storage Format Proposal
 C* has come a long way over the past few years, and unfortunately our storage 
 format hasn't kept pace with the data models we are now encouraging people to 
 utilise. This ticket proposes a collections of storage primitives that can be 
 combined to serve these data models more optimally.
 It would probably help to first state the data model at the most abstract 
 level. We have a fixed three-tier structure: We have the partition key, the 
 clustering columns, and the data columns. Each have their own characteristics 
 and so require their own specialised treatment.
 I should note that these changes will necessarily be delivered in stages, and 
 that we will be making some assumptions about what the most useful features 
 to support initially will be. Any features not supported will require 
 sticking with the old format until we extend support to all C* functionality.
 h3. Partition Key
 * This really has two components: the partition, and the value. Although the 
 partition is primarily used to distribute across nodes, it can also be used 
 to optimise lookups for a given key within a node
 * Generally partitioning is by hash, and for the moment I want to focus this 
 ticket on the assumption that this is the case
 * Given this, it makes sense to optimise our storage format to permit O(1) 
 searching of a given partition. It may be possible to achieve this with 
 little overhead based on the fact we store the hashes in order and know they 
 are approximately randomly distributed, as this effectively forms an 
 immutable contiguous split-ordered list (see Shalev/Shavit, or 
 CASSANDRA-7282), so we only need to store an amount of data based on how 
 imperfectly distributed the hashes are, or at worst a single value per block.
 * This should completely obviate the need for a separate key-cache, which 
 will be relegated to supporting the old storage format only
 h3. Primary Key / Clustering Columns
 * Given we have a hierarchical data model, I propose the use of a 
 cache-oblivious trie
 * The main advantage of the trie is that it is extremely compact and 
 _supports optimally efficient merges with other tries_ so that we can support 
 more efficient reads when multiple sstables are touched
 * The trie will be preceded by a small amount of related data; the full 
 partition key, a timestamp epoch (for offset-encoding timestamps) and any 
 other partition level optimisation data, such as (potentially) a min/max 
 timestamp to abort merges earlier
 * Initially I propose to limit the trie to byte-order comparable data types 
 only (the number of which we can expand through translations of the important 
 types that are not currently)
 * Crucially the trie will also encapsulate any range tombstones, so that 
 these are merged early in the process and avoids re-iterating the same data
 * Results in true bidirectional streaming without having to read entire range 
 into memory
 h3. Values
 There are generally two approaches to storing rows of data: columnar, or 
 row-oriented. The above two data structures can be combined with a value 
 storage scheme that is based on either. However, given the current model we 
 have of reading large 64Kb blocks for any read, I am inclined to focus on 
 columnar support first, as this delivers order-of-magnitude benefits to those 
 users with the correct workload, while for most workloads our 64Kb blocks are 
 large enough to store row-oriented data in a column-oriented fashion without 
 any performance degradation (I'm happy to consign very large row support 

[jira] [Updated] (CASSANDRA-8693) QueryProcessor never removes internal statements from its cache

2015-01-27 Thread Edward Ribeiro (JIRA)

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

Edward Ribeiro updated CASSANDRA-8693:
--
Summary: QueryProcessor never removes internal statements from its cache  
(was: QueryProcessor.executeInternal(statement) never evictes statements from 
internal cache)

 QueryProcessor never removes internal statements from its cache
 ---

 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
  Labels: cql3
 Fix For: 2.1.2

 Attachments: qp-internal.patch


 QueryProcessor holds a reference to {{internalStatements}}, a map for 
 prepared statements used internally.
 Those commands don't mix with the ones created by the user, but the problem 
 is that if a KS/CF is dropped and then recreated {{internalStatements}} 
 entries will point to invalid PreparedStatements (old cf_id) so any operation 
 on those statements will fail therefore. In fact, as of today, this map is 
 never actually cleaned no matter what.
 This problem is similar to the ones addressed by 
 https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
 https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
 any further context. 
 I am attaching a patch to this issue. 



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


[jira] [Updated] (CASSANDRA-8693) QueryProcessor never removes internal statements from its cache

2015-01-27 Thread Edward Ribeiro (JIRA)

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

Edward Ribeiro updated CASSANDRA-8693:
--
Description: 
QueryProcessor holds a reference to {{internalStatements}}, a map for prepared 
statements used internally.

Those commands don't mix with the ones created by the user, but the problem is 
that if a KS/CF is dropped and then recreated {{internalStatements}} entries 
will point to invalid PreparedStatements (old cf_id) so any operation on those 
statements will fail thereafter. In fact, as of today, this map is never 
actually cleaned, no matter what.

This problem is similar to the ones addressed by 
https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
any further context. 

I am attaching a patch to this issue. 

  was:
QueryProcessor holds a reference to {{internalStatements}}, a map for prepared 
statements used internally.

Those commands don't mix with the ones created by the user, but the problem is 
that if a KS/CF is dropped and then recreated {{internalStatements}} entries 
will point to invalid PreparedStatements (old cf_id) so any operation on those 
statements will fail therefore. In fact, as of today, this map is never 
actually cleaned no matter what.

This problem is similar to the ones addressed by 
https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
any further context. 

I am attaching a patch to this issue. 


 QueryProcessor never removes internal statements from its cache
 ---

 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
Assignee: Edward Ribeiro
  Labels: cql3
 Fix For: 2.1.2

 Attachments: qp-internal.patch


 QueryProcessor holds a reference to {{internalStatements}}, a map for 
 prepared statements used internally.
 Those commands don't mix with the ones created by the user, but the problem 
 is that if a KS/CF is dropped and then recreated {{internalStatements}} 
 entries will point to invalid PreparedStatements (old cf_id) so any operation 
 on those statements will fail thereafter. In fact, as of today, this map is 
 never actually cleaned, no matter what.
 This problem is similar to the ones addressed by 
 https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
 https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
 any further context. 
 I am attaching a patch to this issue. 



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


[jira] [Assigned] (CASSANDRA-8693) QueryProcessor never removes internal statements from its cache

2015-01-27 Thread Edward Ribeiro (JIRA)

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

Edward Ribeiro reassigned CASSANDRA-8693:
-

Assignee: Edward Ribeiro

 QueryProcessor never removes internal statements from its cache
 ---

 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
Assignee: Edward Ribeiro
  Labels: cql3
 Fix For: 2.1.2

 Attachments: qp-internal.patch


 QueryProcessor holds a reference to {{internalStatements}}, a map for 
 prepared statements used internally.
 Those commands don't mix with the ones created by the user, but the problem 
 is that if a KS/CF is dropped and then recreated {{internalStatements}} 
 entries will point to invalid PreparedStatements (old cf_id) so any operation 
 on those statements will fail therefore. In fact, as of today, this map is 
 never actually cleaned no matter what.
 This problem is similar to the ones addressed by 
 https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
 https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
 any further context. 
 I am attaching a patch to this issue. 



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


[jira] [Updated] (CASSANDRA-8693) QueryProcessor.executeInternal(statement) never evictes statements from internal cache

2015-01-27 Thread Edward Ribeiro (JIRA)

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

Edward Ribeiro updated CASSANDRA-8693:
--
Summary: QueryProcessor.executeInternal(statement) never evictes statements 
from internal cache  (was: QueryProcessor.executeInternal(statement) are never 
evicted from internal cache)

 QueryProcessor.executeInternal(statement) never evictes statements from 
 internal cache
 --

 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
  Labels: cql3
 Fix For: 2.1.2

 Attachments: qp-internal.patch


 QueryProcessor holds a reference to {{internalStatements}}, a map for 
 prepared statements used internally.
 Those commands don't mix with the ones created by the user, but the problem 
 is that if a KS/CF is dropped and then recreated {{internalStatements}} 
 entries will point to invalid PreparedStatements (old cf_id) so any operation 
 on those statements will fail therefore. In fact, as of today, this map is 
 never actually cleaned no matter what.
 This problem is similar to the ones addressed by 
 https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
 https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
 any further context. 
 I am attaching a patch to this issue. 



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


[jira] [Updated] (CASSANDRA-8518) Impose In-Flight Data Limit

2015-01-27 Thread Benedict (JIRA)

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

Benedict updated CASSANDRA-8518:

Summary: Impose In-Flight Data Limit  (was: Cassandra Query Request Size 
Estimator)

 Impose In-Flight Data Limit
 ---

 Key: CASSANDRA-8518
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8518
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Cheng Ren

 We have been suffering from cassandra node crash due to out of memory for a 
 long time. The heap dump from the recent crash shows there are 22 native 
 transport request threads each of which consumes 3.3% of heap size, taking 
 more than 70% in total.  
 Heap dump:
 !https://dl-web.dropbox.com/get/attach1.png?_subject_uid=303980955w=AAAVOoncBoZ5aOPbDg2TpRkUss7B-2wlrnhUAv19b27OUA|height=400,width=600!
 Expanded view of one thread:
 !https://dl-web.dropbox.com/get/Screen%20Shot%202014-12-18%20at%204.06.29%20PM.png?_subject_uid=303980955w=AACUO4wrbxheRUxv8fwQ9P52T6gBOm5_g9zeIe8odu3V3w|height=400,width=600!
 The cassandra we are using now (2.0.4) utilized MemoryAwareThreadPoolExecutor 
 as the request executor and provided a default request size estimator which 
 constantly returns 1, meaning it limits only the number of requests being 
 pushed to the pool. To have more fine-grained control on handling requests 
 and better protect our node from OOM issue, we propose implementing a more 
 precise estimator. 
 Here is our two cents:
 For update/delete/insert request: Size could be estimated by adding size of 
 all class members together.
 For scan query, the major part of the request is response, which can be 
 estimated from the history data. For example if we receive a scan query on a 
 column family for a certain token range, we keep track of its response size 
 used as the estimated response size for later scan query on the same cf. 
 For future requests on the same cf, response size could be calculated by 
 token range*recorded size/ recorded token range. The request size should be 
 estimated as (query size + estimated response size).
 We believe what we're proposing here can be useful for other people in the 
 Cassandra community as well. Would you mind providing us feedbacks? Please 
 let us know if you have any concerns or suggestions regarding this proposal.
 Thanks,
 Cheng



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


[jira] [Updated] (CASSANDRA-8518) Impose In-Flight Data Limit

2015-01-27 Thread Benedict (JIRA)

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

Benedict updated CASSANDRA-8518:

Labels: performance  (was: )

 Impose In-Flight Data Limit
 ---

 Key: CASSANDRA-8518
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8518
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Cheng Ren
  Labels: performance

 We have been suffering from cassandra node crash due to out of memory for a 
 long time. The heap dump from the recent crash shows there are 22 native 
 transport request threads each of which consumes 3.3% of heap size, taking 
 more than 70% in total.  
 Heap dump:
 !https://dl-web.dropbox.com/get/attach1.png?_subject_uid=303980955w=AAAVOoncBoZ5aOPbDg2TpRkUss7B-2wlrnhUAv19b27OUA|height=400,width=600!
 Expanded view of one thread:
 !https://dl-web.dropbox.com/get/Screen%20Shot%202014-12-18%20at%204.06.29%20PM.png?_subject_uid=303980955w=AACUO4wrbxheRUxv8fwQ9P52T6gBOm5_g9zeIe8odu3V3w|height=400,width=600!
 The cassandra we are using now (2.0.4) utilized MemoryAwareThreadPoolExecutor 
 as the request executor and provided a default request size estimator which 
 constantly returns 1, meaning it limits only the number of requests being 
 pushed to the pool. To have more fine-grained control on handling requests 
 and better protect our node from OOM issue, we propose implementing a more 
 precise estimator. 
 Here is our two cents:
 For update/delete/insert request: Size could be estimated by adding size of 
 all class members together.
 For scan query, the major part of the request is response, which can be 
 estimated from the history data. For example if we receive a scan query on a 
 column family for a certain token range, we keep track of its response size 
 used as the estimated response size for later scan query on the same cf. 
 For future requests on the same cf, response size could be calculated by 
 token range*recorded size/ recorded token range. The request size should be 
 estimated as (query size + estimated response size).
 We believe what we're proposing here can be useful for other people in the 
 Cassandra community as well. Would you mind providing us feedbacks? Please 
 let us know if you have any concerns or suggestions regarding this proposal.
 Thanks,
 Cheng



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


[jira] [Created] (CASSANDRA-8694) Repair of empty keyspace hangs rather than ignoring the request

2015-01-27 Thread Ryan McGuire (JIRA)
Ryan McGuire created CASSANDRA-8694:
---

 Summary: Repair of empty keyspace hangs rather than ignoring the 
request
 Key: CASSANDRA-8694
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8694
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan McGuire
Priority: Minor


Create a two node cluster, create a keyspace, don't create any tables. Initiate 
a repair:

{code}
04:32 PM:~$ ccm create -v git:cassandra-2.1 test -n 2 -s
Fetching Cassandra updates...
Current cluster is now: test
04:33 PM:~$ ccm node1 cqlsh
Connected to test at 127.0.0.1:9042.
[cqlsh 5.0.1 | Cassandra 2.1.2-SNAPSHOT | CQL spec 3.2.0 | Native protocol v3]
Use HELP for help.
cqlsh create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 2};
cqlsh 
04:34 PM:~$ ccm node1 nodetool -- repair
[2015-01-27 16:34:11,741] Nothing to repair for keyspace 'system'
[2015-01-27 16:34:11,748] Starting repair command #1, repairing 2 ranges for 
keyspace test (parallelism=SEQUENTIAL, full=true)
{code}

The repair hangs.

Do the same thing but add a table, and the repair completes very quickly.



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


[jira] [Updated] (CASSANDRA-8678) CREATE TABLE accepts value for default_time_to_live on counter table

2015-01-27 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa updated CASSANDRA-8678:
--
Attachment: cassandra-8678.diff

Prevent CREATE TABLE with default_time_to_live if counter columns are present. 

 CREATE TABLE accepts value for default_time_to_live on counter table
 

 Key: CASSANDRA-8678
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8678
 Project: Cassandra
  Issue Type: Wish
  Components: Tools
Reporter: Aaron Ploetz
Priority: Minor
 Fix For: 2.1.2

 Attachments: cassandra-8678.diff


 I can create a counter table (via cqlsh) with a default_time_to_live:
 CREATE TABLE IF NOT EXISTS metrics2(
   time timestamp,
   value counter,
   PRIMARY KEY ((time))
 ) WITH default_time_to_live=10;
 Upsert a row that increments the counter:
 {{UPDATE metrics2 SET value=value+1 WHERE timestamp='2015-01-24 10:48 
 -0600';}}
 Wait 10 seconds, and select, and the row is (of course) still there.  There 
 should probably be a warning or error preventing the creation of a table that 
 has both counter columns and a value set for default_time_to_live.



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


[jira] [Created] (CASSANDRA-8693) QueryProcessor.executeInternal(statement) are never evicted from internal cache

2015-01-27 Thread Edward Ribeiro (JIRA)
Edward Ribeiro created CASSANDRA-8693:
-

 Summary: QueryProcessor.executeInternal(statement) are never 
evicted from internal cache
 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
 Fix For: 2.1.2
 Attachments: qp-internal.patch

QueryProcessor holds a reference to {{internalStatements}}, a map for prepared 
statements used internally.

Those commands don't mix with the ones created by the user, but the problem is 
that if a KS/CF is dropped and then recreated {{internalStatements}} entries 
will point to invalid PreparedStatements (old cf_id) so any operation on those 
statements will fail therefore. In fact, as of today, this map is never 
actually cleaned no matter what.

This problem is similar to the ones addressed by 
https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
any further context. 

I am attaching a patch to this issue. 



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


[jira] [Updated] (CASSANDRA-8694) Repair of empty keyspace hangs rather than ignoring the request

2015-01-27 Thread Ryan McGuire (JIRA)

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

Ryan McGuire updated CASSANDRA-8694:

Reproduced In: 2.1.2

 Repair of empty keyspace hangs rather than ignoring the request
 ---

 Key: CASSANDRA-8694
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8694
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan McGuire
Priority: Minor

 Create a two node cluster, create a keyspace, don't create any tables. 
 Initiate a repair:
 {code}
 04:32 PM:~$ ccm create -v git:cassandra-2.1 test -n 2 -s
 Fetching Cassandra updates...
 Current cluster is now: test
 04:33 PM:~$ ccm node1 cqlsh
 Connected to test at 127.0.0.1:9042.
 [cqlsh 5.0.1 | Cassandra 2.1.2-SNAPSHOT | CQL spec 3.2.0 | Native protocol v3]
 Use HELP for help.
 cqlsh create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
 'replication_factor': 2};
 cqlsh 
 04:34 PM:~$ ccm node1 nodetool -- repair
 [2015-01-27 16:34:11,741] Nothing to repair for keyspace 'system'
 [2015-01-27 16:34:11,748] Starting repair command #1, repairing 2 ranges for 
 keyspace test (parallelism=SEQUENTIAL, full=true)
 {code}
 The repair hangs.
 Do the same thing but add a table, and the repair completes very quickly.



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


[jira] [Comment Edited] (CASSANDRA-7705) Safer Resource Management

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko edited comment on CASSANDRA-7705 at 1/27/15 11:27 PM:


Usually you simply wrap it in Collections#newSetFromMap() to get the equivalent 
of the non-existent CHS.


was (Author: iamaleksey):
Usually you simply wrap it in Collections#newSetFromMap() to get the equivalent 
of the non-existent CHM.

 Safer Resource Management
 -

 Key: CASSANDRA-7705
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7705
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Benedict
Assignee: Benedict
 Fix For: 3.0


 We've had a spate of bugs recently with bad reference counting. these can 
 have potentially dire consequences, generally either randomly deleting data 
 or giving us infinite loops. 
 Since in 2.1 we only reference count resources that are relatively expensive 
 and infrequently managed (or in places where this safety is probably not as 
 necessary, e.g. SerializingCache), we could without any negative consequences 
 (and only slight code complexity) introduce a safer resource management 
 scheme for these more expensive/infrequent actions.
 Basically, I propose when we want to acquire a resource we allocate an object 
 that manages the reference. This can only be released once; if it is released 
 twice, we fail immediately at the second release, reporting where the bug is 
 (rather than letting it continue fine until the next correct release corrupts 
 the count). The reference counter remains the same, but we obtain guarantees 
 that the reference count itself is never badly maintained, although code 
 using it could mistakenly release its own handle early (typically this is 
 only an issue when cleaning up after a failure, in which case under the new 
 scheme this would be an innocuous error)



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


[jira] [Commented] (CASSANDRA-7688) Add data sizing to a system table

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-7688:
--

bq. I would have thought it’d be easier to just expose the Storage proxy call 
via cql?

It wouldn't, unless you propose to create an extra CQL statement just for this, 
which is something that's not gonna happen.

Otherwise you'd need support for virtual tables, and that's 3.1 territory at 
best.

 Add data sizing to a system table
 -

 Key: CASSANDRA-7688
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7688
 Project: Cassandra
  Issue Type: New Feature
Reporter: Jeremiah Jordan
Assignee: Aleksey Yeschenko
 Fix For: 2.1.3


 Currently you can't implement something similar to describe_splits_ex purely 
 from the a native protocol driver.  
 https://datastax-oss.atlassian.net/browse/JAVA-312 is open to expose easily 
 getting ownership information to a client in the java-driver.  But you still 
 need the data sizing part to get splits of a given size.  We should add the 
 sizing information to a system table so that native clients can get to it.



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


[jira] [Created] (CASSANDRA-8695) thrift column definition list sometimes immutable

2015-01-27 Thread Chris Lockfort (JIRA)
Chris Lockfort created CASSANDRA-8695:
-

 Summary: thrift column definition list sometimes immutable
 Key: CASSANDRA-8695
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8695
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Chris Lockfort


When the input cf_def.column_metadata is null, ColumnDefinition.fromThrift can 
return Collections.emptyList() (which is defined to be an immutable list) 

One caller, CFMetaData.internalFromThrift , incorrectly assumes that this list 
will always be mutable and tries to add to this immutable list, which causes 
the Cassandra server to throw an exception.

It looks like this is a bug in 2.1, but not in trunk.
I can include my one-liner patch, but I'm not sure whether it is better to fix 
the caller (to not assume random lists you get back from a method are mutable) 
or the callee (to always return a mutable list).



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


cassandra git commit: remove dead code

2015-01-27 Thread dbrosius
Repository: cassandra
Updated Branches:
  refs/heads/trunk 64e39a039 - 02367e78b


remove dead code


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

Branch: refs/heads/trunk
Commit: 02367e78b573925a864ab8f94ba0a69950ca74f5
Parents: 64e39a0
Author: Dave Brosius dbros...@mebigfatguy.com
Authored: Tue Jan 27 20:35:18 2015 -0500
Committer: Dave Brosius dbros...@mebigfatguy.com
Committed: Tue Jan 27 20:36:45 2015 -0500

--
 test/unit/org/apache/cassandra/Util.java | 37 ---
 1 file changed, 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/02367e78/test/unit/org/apache/cassandra/Util.java
--
diff --git a/test/unit/org/apache/cassandra/Util.java 
b/test/unit/org/apache/cassandra/Util.java
index cdb7ac6..24e334a 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -211,27 +211,6 @@ public class Util
 return cfStore.getColumnFamily(QueryFilter.getIdentityFilter(key, 
cfName, System.currentTimeMillis()));
 }
 
-public static byte[] concatByteArrays(byte[] first, byte[]... remaining)
-{
-int length = first.length;
-for (byte[] array : remaining)
-{
-length += array.length;
-}
-
-byte[] result = new byte[length];
-System.arraycopy(first, 0, result, 0, first.length);
-int offset = first.length;
-
-for (byte[] array : remaining)
-{
-System.arraycopy(array, 0, result, offset, array.length);
-offset += array.length;
-}
-
-return result;
-}
-
 public static boolean equalsCounterId(CounterId n, ByteBuffer context, int 
offset)
 {
 return CounterId.wrap(context, context.position() + offset).equals(n);
@@ -313,22 +292,6 @@ public class Util
 assert thrown : exception.getName() +  not received;
 }
 
-public static ByteBuffer serializeForSSTable(ColumnFamily cf)
-{
-try
-{
-DataOutputBuffer out = new DataOutputBuffer();
-
DeletionTime.serializer.serialize(cf.deletionInfo().getTopLevelDeletion(), out);
-out.writeInt(cf.getColumnCount());
-new ColumnIndex.Builder(cf, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
out).build(cf);
-return ByteBuffer.wrap(out.toByteArray());
-}
-catch (IOException e)
-{
-throw new RuntimeException(e);
-}
-}
-
 public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, 
DecoratedKey key)
 {
 SortedSetCellName s = new TreeSetCellName(cfs.getComparator());



[jira] [Commented] (CASSANDRA-7688) Add data sizing to a system table

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-7688:
--

The primary goal is to remove the dependency on Thrift, so that the spark 
connector and our hadoop code don't have to open an extra Thrift connection in 
addition to the native protocol one. See CASSANDRA-8358 for example.

Having that we'd be able to not start Thrift by default, and simpler 
implementations of spark and hadoop things.

 Add data sizing to a system table
 -

 Key: CASSANDRA-7688
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7688
 Project: Cassandra
  Issue Type: New Feature
Reporter: Jeremiah Jordan
Assignee: Aleksey Yeschenko
 Fix For: 2.1.3


 Currently you can't implement something similar to describe_splits_ex purely 
 from the a native protocol driver.  
 https://datastax-oss.atlassian.net/browse/JAVA-312 is open to expose easily 
 getting ownership information to a client in the java-driver.  But you still 
 need the data sizing part to get splits of a given size.  We should add the 
 sizing information to a system table so that native clients can get to it.



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


[jira] [Updated] (CASSANDRA-8693) QueryProcessor never removes internal statements from its cache

2015-01-27 Thread Brandon Williams (JIRA)

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

Brandon Williams updated CASSANDRA-8693:

Reviewer: Tyler Hobbs

 QueryProcessor never removes internal statements from its cache
 ---

 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
Assignee: Edward Ribeiro
  Labels: cql3
 Fix For: 2.1.2

 Attachments: qp-internal.patch


 QueryProcessor holds a reference to {{internalStatements}}, a map for 
 prepared statements used internally.
 Those commands don't mix with the ones created by the user, but the problem 
 is that if a KS/CF is dropped and then recreated {{internalStatements}} 
 entries will point to invalid PreparedStatements (old cf_id) so any operation 
 on those statements will fail thereafter. In fact, as of today, this map is 
 never actually cleaned, no matter what.
 This problem is similar to the ones addressed by 
 https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
 https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
 any further context. 
 I am attaching a patch to this issue. 



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


[jira] [Commented] (CASSANDRA-8650) Creation and maintenance of roles should not require superuser status

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8650:
--

I'll have full notes posted later. One question - if we do go this route, 
shouldn't we just create another IResource (purely for roles) vs. abusing the 
high-level DataResource?

Then we can use the same permission names, without the ugly underscores, and 
just have them belong to a totally different resource hierarchy.

 Creation and maintenance of roles should not require superuser status
 -

 Key: CASSANDRA-8650
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8650
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Sam Tunnicliffe
Assignee: Sam Tunnicliffe
  Labels: cql, security
 Fix For: 3.0

 Attachments: 8650.txt


 Currently, only roles with superuser status are permitted to 
 create/drop/grant/revoke roles, which violates the principal of least 
 privilege. In addition, in order to run {{ALTER ROLE}} statements a user must 
 log in directly as that role or else be a superuser. This requirement 
 increases the (ab)use of superuser privileges, especially where roles are 
 created without {{LOGIN}} privileges to model groups of permissions granted 
 to individual db users. In this scenario, a superuser is always required if 
 such roles are to be granted and modified.
 We should add more granular permissions to allow administration of roles 
 without requiring superuser status.



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


[jira] [Commented] (CASSANDRA-7705) Safer Resource Management

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-7705:
-

I've uploaded a further polished version 
[here|https://github.com/belliottsmith/cassandra/commits/7705-2.1-x], hopefully 
addressing all of your nits and concerns, and finishing up your ref - tryRef 
refactor.

Responding to a couple of the more specific points that I haven't changed:

bq. In SSTableLoader the Ref.sharedRef() is passed to the constructor in 
SSTableStreamingSections which feels wrong, shouldn't we acquire a new Ref and 
have SSTSS release that once it is done with the sstable?

I didn't want to dive too closely into each prior design decision in this 
patch, I just wanted to replicate the existing behaviour. This is an offline 
operation, so I'm not sure it matters or not, but this is the prior behaviour.

bq. Manager.extant - why a Map here? Could we use a Set?

There is no ConcurrentHashSet in the JDK, and I don't want to use some random 
one. NBHS can leave dangling references to objects lying around after removal 
IIRC, and we don't need the performance here, so CHM it was.


 Safer Resource Management
 -

 Key: CASSANDRA-7705
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7705
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Benedict
Assignee: Benedict
 Fix For: 3.0


 We've had a spate of bugs recently with bad reference counting. these can 
 have potentially dire consequences, generally either randomly deleting data 
 or giving us infinite loops. 
 Since in 2.1 we only reference count resources that are relatively expensive 
 and infrequently managed (or in places where this safety is probably not as 
 necessary, e.g. SerializingCache), we could without any negative consequences 
 (and only slight code complexity) introduce a safer resource management 
 scheme for these more expensive/infrequent actions.
 Basically, I propose when we want to acquire a resource we allocate an object 
 that manages the reference. This can only be released once; if it is released 
 twice, we fail immediately at the second release, reporting where the bug is 
 (rather than letting it continue fine until the next correct release corrupts 
 the count). The reference counter remains the same, but we obtain guarantees 
 that the reference count itself is never badly maintained, although code 
 using it could mistakenly release its own handle early (typically this is 
 only an issue when cleaning up after a failure, in which case under the new 
 scheme this would be an innocuous error)



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


[jira] [Updated] (CASSANDRA-8650) Creation and maintenance of roles should not require superuser status

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko updated CASSANDRA-8650:
-
Issue Type: Sub-task  (was: Improvement)
Parent: CASSANDRA-8394

 Creation and maintenance of roles should not require superuser status
 -

 Key: CASSANDRA-8650
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8650
 Project: Cassandra
  Issue Type: Sub-task
  Components: Core
Reporter: Sam Tunnicliffe
Assignee: Sam Tunnicliffe
  Labels: cql, security
 Fix For: 3.0

 Attachments: 8650.txt


 Currently, only roles with superuser status are permitted to 
 create/drop/grant/revoke roles, which violates the principal of least 
 privilege. In addition, in order to run {{ALTER ROLE}} statements a user must 
 log in directly as that role or else be a superuser. This requirement 
 increases the (ab)use of superuser privileges, especially where roles are 
 created without {{LOGIN}} privileges to model groups of permissions granted 
 to individual db users. In this scenario, a superuser is always required if 
 such roles are to be granted and modified.
 We should add more granular permissions to allow administration of roles 
 without requiring superuser status.



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


[jira] [Resolved] (CASSANDRA-7216) Restricted superuser account request

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko resolved CASSANDRA-7216.
--
Resolution: Duplicate

 Restricted superuser account request
 

 Key: CASSANDRA-7216
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7216
 Project: Cassandra
  Issue Type: Improvement
Reporter: Oded Peer
Assignee: Dave Brosius
Priority: Minor
 Fix For: 3.0

 Attachments: 7216-POC.txt, 7216.txt


 I am developing a multi-tenant service.
 Every tenant has its own user, keyspace and can access only his keyspace.
 As new tenants are provisioned there is a need to create new users and 
 keyspaces.
 Only a superuser can issue CREATE USER requests, so we must have a super user 
 account in the system. On the other hand super users have access to all the 
 keyspaces, which poses a security risk.
 For tenant provisioning I would like to have a restricted account which can 
 only create new users, without read access to keyspaces.



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


[jira] [Assigned] (CASSANDRA-8422) cassandra won't start up due to Unable to gossip with any seeds on the decommissioned node

2015-01-27 Thread Russ Hatch (JIRA)

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

Russ Hatch reassigned CASSANDRA-8422:
-

Assignee: Russ Hatch

 cassandra won't start up due to Unable to gossip with any seeds on the 
 decommissioned node
 

 Key: CASSANDRA-8422
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8422
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Masashi Ozawa
Assignee: Russ Hatch

 - 2-node
   * nodeA - seed
   * nodeB
 1. decommission nodeB from the cluster with nodetool
when it's finished, kill cassandra process on nodeB
 2. delete data from commit/cache/data directories on nodeB
 3. try to start cassandra on nodeB (first time)
= FAILED with Unable to gossip with any seeds
 4. try to start cassandra on nodeB (second time)
   = OK
 It was not a one-time shot. I tried it a several times and encountered the 
 same issue for some reason.
 {code}
 ERROR [main] 2014-11-27 18:44:55,017 CassandraDaemon.java (line 513) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
 at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1211)
 at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:445)
 at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:659)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:611)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:503)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
  INFO [StorageServiceShutdownHook] 2014-11-27 18:44:55,076 Gossiper.java 
 (line 1307) Announcing shutdown
 {code}



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


[jira] [Updated] (CASSANDRA-8693) QueryProcessor never removes internal statements from its cache

2015-01-27 Thread Brandon Williams (JIRA)

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

Brandon Williams updated CASSANDRA-8693:

Fix Version/s: (was: 2.1.2)
   2.1.3

 QueryProcessor never removes internal statements from its cache
 ---

 Key: CASSANDRA-8693
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8693
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Edward Ribeiro
Assignee: Edward Ribeiro
  Labels: cql3
 Fix For: 2.1.3

 Attachments: qp-internal.patch


 QueryProcessor holds a reference to {{internalStatements}}, a map for 
 prepared statements used internally.
 Those commands don't mix with the ones created by the user, but the problem 
 is that if a KS/CF is dropped and then recreated {{internalStatements}} 
 entries will point to invalid PreparedStatements (old cf_id) so any operation 
 on those statements will fail thereafter. In fact, as of today, this map is 
 never actually cleaned, no matter what.
 This problem is similar to the ones addressed by 
 https://issues.apache.org/jira/browse/CASSANDRA-8652 and 
 https://issues.apache.org/jira/browse/CASSANDRA-7566, so those issues provide 
 any further context. 
 I am attaching a patch to this issue. 



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


[jira] [Commented] (CASSANDRA-7281) SELECT on tuple relations are broken for mixed ASC/DESC clustering order

2015-01-27 Thread Marcin Szymaniuk (JIRA)

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

Marcin Szymaniuk commented on CASSANDRA-7281:
-

Ah yes I will provide another one. I missed your message.

 SELECT on tuple relations are broken for mixed ASC/DESC clustering order
 

 Key: CASSANDRA-7281
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7281
 Project: Cassandra
  Issue Type: Bug
Reporter: Sylvain Lebresne
Assignee: Marcin Szymaniuk
 Fix For: 2.1.3, 2.0.13

 Attachments: 
 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-.patch, 
 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v2.patch, 
 0001-CASSANDRA-7281-SELECT-on-tuple-relations-are-broken-v3.patch


 As noted on 
 [CASSANDRA-6875|https://issues.apache.org/jira/browse/CASSANDRA-6875?focusedCommentId=13992153page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13992153],
  the tuple notation is broken when the clustering order mixes ASC and DESC 
 directives because the range of data they describe don't correspond to a 
 single continuous slice internally. To copy the example from CASSANDRA-6875:
 {noformat}
 cqlsh:ks create table foo (a int, b int, c int, PRIMARY KEY (a, b, c)) WITH 
 CLUSTERING ORDER BY (b DESC, c ASC);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 2, 0);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 1, 0);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 1, 1);
 cqlsh:ks INSERT INTO foo (a, b, c) VALUES (0, 0, 0);
 cqlsh:ks SELECT * FROM foo WHERE a=0;
  a | b | c
 ---+---+---
  0 | 2 | 0
  0 | 1 | 0
  0 | 1 | 1
  0 | 0 | 0
 (4 rows)
 cqlsh:ks SELECT * FROM foo WHERE a=0 AND (b, c)  (1, 0);
  a | b | c
 ---+---+---
  0 | 2 | 0
 (1 rows)
 {noformat}
 The last query should really return {{(0, 2, 0)}} and {{(0, 1, 1)}}.
 For that specific example we should generate 2 internal slices, but I believe 
 that with more clustering columns we may have more slices.



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


[jira] [Commented] (CASSANDRA-8072) Exception during startup: Unable to gossip with any seeds

2015-01-27 Thread Russ Hatch (JIRA)

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

Russ Hatch commented on CASSANDRA-8072:
---

I have been able to reproduce this issue using the steps provided in 
CASSANDRA-8422.

During one repro attempt I tried to run 'nodetool gossipinfo' in a loop on the 
non-seed node (just before the reported exception was expected to occur), and 
was surprised to see it complain about attempting to use a closed connection.

Using netstat I had a look at the seed and the non-seed node, and can see a 
lingering CLOSE_WAIT connection on the seed node -- I'm wondering if cassandra 
could somehow be trying to reuse this stale connection, making the seed unable 
to connect back to the non-seed (and making it think the seed is unavailable).

It may also be relevant that the non-seed node has a connection in state 
FIN_WAIT2 for approx. 10-15 seconds after stopping the cassandra process.

 Exception during startup: Unable to gossip with any seeds
 -

 Key: CASSANDRA-8072
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8072
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan Springer
Assignee: Brandon Williams
 Attachments: casandra-system-log-with-assert-patch.log


 When Opscenter 4.1.4 or 5.0.1 tries to provision a 2-node DSC 2.0.10 cluster 
 in either ec2 or locally, an error occurs sometimes with one of the nodes 
 refusing to start C*.  The error in the /var/log/cassandra/system.log is:
 ERROR [main] 2014-10-06 15:54:52,292 CassandraDaemon.java (line 513) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
 at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1200)
 at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:444)
 at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:655)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:609)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:502)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
  INFO [StorageServiceShutdownHook] 2014-10-06 15:54:52,326 Gossiper.java 
 (line 1279) Announcing shutdown
  INFO [StorageServiceShutdownHook] 2014-10-06 15:54:54,326 
 MessagingService.java (line 701) Waiting for messaging service to quiesce
  INFO [ACCEPT-localhost/127.0.0.1] 2014-10-06 15:54:54,327 
 MessagingService.java (line 941) MessagingService has terminated the accept() 
 thread
 This errors does not always occur when provisioning a 2-node cluster, but 
 probably around half of the time on only one of the nodes.  I haven't been 
 able to reproduce this error with DSC 2.0.9, and there have been no code or 
 definition file changes in Opscenter.
 I can reproduce locally with the above steps.  I'm happy to test any proposed 
 fixes since I'm the only person able to reproduce reliably so far.



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


[jira] [Resolved] (CASSANDRA-8422) cassandra won't start up due to Unable to gossip with any seeds on the decommissioned node

2015-01-27 Thread Russ Hatch (JIRA)

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

Russ Hatch resolved CASSANDRA-8422.
---
Resolution: Duplicate

Thanks for the great repro steps, they have been very helpful!

I'm resolving this as a duplicated, but will continue working the issue on 
CASSANDRA-8072.

 cassandra won't start up due to Unable to gossip with any seeds on the 
 decommissioned node
 

 Key: CASSANDRA-8422
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8422
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Masashi Ozawa
Assignee: Russ Hatch

 - 2-node
   * nodeA - seed
   * nodeB
 1. decommission nodeB from the cluster with nodetool
when it's finished, kill cassandra process on nodeB
 2. delete data from commit/cache/data directories on nodeB
 3. try to start cassandra on nodeB (first time)
= FAILED with Unable to gossip with any seeds
 4. try to start cassandra on nodeB (second time)
   = OK
 It was not a one-time shot. I tried it a several times and encountered the 
 same issue for some reason.
 {code}
 ERROR [main] 2014-11-27 18:44:55,017 CassandraDaemon.java (line 513) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
 at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1211)
 at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:445)
 at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:659)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:611)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:503)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
  INFO [StorageServiceShutdownHook] 2014-11-27 18:44:55,076 Gossiper.java 
 (line 1307) Announcing shutdown
 {code}



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


[jira] [Updated] (CASSANDRA-8072) Exception during startup: Unable to gossip with any seeds

2015-01-27 Thread Russ Hatch (JIRA)

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

Russ Hatch updated CASSANDRA-8072:
--
Reproduced In: 2.0.11, 2.0.10  (was: 2.0.10)

 Exception during startup: Unable to gossip with any seeds
 -

 Key: CASSANDRA-8072
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8072
 Project: Cassandra
  Issue Type: Bug
Reporter: Ryan Springer
Assignee: Brandon Williams
 Attachments: casandra-system-log-with-assert-patch.log


 When Opscenter 4.1.4 or 5.0.1 tries to provision a 2-node DSC 2.0.10 cluster 
 in either ec2 or locally, an error occurs sometimes with one of the nodes 
 refusing to start C*.  The error in the /var/log/cassandra/system.log is:
 ERROR [main] 2014-10-06 15:54:52,292 CassandraDaemon.java (line 513) 
 Exception encountered during startup
 java.lang.RuntimeException: Unable to gossip with any seeds
 at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1200)
 at 
 org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:444)
 at 
 org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:655)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:609)
 at 
 org.apache.cassandra.service.StorageService.initServer(StorageService.java:502)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:378)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
  INFO [StorageServiceShutdownHook] 2014-10-06 15:54:52,326 Gossiper.java 
 (line 1279) Announcing shutdown
  INFO [StorageServiceShutdownHook] 2014-10-06 15:54:54,326 
 MessagingService.java (line 701) Waiting for messaging service to quiesce
  INFO [ACCEPT-localhost/127.0.0.1] 2014-10-06 15:54:54,327 
 MessagingService.java (line 941) MessagingService has terminated the accept() 
 thread
 This errors does not always occur when provisioning a 2-node cluster, but 
 probably around half of the time on only one of the nodes.  I haven't been 
 able to reproduce this error with DSC 2.0.9, and there have been no code or 
 definition file changes in Opscenter.
 I can reproduce locally with the above steps.  I'm happy to test any proposed 
 fixes since I'm the only person able to reproduce reliably so far.



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


[jira] [Commented] (CASSANDRA-7705) Safer Resource Management

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-7705:
--

Usually you simply wrap it in Collections#newSetFromMap() to get the equivalent 
of the non-existent CHM.

 Safer Resource Management
 -

 Key: CASSANDRA-7705
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7705
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Benedict
Assignee: Benedict
 Fix For: 3.0


 We've had a spate of bugs recently with bad reference counting. these can 
 have potentially dire consequences, generally either randomly deleting data 
 or giving us infinite loops. 
 Since in 2.1 we only reference count resources that are relatively expensive 
 and infrequently managed (or in places where this safety is probably not as 
 necessary, e.g. SerializingCache), we could without any negative consequences 
 (and only slight code complexity) introduce a safer resource management 
 scheme for these more expensive/infrequent actions.
 Basically, I propose when we want to acquire a resource we allocate an object 
 that manages the reference. This can only be released once; if it is released 
 twice, we fail immediately at the second release, reporting where the bug is 
 (rather than letting it continue fine until the next correct release corrupts 
 the count). The reference counter remains the same, but we obtain guarantees 
 that the reference count itself is never badly maintained, although code 
 using it could mistakenly release its own handle early (typically this is 
 only an issue when cleaning up after a failure, in which case under the new 
 scheme this would be an innocuous error)



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


[jira] [Commented] (CASSANDRA-8502) Static columns returning null for pages after first

2015-01-27 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-8502:
-

Remarks on the patch:
* I don't think this handle range queries (for the reversed case) as 
SliceFromReadCommand is not used for those (the comment in DataRange that the 
reverse case is handled by SliceFromReadCommand is thus incorrect). I believe 
this case should be dealt with in SStableScanner, though of course this will 
add to the uglyness. Not saying we shouldn't do it but I'll admit that at least 
as far as 2.0 goes, this is crossing the boundaries of my own confort zone.
* Not sure about the modifications in {{SliceQueryFilter.trim}} and 
{{AbstractQueryPager.discardHead}}. For non-reversed queries, this logic is 
already handled by the column counter.  It's true that it doesn't work for 
reversed queries, but the right place to fix that is imo the column counter 
(which probably need to have a reversed variant) as this also affects 
{{collectReducedColumns}} (and possibly other places).
* I think {{AbstractQueryPager.firstNonStaticColumns}} deserves a comment as to 
why we need to skip static columns. And in fact, I believe we only need to 
because the detection of static slices tests for {{EMPTY_BYTE_BUFFER}} which is 
fragile, so I'd personally prefer making the detection more reliable (by 
comparing if a slice start before/stop after the end of the static block).
* Not convinced {{lastKeyFilterWasUpdatedFor}} is actually an optimization (at 
least not in all cases): the code may at best call {{sliceForKey}} with the 
same key a handful of times (2, 3?), but if {{lastKeyFilterWasUpdatedFor}} is 
set, every other key (which is generally a lot) will trigger a few more useless 
comparisons. So as it adds complexity, let's leave that kind of unproven 
optimization out of this patch.
* Why do we need {{setStaticColumns}} in {{CFMetaData}}? It's only used by the 
patch with an empty hashset which feels weird since that should be the default. 
If it's just to make sure the staticColumns field in {{CFMetaData}} have been 
initialized in the test it's used for, then let's maybe call 
{{CFMetaData.rebuild}} instead.
* Nit: there is a typo ({{s/once/one}}) in the javadoc of 
{{splitOutStaticSlice}}.


 Static columns returning null for pages after first
 ---

 Key: CASSANDRA-8502
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8502
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Flavien Charlon
Assignee: Tyler Hobbs
 Fix For: 2.1.3, 2.0.13

 Attachments: 8502-2.0.txt, null-static-column.txt


 When paging is used for a query containing a static column, the first page 
 contains the right value for the static column, but subsequent pages have 
 null null for the static column instead of the expected value.
 Repro steps:
 - Create a table with a static column
 - Create a partition with 500 cells
 - Using cqlsh, query that partition
 Actual result:
 - You will see that first, the static column appears as expected, but if you 
 press a key after ---MORE---, the static columns will appear as null.
 See the attached file for a repro of the output.
 I am using a single node cluster.



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


[jira] [Commented] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-8683:
-

I think this might still not be quite right, but we're on the right track. This 
also highlights another bug (CASSANDRA-8691).

I suspect it may be the mistake that we are passing in 
.getToken().maxKeyBound() to getPosition - it could be we are looking up a 
value beyond that present in the index, because there are multiple adjacent 
keys with the same token, and we are looking past all of them.

That said, if we fix CASSANDRA-8691 we will need to do something about the last 
index position. There are two possibilities: 1) Like suggested a while back, we 
could simply ignore the last record for purposes of getPositionsForRanges(), 
since our paired source file will contain the row. Or 2) during index 
construction we could perhaps retain a lookup of the records immediately 
following an index boundary, for the past few boundaries only. We could then 
use this as our last key instead. I'm not sure which I prefer, as 1) creates 
some risk it will not be accounted for in future; 2) creates some unnecessary 
complexity.

 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the repairing files for non-incremental repairs, but the bug should exist in 
 2.1 as well)



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


[jira] [Updated] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson updated CASSANDRA-8683:
---
Attachment: 0001-avoid-NPE-in-getPositionsForRanges.patch

I was wrong, the files do still exist

problem is in getPositionsForRanges - if last token is the actual last token in 
the file (with early opening, it might not be), getPositions returns null

attaching patch to fix

 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the repairing files for non-incremental repairs, but the bug should exist in 
 2.1 as well)



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


[jira] [Commented] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8683:


[~aboudreault] it could explain the exceptions you saw, could you rerun with 
the patch?

 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the repairing files for non-incremental repairs, but the bug should exist in 
 2.1 as well)



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


[jira] [Created] (CASSANDRA-8691) SSTableReader.getPosition() does not correctly filter out queries that exceed its bounds

2015-01-27 Thread Benedict (JIRA)
Benedict created CASSANDRA-8691:
---

 Summary: SSTableReader.getPosition() does not correctly filter out 
queries that exceed its bounds
 Key: CASSANDRA-8691
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8691
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Benedict
 Fix For: 2.1.3


This has been true for some time, but I worry about including it before 2.1 in 
case we depend on the brokenness somehow. In getPosition, we test:

{code}
if (first.compareTo(key)  0 || last.compareTo(key)  0)
{
if (op == Operator.EQ  updateCacheAndStats)
bloomFilterTracker.addFalsePositive();

if (op.apply(1)  0)
{
Tracing.trace(Check against min and max keys allows skipping 
sstable {}, descriptor.generation);
return null;
}
}
{code}

However, op.apply(1) always returns = 0 unless op == Operation.EQ



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


[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log

2015-01-27 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-6809:


Rebased and applied review comments. New version uploaded 
[here|https://github.com/blambov/cassandra/compare/6809-compressed-logs].

 Compressed Commit Log
 -

 Key: CASSANDRA-6809
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809
 Project: Cassandra
  Issue Type: Improvement
Reporter: Benedict
Assignee: Branimir Lambov
Priority: Minor
  Labels: performance
 Fix For: 3.0

 Attachments: ComitLogStress.java, logtest.txt


 It seems an unnecessary oversight that we don't compress the commit log. 
 Doing so should improve throughput, but some care will need to be taken to 
 ensure we use as much of a segment as possible. I propose decoupling the 
 writing of the records from the segments. Basically write into a (queue of) 
 DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X 
 MB written to the CL (where X is ordinarily CLS size), and then pack as many 
 of the compressed chunks into a CLS as possible.



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


[jira] [Created] (CASSANDRA-8696) nodetool repair on cassandra 2.1.2 keyspaces returnjava.lang.RuntimeException: Could not create snapshot

2015-01-27 Thread Jeff Liu (JIRA)
Jeff Liu created CASSANDRA-8696:
---

 Summary: nodetool repair on cassandra 2.1.2 keyspaces 
returnjava.lang.RuntimeException: Could not create snapshot
 Key: CASSANDRA-8696
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8696
 Project: Cassandra
  Issue Type: Bug
Reporter: Jeff Liu


When trying to run nodetool repair -pr on cassandra node ( 2.1.2), cassandra 
throw java exceptions: cannot create snapshot. 

the error log from system.log:

{noformat}
INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:28,815 
StreamResultFuture.java:166 - [Stream #692c1450-a692-11e4-9973-070e938df227 
ID#0] Prepare completed. Receiving 2 files(221187 bytes), sending 5 
files(632105 bytes)
INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:29,046 
StreamResultFuture.java:180 - [Stream #692c1450-a692-11e4-9973-070e938df227] 
Session with /10.97.9.110 is complete
INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:29,046 
StreamResultFuture.java:212 - [Stream #692c1450-a692-11e4-9973-070e938df227] 
All sessions completed
INFO  [STREAM-IN-/10.97.9.110] 2015-01-28 02:07:29,047 
StreamingRepairTask.java:96 - [repair #685e3d00-a692-11e4-9973-070e938df227] 
streaming task succeed, returning response to /10.98.194.68
INFO  [RepairJobTask:1] 2015-01-28 02:07:29,065 StreamResultFuture.java:86 - 
[Stream #692c6270-a692-11e4-9973-070e938df227] Executing streaming plan for 
Repair
INFO  [StreamConnectionEstablisher:4] 2015-01-28 02:07:29,065 
StreamSession.java:213 - [Stream #692c6270-a692-11e4-9973-070e938df227] 
Starting streaming to /10.66.187.201
INFO  [StreamConnectionEstablisher:4] 2015-01-28 02:07:29,070 
StreamCoordinator.java:209 - [Stream #692c6270-a692-11e4-9973-070e938df227, 
ID#0] Beginning stream session with /10.66.187.201
INFO  [STREAM-IN-/10.66.187.201] 2015-01-28 02:07:29,465 
StreamResultFuture.java:166 - [Stream #692c6270-a692-11e4-9973-070e938df227 
ID#0] Prepare completed. Receiving 5 files(627994 bytes), sending 5 
files(632105 bytes)
INFO  [StreamReceiveTask:22] 2015-01-28 02:07:31,971 
StreamResultFuture.java:180 - [Stream #692c6270-a692-11e4-9973-070e938df227] 
Session with /10.66.187.201 is complete
INFO  [StreamReceiveTask:22] 2015-01-28 02:07:31,972 
StreamResultFuture.java:212 - [Stream #692c6270-a692-11e4-9973-070e938df227] 
All sessions completed
INFO  [StreamReceiveTask:22] 2015-01-28 02:07:31,972 
StreamingRepairTask.java:96 - [repair #685e3d00-a692-11e4-9973-070e938df227] 
streaming task succeed, returning response to /10.98.194.68
ERROR [RepairJobTask:1] 2015-01-28 02:07:39,444 RepairJob.java:127 - Error 
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /10.97.9.110
at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:347) 
~[apache-cassandra-2.1.2.jar:2.1.2]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_45]
at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_45]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_45]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
INFO  [AntiEntropySessions:6] 2015-01-28 02:07:39,445 RepairSession.java:260 - 
[repair #6f85e740-a692-11e4-9973-070e938df227] new session: will sync 
/10.98.194.68, /10.66.187.201, /10.226.218.135 on range 
(12817179804668051873746972069086
2638799,12863540308359254031520865977436165] for events.[bigint0text, 
bigint0boolean, bigint0int, dataset_catalog, column_categories, bigint0double, 
bigint0bigint]
ERROR [AntiEntropySessions:5] 2015-01-28 02:07:39,445 RepairSession.java:303 - 
[repair #685e3d00-a692-11e4-9973-070e938df227] session completed with the 
following error
java.io.IOException: Failed during snapshot creation.
at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) 
~[apache-cassandra-2.1.2.jar:2.1.2]
at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_45]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_45]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
ERROR [AntiEntropySessions:5] 2015-01-28 02:07:39,446 CassandraDaemon.java:153 
- Exception in thread Thread[AntiEntropySessions:5,5,RMI Runtime]
java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
creation.
at 

[jira] [Commented] (CASSANDRA-8691) SSTableReader.getPosition() does not correctly filter out queries that exceed its bounds

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-8691:
-

We need to consider behaviour with getToken(last) for OpenEarly, as this will 
break our current approach. However we could simply lookup EQ to last, since we 
have an overlapping record with the source files. This might be the neatest 
solution to that particular problem.

 SSTableReader.getPosition() does not correctly filter out queries that exceed 
 its bounds
 

 Key: CASSANDRA-8691
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8691
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Benedict
 Fix For: 2.1.3


 This has been true for some time, but I worry about including it before 2.1 
 in case we depend on the brokenness somehow. In getPosition, we test:
 {code}
 if (first.compareTo(key)  0 || last.compareTo(key)  0)
 {
 if (op == Operator.EQ  updateCacheAndStats)
 bloomFilterTracker.addFalsePositive();
 if (op.apply(1)  0)
 {
 Tracing.trace(Check against min and max keys allows skipping 
 sstable {}, descriptor.generation);
 return null;
 }
 }
 {code}
 However, op.apply(1) always returns = 0 unless op == Operation.EQ



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


[jira] [Commented] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-8683:
-

But reaching there is a different bug, especially if opened early. We 
explicitly leave an entire index summary page (and hence effective interval of 
index) distance between the last token and where we've written to, so that we 
have room to scan past if necessary.


 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the repairing files for non-incremental repairs, but the bug should exist in 
 2.1 as well)



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


[jira] [Commented] (CASSANDRA-8268) Token serialization should not assume all tokens are created by the database partitioner

2015-01-27 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-8268:


The branch rebased cleanly and ran tests successfully (updated 
[here|https://github.com/blambov/cassandra/tree/8268-token-serialization-partitioner]),
 the patch should work without changes.

 Token serialization should not assume all tokens are created by the database 
 partitioner
 

 Key: CASSANDRA-8268
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8268
 Project: Cassandra
  Issue Type: Bug
Reporter: Branimir Lambov
Assignee: Branimir Lambov
Priority: Minor
 Attachments: 8268.patch


 This is also a hidden reference to StorageService.getPartitioner within 
 Token, but this needs to be handled more carefully as we need to check 
 whether tokens serialized with the wrong partitioner could be present in user 
 data.



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


[jira] [Comment Edited] (CASSANDRA-8268) Token serialization should not assume all tokens are created by the database partitioner

2015-01-27 Thread Branimir Lambov (JIRA)

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

Branimir Lambov edited comment on CASSANDRA-8268 at 1/27/15 12:56 PM:
--

The branch rebased cleanly and ran tests successfully (updated [at the same 
location|https://github.com/blambov/cassandra/tree/8268-token-serialization-partitioner]),
 the patch should work without changes.


was (Author: blambov):
The branch rebased cleanly and ran tests successfully (updated 
[here|https://github.com/blambov/cassandra/tree/8268-token-serialization-partitioner]),
 the patch should work without changes.

 Token serialization should not assume all tokens are created by the database 
 partitioner
 

 Key: CASSANDRA-8268
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8268
 Project: Cassandra
  Issue Type: Bug
Reporter: Branimir Lambov
Assignee: Branimir Lambov
Priority: Minor
 Attachments: 8268.patch


 This is also a hidden reference to StorageService.getPartitioner within 
 Token, but this needs to be handled more carefully as we need to check 
 whether tokens serialized with the wrong partitioner could be present in user 
 data.



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


[jira] [Commented] (CASSANDRA-8268) Token serialization should not assume all tokens are created by the database partitioner

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko commented on CASSANDRA-8268:
--

Thanks. Will review ASAP.

 Token serialization should not assume all tokens are created by the database 
 partitioner
 

 Key: CASSANDRA-8268
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8268
 Project: Cassandra
  Issue Type: Bug
Reporter: Branimir Lambov
Assignee: Branimir Lambov
Priority: Minor
 Attachments: 8268.patch


 This is also a hidden reference to StorageService.getPartitioner within 
 Token, but this needs to be handled more carefully as we need to check 
 whether tokens serialized with the wrong partitioner could be present in user 
 data.



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


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

2015-01-27 Thread snazy
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: dd959f0f5dd6be7006c79b3f6ed2e9dbd98a720c
Parents: 6068dae 3e5edb8
Author: Ariel Weisberg ariel.weisb...@datastax.com
Authored: Tue Jan 27 13:34:39 2015 +0100
Committer: Robert Stupp sn...@snazy.de
Committed: Tue Jan 27 13:34:39 2015 +0100

--
 .../cassandra/config/DatabaseDescriptor.java| 153 ++-
 .../exceptions/ConfigurationException.java  |  16 ++
 .../cassandra/service/CassandraDaemon.java  |  36 -
 3 files changed, 123 insertions(+), 82 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd959f0f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
--
diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 8cc2da4,1dd1688..2891e9a
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@@ -95,16 -115,36 +95,16 @@@ public class DatabaseDescripto
  private static String localDC;
  private static ComparatorInetAddress localComparator;
  
++public static void forceStaticInitialization() {}
  static
  {
 -// In client mode, we use a default configuration. Note that the 
fields of this class will be
 -// left unconfigured however (the partitioner or localDC will be null 
for instance) so this
 -// should be used with care.
  try
  {
 -if (Config.isClientMode())
 -{
 -conf = new Config();
 -// at least we have to set memoryAllocator to open SSTable in 
client mode
 -memoryAllocator = 
FBUtilities.newOffHeapAllocator(conf.memory_allocator);
 -}
 -else
 -{
 -applyConfig(loadConfig());
 -}
 -}
 -catch (ConfigurationException e)
 -{
 -logger.error(Fatal configuration error, e);
 -System.err.println(e.getMessage() + \nFatal configuration error; 
unable to start. See log for stacktrace.);
 -System.exit(1);
 +applyConfig(loadConfig());
  }
  catch (Exception e)
  {
 -logger.error(Fatal error during configuration loading, e);
 -System.err.println(e.getMessage() + \nFatal error during 
configuration loading; unable to start. See log for stacktrace.);
--JVMStabilityInspector.inspectThrowable(e);
- throw new ExceptionInInitializerError(e.getMessage() + \nFatal 
configuration error; unable to start. See log for stacktrace.);
 -System.exit(1);
++throw new ExceptionInInitializerError(e);
  }
  }
  
@@@ -118,24 -158,45 +118,45 @@@
  return loader.loadConfig();
  }
  
+ private static InetAddress getNetworkInterfaceAddress(String intf, String 
configName) throws ConfigurationException
+ {
+ try
+ {
+ NetworkInterface ni = NetworkInterface.getByName(intf);
+ if (ni == null)
 -throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ could not be found);
++throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ could not be found, false);
+ EnumerationInetAddress addrs = ni.getInetAddresses();
+ if (!addrs.hasMoreElements())
 -throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ was found, but had no addresses);
++throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ was found, but had no addresses, false);
+ InetAddress retval = listenAddress = addrs.nextElement();
+ if (addrs.hasMoreElements())
 -throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ can't have more than one address);
++throw new ConfigurationException(Configured  + configName + 
 \ + intf + \ can't have more than one address, false);
+ return retval;
+ }
+ catch (SocketException e)
+ {
+ throw new ConfigurationException(Configured  + configName +  
\ + intf + \ caused an exception, e);
+ }
+ }
+ 
  private static void applyConfig(Config config) throws 
ConfigurationException
  {
  conf = config;
  
  if (conf.commitlog_sync == null)
  {
--throw new ConfigurationException(Missing required 

[jira] [Commented] (CASSANDRA-8683) Incremental repairs broken with early opening of compaction results

2015-01-27 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-8683:
-

So I introduced the following code to the end of getPosition:

{code}
assert op == SSTableReader.Operator.EQ;
if (updateCacheAndStats)
bloomFilterTracker.addFalsePositive();
Tracing.trace(Partition index lookup complete (bloom filter false 
positive) for sstable {}, descriptor.generation);
return null;
{code}

and this resulted in the following output (after enough runs):

{noformat}
[junit] Testcase: 
testValidationMultipleSSTablePerLevel(org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest):
  Caused an ERROR
[junit] java.lang.AssertionError
[junit] java.util.concurrent.ExecutionException: java.lang.AssertionError
[junit] at java.util.concurrent.FutureTask.report(FutureTask.java:122)
[junit] at java.util.concurrent.FutureTask.get(FutureTask.java:188)
[junit] at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest.testValidationMultipleSSTablePerLevel(LeveledCompactionStrategyTest.java:184)
[junit] Caused by: java.lang.AssertionError
[junit] at 
org.apache.cassandra.io.sstable.format.big.BigTableReader.getPosition(BigTableReader.java:243)
[junit] at 
org.apache.cassandra.io.sstable.format.SSTableReader.getPosition(SSTableReader.java:1355)
[junit] at 
org.apache.cassandra.io.sstable.format.SSTableReader.getPositionsForRanges(SSTableReader.java:1282)
[junit] at 
org.apache.cassandra.io.sstable.format.big.BigTableScanner.getScanner(BigTableScanner.java:67)
[junit] at 
org.apache.cassandra.io.sstable.format.big.BigTableReader.getScanner(BigTableReader.java:101)
[junit] at 
org.apache.cassandra.io.sstable.format.SSTableReader.getScanner(SSTableReader.java:1538)
[junit] at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.init(LeveledCompactionStrategy.java:318)
[junit] at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getScanners(LeveledCompactionStrategy.java:245)
[junit] at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getScanners(WrappingCompactionStrategy.java:357)
[junit] at 
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:999)
[junit] at 
org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:95)
[junit] at 
org.apache.cassandra.db.compaction.CompactionManager$9.call(CompactionManager.java:591)
[junit] at java.util.concurrent.FutureTask.run(FutureTask.java:262)
[junit] at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[junit] at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[junit] at java.lang.Thread.run(Thread.java:745)
{noformat}

Which suggests that it is indeed that we're skipping over the entire contents 
of our index page. However this may have nothing to do with the 
getToken().maxKeyBound() business, since that would likely be deterministic 
(although this is likely also a bug). It's possible that the memory being freed 
is happening at the right timing interval to give us a bad binary search 
result, though. This seems a little unlikely, but _is_ possible, so I suggest 
we get 7705 finished and running so we can see nail down the early release of 
memory, and then continue our investigations here.

 Incremental repairs broken with early opening of compaction results
 ---

 Key: CASSANDRA-8683
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8683
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Eriksson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-avoid-NPE-in-getPositionsForRanges.patch


 Incremental repairs holds a set of the sstables it started the repair on (we 
 need to know which sstables were actually validated to be able to anticompact 
 them). This includes any tmplink files that existed when the compaction 
 started (if we wouldn't include those, we would miss data since we move the 
 start point of the existing non-tmplink files)
 With CASSANDRA-6916 we swap out those instances with new ones 
 (SSTR.cloneWithNewStart / SSTW.openEarly), meaning that the underlying file 
 can get deleted even though we hold a reference.
 This causes the unit test error: 
 http://cassci.datastax.com/job/trunk_utest/1330/testReport/junit/org.apache.cassandra.db.compaction/LeveledCompactionStrategyTest/testValidationMultipleSSTablePerLevel/
 (note that it only fails on trunk though, in 2.1 we don't hold references to 
 the 

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

2015-01-27 Thread snazy
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 64e39a03966e8ccb301f5caf77f1b89d4eef1da4
Parents: dd959f0 325169e
Author: Ariel Weisberg ariel.weisb...@datastax.com
Authored: Tue Jan 27 13:41:33 2015 +0100
Committer: Robert Stupp sn...@snazy.de
Committed: Tue Jan 27 13:41:33 2015 +0100

--
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/64e39a03/CHANGES.txt
--
diff --cc CHANGES.txt
index 0c2db2f,b247127..5c6bb81
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,61 -1,7 +1,62 @@@
 -2.1.3
 +3.0
+  * rpc_interface and listen_interface generate NPE on startup when specified 
interface doesn't exist (CASSANDRA-8677)
   * Fix ArrayIndexOutOfBoundsException in nodetool cfhistograms 
(CASSANDRA-8514)
 - * Switch from yammer metrics for nodetool cf/proxy histograms 
(CASSANDRA-8662)
 + * Serializing Row cache alternative, fully off heap (CASSANDRA-7438)
 + * Duplicate rows returned when in clause has repeated values (CASSANDRA-6707)
 + * Make CassandraException unchecked, extend RuntimeException (CASSANDRA-8560)
 + * Support direct buffer decompression for reads (CASSANDRA-8464)
 + * DirectByteBuffer compatible LZ4 methods (CASSANDRA-7039)
 + * Add role based access control (CASSANDRA-7653)
 + * Group sstables for anticompaction correctly (CASSANDRA-8578)
 + * Add ReadFailureException to native protocol, respond
 +   immediately when replicas encounter errors while handling
 +   a read request (CASSANDRA-7886)
 + * Switch CommitLogSegment from RandomAccessFile to nio (CASSANDRA-8308)
 + * Allow mixing token and partition key restrictions (CASSANDRA-7016)
 + * Support index key/value entries on map collections (CASSANDRA-8473)
 + * Modernize schema tables (CASSANDRA-8261)
 + * Support for user-defined aggregation functions (CASSANDRA-8053)
 + * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419)
 + * Refactor SelectStatement, return IN results in natural order instead
 +   of IN value list order and ignore duplicate values in partition key IN 
restrictions (CASSANDRA-7981)
 + * Support UDTs, tuples, and collections in user-defined
 +   functions (CASSANDRA-7563)
 + * Fix aggregate fn results on empty selection, result column name,
 +   and cqlsh parsing (CASSANDRA-8229)
 + * Mark sstables as repaired after full repair (CASSANDRA-7586)
 + * Extend Descriptor to include a format value and refactor reader/writer
 +   APIs (CASSANDRA-7443)
 + * Integrate JMH for microbenchmarks (CASSANDRA-8151)
 + * Keep sstable levels when bootstrapping (CASSANDRA-7460)
 + * Add Sigar library and perform basic OS settings check on startup 
(CASSANDRA-7838)
 + * Support for aggregation functions (CASSANDRA-4914)
 + * Remove cassandra-cli (CASSANDRA-7920)
 + * Accept dollar quoted strings in CQL (CASSANDRA-7769)
 + * Make assassinate a first class command (CASSANDRA-7935)
 + * Support IN clause on any clustering column (CASSANDRA-4762)
 + * Improve compaction logging (CASSANDRA-7818)
 + * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917)
 + * Do anticompaction in groups (CASSANDRA-6851)
 + * Support user-defined functions (CASSANDRA-7395, 7526, 7562, 7740, 7781, 
7929,
 +   7924, 7812, 8063, 7813, 7708)
 + * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416)
 + * Move sstable RandomAccessReader to nio2, which allows using the
 +   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
 + * Remove CQL2 (CASSANDRA-5918)
 + * Add Thrift get_multi_slice call (CASSANDRA-6757)
 + * Optimize fetching multiple cells by name (CASSANDRA-6933)
 + * Allow compilation in java 8 (CASSANDRA-7028)
 + * Make incremental repair default (CASSANDRA-7250)
 + * Enable code coverage thru JaCoCo (CASSANDRA-7226)
 + * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
 + * Shorten SSTable path (CASSANDRA-6962)
 + * Use unsafe mutations for most unit tests (CASSANDRA-6969)
 + * Fix race condition during calculation of pending ranges (CASSANDRA-7390)
 + * Fail on very large batch sizes (CASSANDRA-8011)
 + * Improve concurrency of repair (CASSANDRA-6455, 8208)
 +
 +
 +2.1.3
   * Make sure we don't add tmplink files to the compaction
 strategy (CASSANDRA-8580)
   * (cqlsh) Handle maps with blob keys (CASSANDRA-8372)



[1/3] cassandra git commit: Add missing line to CHANGES.txt for CASSANDRA-8677

2015-01-27 Thread snazy
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 3e5edb82c - 325169e82
  refs/heads/trunk dd959f0f5 - 64e39a039


Add missing line to CHANGES.txt for CASSANDRA-8677


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

Branch: refs/heads/cassandra-2.1
Commit: 325169e824b1c5c3891446e4a26d32478fde3b45
Parents: 3e5edb8
Author: Ariel Weisberg ariel.weisb...@datastax.com
Authored: Tue Jan 27 13:40:32 2015 +0100
Committer: Robert Stupp sn...@snazy.de
Committed: Tue Jan 27 13:40:32 2015 +0100

--
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/325169e8/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 4d50496..b247127 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.3
+ * rpc_interface and listen_interface generate NPE on startup when specified 
interface doesn't exist (CASSANDRA-8677)
  * Fix ArrayIndexOutOfBoundsException in nodetool cfhistograms (CASSANDRA-8514)
  * Switch from yammer metrics for nodetool cf/proxy histograms (CASSANDRA-8662)
  * Make sure we don't add tmplink files to the compaction



[2/3] cassandra git commit: Add missing line to CHANGES.txt for CASSANDRA-8677

2015-01-27 Thread snazy
Add missing line to CHANGES.txt for CASSANDRA-8677


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

Branch: refs/heads/trunk
Commit: 325169e824b1c5c3891446e4a26d32478fde3b45
Parents: 3e5edb8
Author: Ariel Weisberg ariel.weisb...@datastax.com
Authored: Tue Jan 27 13:40:32 2015 +0100
Committer: Robert Stupp sn...@snazy.de
Committed: Tue Jan 27 13:40:32 2015 +0100

--
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/325169e8/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index 4d50496..b247127 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.3
+ * rpc_interface and listen_interface generate NPE on startup when specified 
interface doesn't exist (CASSANDRA-8677)
  * Fix ArrayIndexOutOfBoundsException in nodetool cfhistograms (CASSANDRA-8514)
  * Switch from yammer metrics for nodetool cf/proxy histograms (CASSANDRA-8662)
  * Make sure we don't add tmplink files to the compaction



[jira] [Updated] (CASSANDRA-8268) Token serialization should not assume all tokens are created by the database partitioner

2015-01-27 Thread Aleksey Yeschenko (JIRA)

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

Aleksey Yeschenko updated CASSANDRA-8268:
-
Fix Version/s: 3.0

 Token serialization should not assume all tokens are created by the database 
 partitioner
 

 Key: CASSANDRA-8268
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8268
 Project: Cassandra
  Issue Type: Bug
Reporter: Branimir Lambov
Assignee: Branimir Lambov
Priority: Minor
 Fix For: 3.0

 Attachments: 8268.patch


 This is also a hidden reference to StorageService.getPartitioner within 
 Token, but this needs to be handled more carefully as we need to check 
 whether tokens serialized with the wrong partitioner could be present in user 
 data.



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


[jira] [Updated] (CASSANDRA-8638) CQLSH -f option should ignore BOM in files

2015-01-27 Thread Abhishek Gupta (JIRA)

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

Abhishek Gupta updated CASSANDRA-8638:
--
Attachment: 0002-bug-CASSANDRA-8638.patch

Please review and use this patch 0002-bug-CASSANDRA-8638.patch

As discussed earlier.

 CQLSH -f option should ignore BOM in files
 --

 Key: CASSANDRA-8638
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8638
 Project: Cassandra
  Issue Type: Improvement
  Components: Tools
 Environment: Red Hat linux
Reporter: Sotirios Delimanolis
Priority: Trivial
  Labels: cqlsh, lhf
 Fix For: 2.1.3

 Attachments: 0001-bug-CASSANDRA-8638.patch, 
 0002-bug-CASSANDRA-8638.patch


 I fell in byte order mark trap trying to execute a CQL script through CQLSH. 
 The file contained the simple (plus BOM)
 {noformat}
 CREATE KEYSPACE IF NOT EXISTS xobni WITH replication = {'class': 
 'SimpleStrategy', 'replication_factor': '3'}  AND durable_writes = true; 
 -- and another CREATE TABLE bucket_flags query
 {noformat}
 I executed the script
 {noformat}
 [~]$ cqlsh --file /home/selimanolis/Schema/patches/setup.cql 
 /home/selimanolis/Schema/patches/setup.cql:2:Invalid syntax at char 1
 /home/selimanolis/Schema/patches/setup.cql:2:  CREATE KEYSPACE IF NOT EXISTS 
 test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 
 '3'}  AND durable_writes = true; 
 /home/selimanolis/Schema/patches/setup.cql:2:  ^
 /home/selimanolis/Schema/patches/setup.cql:22:ConfigurationException: 
 ErrorMessage code=2300 [Query invalid because of configuration issue] 
 message=Cannot add column family 'bucket_flags' to non existing keyspace 
 'test'.
 {noformat}
 I realized much later that the file had a BOM which was seemingly screwing 
 with how CQLSH parsed the file.
 It would be nice to have CQLSH ignore the BOM when processing files.



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


[jira] [Commented] (CASSANDRA-8684) Replace usage of Adler32 with CRC32

2015-01-27 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg commented on CASSANDRA-8684:
---

Added xxhash Java and Native implementations. Very impressive. Combining xxhash 
native and JNI and switching depending on the amount  of data being hashed 
would yield the best performance.

xxHashJava beats CRC32 with the intrinsic on Sandy Bridge at any size.
https://docs.google.com/spreadsheets/d/1cxf-V4b8dXdz1vLb5ySUNxK09bukDHHpq79a09xHw20/edit#gid=722827453

 Replace usage of Adler32 with CRC32
 ---

 Key: CASSANDRA-8684
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8684
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Ariel Weisberg
Assignee: Ariel Weisberg
 Attachments: CRCBenchmark.java, PureJavaCrc32.java, Sample.java


 I could not find a situation in which Adler32 outperformed PureJavaCrc32 much 
 less the intrinsic from Java 8. For small allocations PureJavaCrc32 was much 
 faster probably due to the JNI overhead of invoking the native Adler32 
 implementation where the array has to be allocated and copied.
 I tested on a 65w Sandy Bridge i5 running Ubuntu 14.04 with JDK 1.7.0_71 as 
 well as a c3.8xlarge running Ubuntu 14.04.
 I think it makes sense to stop using Adler32 when generating new checksums.
 c3.8xlarge, results are time in milliseconds, lower is better
 ||Allocation size|Adler32|CRC32|PureJavaCrc32||
 |64|47636|46075|25782|
 |128|36755|36712|23782|
 |256|31194|32211|22731|
 |1024|27194|28792|22010|
 |1048576|25941|27807|21808|
 |536870912|25957|27840|21836|
 i5
 ||Allocation size|Adler32|CRC32|PureJavaCrc32||
 |64|50539|50466|26826|
 |128|37092|38533|24553|
 |256|30630|32938|23459|
 |1024|26064|29079|22592|
 |1048576|24357|27911|22481|
 |536870912|24838|28360|22853|
 Another fun fact. Performance of the CRC32 intrinsic appears to double from 
 Sandy Bridge - Haswell. Unless I am measuring something different when going 
 from Linux/Sandy to Haswell/OS X.
 The intrinsic/JDK 8 implementation also operates against DirectByteBuffers 
 better and coding against the wrapper will get that boost when run with Java 
 8.



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


[jira] [Created] (CASSANDRA-8692) Coalesce intra-cluster network messages

2015-01-27 Thread Ariel Weisberg (JIRA)
Ariel Weisberg created CASSANDRA-8692:
-

 Summary: Coalesce intra-cluster network messages
 Key: CASSANDRA-8692
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8692
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Ariel Weisberg
Assignee: Ariel Weisberg


While researching CASSANDRA-8457 we found that it is effective and can be done 
without introducing additional latency at low concurrency/throughput.

The patch from that was used and found to be useful in a real life scenario so 
I propose we implement this in 2.1 in addition to 3.0.

The change set is a single file and is small enough to be reviewable.i



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


[jira] [Commented] (CASSANDRA-8635) STCS cold sstable omission does not handle overwrites without reads

2015-01-27 Thread Carl Yeksigian (JIRA)

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

Carl Yeksigian commented on CASSANDRA-8635:
---

Small nit: remove the unused params from the javadocs. Otherwise, +1

 STCS cold sstable omission does not handle overwrites without reads
 ---

 Key: CASSANDRA-8635
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8635
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Marcus Eriksson
Priority: Critical
 Fix For: 2.1.3

 Attachments: 
 0001-Include-cold-sstables-in-compactions-if-they-overlap.patch, 8635-v2.patch


 In 2.1, STCS may omit cold SSTables from compaction (CASSANDRA-6109).  If 
 data is regularly overwritten or deleted (but not enough to trigger a 
 single-sstable tombstone purging compaction), data size on disk may 
 continuously grow if:
 * The table receives very few reads
 * The reads only touch the newest SSTables
 Basically, if the overwritten data is never read and there aren't many 
 tombstones, STCS has no incentive to compact the sstables.  We should take 
 sstable overlap into consideration as well as coldness to address this case.



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


[jira] [Commented] (CASSANDRA-8542) Make get_range_slices and related succeed most of the time on tombstone heavy column families

2015-01-27 Thread Benjamin Janssen (JIRA)

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

Benjamin Janssen commented on CASSANDRA-8542:
-

Anyone had a chance to look into this ticket yet?

 Make get_range_slices and related succeed most of the time on tombstone heavy 
 column families
 -

 Key: CASSANDRA-8542
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8542
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Shawn Walker
Priority: Minor
 Attachments: trunk-8542-InvertibleBloomReconciler.txt


 I've got a ColumnFamily in which some rows end up being used in a queue-like 
 fashion, and where many tombstones tend to pile up at the left end of the 
 row.  As a result, I run into {{TombstoneOverwhelming}} (e.g. CASSANDRA-6117) 
 when trying to list the columns of said rows.
 Please don't yell at me too loudly. I know the issue I'm describing will 
 generally be considered as being due to poor use of Cassandra.  I understand 
 the rationale of the current behavior, and am hesitant to play with fire by 
 increasing {{tombstone_fail_threshold}} to a high value.  Instead, what I 
 propose is an alternate method of resolving such queries on the read path.
 
 This is based on the following observation: on the coordinator node, when 
 {{RangeSliceResponseResolver}} is resolving a range slice query, it needn't 
 be aware of all tombstones that all responding nodes have within the 
 specified range.  Instead, it would suffice if it could determine only those 
 tombstones which aren't shared by all responding nodes. E.g. if node #1 
 responds with tombstones (A, B, D), node #2 responds with tombstones (A, B, 
 C), and node #3 responds with tombstones (A, B, C, D), 
 {{RangeSliceResponseResolver}} need only actually know about the tombstones 
 (C, D): All of the responders will already have removed any relevant data for 
 the tombstones (A, B) from their individual responses.
 Arranging for {{RangeSliceResponseResolver}} to discover only the non-common 
 tombstones can be accomplished by using a variant of the invertible bloom 
 filter data structure described in e.g. 
 http://conferences.sigcomm.org/sigcomm/2011/papers/sigcomm/p218.pdf.  Using 
 an invertible Bloom filter, each responding node can build a (roughly) fixed 
 size data structure holding a representation of all the tombstones that node 
 encounters.  The coordinator node can then combine the invertible Bloom 
 filters.  If there aren't too many non-common tombstones, the coordinator 
 node will be able to enumerate all of them, and so resolve the range slice 
 query.
 I see accomplishing this in a few discrete steps:
 1. Implement an appropriate variant of the invertible bloom filter.  I've 
 started this already, and will shortly upload a patch including my 
 {{InvertibleBloomReconciler}} implementation.  From a black box perspective, 
 {{InvertibleBloomReconcilerTest.verifyFiveWayReconciliation()}} demonstrates 
 how this data structure and algorithm could be used.
 2. (db layer) Wrap {{InvertibleBloomReconciler}} into 
 {{TombstoneReconciler}}, a structure for spilling tombstones into.  Refactor 
 large swaths of {{org.apache.cassandra.db}}'s read path to accomodate the 
 possibility of placing tombstones discovered during a read into a 
 {{TombstoneReconciler}} instead of returning them within a {{Row}}, 
 {{ListRow}}, {{ColumnFamily}}, etc.  I've attempted a start on this, though 
 this means carrying the {{TombstoneReconciler}} around through most of 
 {{ColumnFamilyStore}}, practically all of {{org.apache.db.filter}}, and other 
 places I haven't yet discovered.  I'm wondering if I wouldn't be better off 
 waiting for CASSANDRA-8099 before starting this step -- a fully iterator flow 
 through {{org.apache.cassandra.db}} could make this easier, cleaner, and have 
 significantly lower code impact.
 3. (dynamo layer) Arrange for {{RangeSliceCommand}} to include parameters 
 for the IBR (table size, hash count, seed), possibly by making these part of 
 {{CFMetaData}}.  Allow a {{RangeSliceResponse}} to optionally return a 
 {{TombstoneReconciler}} in addition to its {{ListRow}}.  Refactor 
 {{RangeSliceResponseResolver}} to be capable of handling 
 {{TombstoneReconciler}} s.  Possibly refactor 
 {{StorageProxy.getRangeSlices(...)}} to disable read repair if any responses 
 contained a {{TombstoneReconciler}}.
 Since the invertible bloom filter is a probabilistic data structure, it is 
 possible that resolving a query in this manner could fail.  As such, I'm 
 proposing that the current read path not make use of the 
 {{TombstoneReconciler}} idea unless it would otherwise encounter a 
 

[jira] [Created] (CASSANDRA-8697) remove unused yaml setting: commitlog_periodic_queue_size

2015-01-27 Thread Dave Brosius (JIRA)
Dave Brosius created CASSANDRA-8697:
---

 Summary: remove unused yaml setting: commitlog_periodic_queue_size
 Key: CASSANDRA-8697
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8697
 Project: Cassandra
  Issue Type: Improvement
  Components: Config
Reporter: Dave Brosius
Assignee: Dave Brosius
Priority: Trivial
 Fix For: 3.0
 Attachments: commitlog_periodic_queue_size_removal.txt

remove yaml setting, and supporting code for commitlog_periodic_queue_size, as 
it's not used.



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


cassandra git commit: javadoc

2015-01-27 Thread dbrosius
Repository: cassandra
Updated Branches:
  refs/heads/trunk 02367e78b - 2b028b2b2


javadoc


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

Branch: refs/heads/trunk
Commit: 2b028b2b248e945d28a73537db9e41643138bcd7
Parents: 02367e7
Author: Dave Brosius dbros...@mebigfatguy.com
Authored: Tue Jan 27 21:48:24 2015 -0500
Committer: Dave Brosius dbros...@mebigfatguy.com
Committed: Tue Jan 27 21:48:24 2015 -0500

--
 src/java/org/apache/cassandra/sink/SinkManager.java | 4 
 1 file changed, 4 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b028b2b/src/java/org/apache/cassandra/sink/SinkManager.java
--
diff --git a/src/java/org/apache/cassandra/sink/SinkManager.java 
b/src/java/org/apache/cassandra/sink/SinkManager.java
index 9b422dc..303c107 100644
--- a/src/java/org/apache/cassandra/sink/SinkManager.java
+++ b/src/java/org/apache/cassandra/sink/SinkManager.java
@@ -25,6 +25,10 @@ import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 
+/**
+ * a class used only for testing to avoid sending/receiving data from a junit 
test.
+ * needs to be in the source tree as MessagingService calls it directly.
+ */
 public class SinkManager
 {
 private static final SetIMessageSink messageSinks = new 
CopyOnWriteArraySet();