[jira] [Commented] (CASSANDRA-6696) Drive replacement in JBOD can cause data to reappear.
[ https://issues.apache.org/jira/browse/CASSANDRA-6696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956205#comment-13956205 ] Marcus Eriksson commented on CASSANDRA-6696: pushed a new version to https://github.com/krummas/cassandra/commits/marcuse/6696-2 * removed SSTWInterface, instead created a helper class that is reused in most places * multithreaded flush, one thread per disk * support multiple flush dirs * sort compaction/flush dirs lexicographically to make sure we always put the same tokens on the same disks (even if you rearrange dirs in config etc) * avoids compaction loops by making sure we never start STCS compactions with any sstables that don't intersect (which the sstables on different disks wont) * RandomP and Murmur3P supported, the rest will dump data on the first disk for now TODO: * ask user@ for remove-OPP/BOP feedback, otherwise make them work with JBOD, in the old way Drive replacement in JBOD can cause data to reappear. -- Key: CASSANDRA-6696 URL: https://issues.apache.org/jira/browse/CASSANDRA-6696 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: Marcus Eriksson Fix For: 3.0 In JBOD, when someone gets a bad drive, the bad drive is replaced with a new empty one and repair is run. This can cause deleted data to come back in some cases. Also this is true for corrupt stables in which we delete the corrupt stable and run repair. Here is an example: Say we have 3 nodes A,B and C and RF=3 and GC grace=10days. row=sankalp col=sankalp is written 20 days back and successfully went to all three nodes. Then a delete/tombstone was written successfully for the same row column 15 days back. Since this tombstone is more than gc grace, it got compacted in Nodes A and B since it got compacted with the actual data. So there is no trace of this row column in node A and B. Now in node C, say the original data is in drive1 and tombstone is in drive2. Compaction has not yet reclaimed the data and tombstone. Drive2 becomes corrupt and was replaced with new empty drive. Due to the replacement, the tombstone in now gone and row=sankalp col=sankalp has come back to life. Now after replacing the drive we run repair. This data will be propagated to all nodes. Note: This is still a problem even if we run repair every gc grace. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-3569) Failure detector downs should not break streams
[ https://issues.apache.org/jira/browse/CASSANDRA-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956208#comment-13956208 ] Sylvain Lebresne commented on CASSANDRA-3569: - bq. other than configuring keepalive on streaming which looks to be a pretty trivial change Setting keep-alive on the connection is trivial, but the default keep alive time on most system (2 hours on linux I believe, or at least on my box) is way too long to be acceptable here. And to the best of my knowledge, we can't change the keep alive time from Java. So if we are to start relying on keep alive, we'll have to change the default keep alive time in some way, being it a call to sysctl (and whatever is equivalent on windows) in the startup script or some JNA call. bq. are we wanting to remove FD from the streaming process and rely on the underlying TCP mechanisms to handle that for us Provided we do deal with the keep alive default timeout as described above, I'm personally fine with relying on that and removing the FD use for streaming (but as said above, regarding repairs, we'll need to keep it for the initial merkle tree computation phase). Failure detector downs should not break streams --- Key: CASSANDRA-3569 URL: https://issues.apache.org/jira/browse/CASSANDRA-3569 Project: Cassandra Issue Type: New Feature Reporter: Peter Schuller Assignee: Joshua McKenzie Fix For: 2.1 beta3 Attachments: 3569-2.0.txt CASSANDRA-2433 introduced this behavior just to get repairs to don't sit there waiting forever. In my opinion the correct fix to that problem is to use TCP keep alive. Unfortunately the TCP keep alive period is insanely high by default on a modern Linux, so just doing that is not entirely good either. But using the failure detector seems non-sensicle to me. We have a communication method which is the TCP transport, that we know is used for long-running processes that you don't want to incorrectly be killed for no good reason, and we are using a failure detector tuned to detecting when not to send real-time sensitive request to nodes in order to actively kill a working connection. So, rather than add complexity with protocol based ping/pongs and such, I propose that we simply just use TCP keep alive for streaming connections and instruct operators of production clusters to tweak net.ipv4.tcp_keepalive_{probes,intvl} as appropriate (or whatever equivalent on their OS). I can submit the patch. Awaiting opinions. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6962) examine shortening path length post-5202
[ https://issues.apache.org/jira/browse/CASSANDRA-6962?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956215#comment-13956215 ] Sylvain Lebresne commented on CASSANDRA-6962: - I'm also not sure I'm a fan of completely removing any indication of the table name in the file name: feels pretty error prone. What about keeping the keyspace/table name as now (for the sake of making it easy to not mix sstables by mistake), but limit them to say 10 characters each (just for the file name), truncating the name if necessary? examine shortening path length post-5202 Key: CASSANDRA-6962 URL: https://issues.apache.org/jira/browse/CASSANDRA-6962 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brandon Williams Assignee: Yuki Morishita Fix For: 2.1 beta2 From CASSANDRA-5202 discussion: {quote} Did we give up on this? Could we clean up the redundancy a little by moving the ID into the directory name? e.g., ks/cf-uuid/version-generation-component.db I'm worried about path length, which is limited on Windows. Edit: to give a specific example, for KS foo Table bar we now have /var/lib/cassandra/flush/foo/bar-2fbb89709a6911e3b7dc4d7d4e3ca4b4/foo-bar-ka-1-Data.db I'm proposing /var/lib/cassandra/flush/foo/bar-2fbb89709a6911e3b7dc4d7d4e3ca4b4/ka-1-Data.db {quote} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6964) error in logs: ByteBuf.release() was not called before it's garbage-collected
[ https://issues.apache.org/jira/browse/CASSANDRA-6964?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6964: Fix Version/s: 2.1 error in logs: ByteBuf.release() was not called before it's garbage-collected - Key: CASSANDRA-6964 URL: https://issues.apache.org/jira/browse/CASSANDRA-6964 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch Fix For: 2.1 beta3 Attachments: node1.log Running some of our paging tests against 2.1 several of these exceptions are triggered: {noformat} run_tests: [java] SLF4J: Failed to load class org.slf4j.impl.StaticLoggerBinder. [java] SLF4J: Defaulting to no-operation (NOP) logger implementation [java] SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. [java] test_zero_page_size_ignored (__main__.TestPagingSize) ... ERROR [java] [java] == [java] ERROR: test_zero_page_size_ignored (__main__.TestPagingSize) [java] -- [java] Traceback (most recent call last): [java] File /home/rhatch/git/knifewine/cassandra-dtest-jython/base.py, line 152, in tearDown [java] raise AssertionError('Unexpected error in %s node log: %s' % (node.name, errors)) [java] AssertionError: Unexpected error in node1 node log: [ERROR [nioEventLoopGroup-3-2] 2014-03-31 17:16:44,337 Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's garbage-collected. Enable advanced leak reporting to find out where the leak occurred. To enable advanced leak reporting, specify the JVM option '-Dio.netty.leakDetectionLevel=advanced' or call ResourceLeakDetector.setLevel()\n] [java] [java] -- [java] Ran 1 test in 37.876s [java] [java] FAILED (errors=1) {noformat} These tests are run through jython with the java driver, so there's a little bit of setup needed (if you have ccm and dtest you are most of the way there): 1. clone and set up https://github.com/riptano/cassandra-dtest-jython . You may need to install ivy and copy ivy.jar to ~/.ant/lib/ivy.jar 2. you should have ccm, and CASSANDRA_DIR should be set in your environment 3. from the root of cassandra-dtest-jython run the tests with 'ant run_tests'. The tests take about 10 minutes run completely. 4. if you don't want to wait for the entire test suite to run, change the bottom of paging_test.py to just run a single test like so: {noformat} if __name__ == '__main__': suite = unittest.TestSuite() suite.addTest(TestPagingSize(test_zero_page_size_ignored)) unittest.TextTestRunner(verbosity=2).run(suite) exit(0) {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6964) error in logs: ByteBuf.release() was not called before it's garbage-collected
[ https://issues.apache.org/jira/browse/CASSANDRA-6964?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6964: Fix Version/s: (was: 2.1) 2.1 beta3 error in logs: ByteBuf.release() was not called before it's garbage-collected - Key: CASSANDRA-6964 URL: https://issues.apache.org/jira/browse/CASSANDRA-6964 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch Fix For: 2.1 beta3 Attachments: node1.log Running some of our paging tests against 2.1 several of these exceptions are triggered: {noformat} run_tests: [java] SLF4J: Failed to load class org.slf4j.impl.StaticLoggerBinder. [java] SLF4J: Defaulting to no-operation (NOP) logger implementation [java] SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. [java] test_zero_page_size_ignored (__main__.TestPagingSize) ... ERROR [java] [java] == [java] ERROR: test_zero_page_size_ignored (__main__.TestPagingSize) [java] -- [java] Traceback (most recent call last): [java] File /home/rhatch/git/knifewine/cassandra-dtest-jython/base.py, line 152, in tearDown [java] raise AssertionError('Unexpected error in %s node log: %s' % (node.name, errors)) [java] AssertionError: Unexpected error in node1 node log: [ERROR [nioEventLoopGroup-3-2] 2014-03-31 17:16:44,337 Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's garbage-collected. Enable advanced leak reporting to find out where the leak occurred. To enable advanced leak reporting, specify the JVM option '-Dio.netty.leakDetectionLevel=advanced' or call ResourceLeakDetector.setLevel()\n] [java] [java] -- [java] Ran 1 test in 37.876s [java] [java] FAILED (errors=1) {noformat} These tests are run through jython with the java driver, so there's a little bit of setup needed (if you have ccm and dtest you are most of the way there): 1. clone and set up https://github.com/riptano/cassandra-dtest-jython . You may need to install ivy and copy ivy.jar to ~/.ant/lib/ivy.jar 2. you should have ccm, and CASSANDRA_DIR should be set in your environment 3. from the root of cassandra-dtest-jython run the tests with 'ant run_tests'. The tests take about 10 minutes run completely. 4. if you don't want to wait for the entire test suite to run, change the bottom of paging_test.py to just run a single test like so: {noformat} if __name__ == '__main__': suite = unittest.TestSuite() suite.addTest(TestPagingSize(test_zero_page_size_ignored)) unittest.TextTestRunner(verbosity=2).run(suite) exit(0) {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6964) error in logs: ByteBuf.release() was not called before it's garbage-collected
[ https://issues.apache.org/jira/browse/CASSANDRA-6964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956220#comment-13956220 ] Sylvain Lebresne commented on CASSANDRA-6964: - Hum, even if those are not a problem (haven't checked really, though I'm slightly surprised that Netty would show such error message for no good reason), they will show up for people and freak them out. So let's keep that open as a reminder that this *needs* to be dealt with Cassandra side before the final 2.1 release. error in logs: ByteBuf.release() was not called before it's garbage-collected - Key: CASSANDRA-6964 URL: https://issues.apache.org/jira/browse/CASSANDRA-6964 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch Fix For: 2.1 beta3 Attachments: node1.log Running some of our paging tests against 2.1 several of these exceptions are triggered: {noformat} run_tests: [java] SLF4J: Failed to load class org.slf4j.impl.StaticLoggerBinder. [java] SLF4J: Defaulting to no-operation (NOP) logger implementation [java] SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. [java] test_zero_page_size_ignored (__main__.TestPagingSize) ... ERROR [java] [java] == [java] ERROR: test_zero_page_size_ignored (__main__.TestPagingSize) [java] -- [java] Traceback (most recent call last): [java] File /home/rhatch/git/knifewine/cassandra-dtest-jython/base.py, line 152, in tearDown [java] raise AssertionError('Unexpected error in %s node log: %s' % (node.name, errors)) [java] AssertionError: Unexpected error in node1 node log: [ERROR [nioEventLoopGroup-3-2] 2014-03-31 17:16:44,337 Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's garbage-collected. Enable advanced leak reporting to find out where the leak occurred. To enable advanced leak reporting, specify the JVM option '-Dio.netty.leakDetectionLevel=advanced' or call ResourceLeakDetector.setLevel()\n] [java] [java] -- [java] Ran 1 test in 37.876s [java] [java] FAILED (errors=1) {noformat} These tests are run through jython with the java driver, so there's a little bit of setup needed (if you have ccm and dtest you are most of the way there): 1. clone and set up https://github.com/riptano/cassandra-dtest-jython . You may need to install ivy and copy ivy.jar to ~/.ant/lib/ivy.jar 2. you should have ccm, and CASSANDRA_DIR should be set in your environment 3. from the root of cassandra-dtest-jython run the tests with 'ant run_tests'. The tests take about 10 minutes run completely. 4. if you don't want to wait for the entire test suite to run, change the bottom of paging_test.py to just run a single test like so: {noformat} if __name__ == '__main__': suite = unittest.TestSuite() suite.addTest(TestPagingSize(test_zero_page_size_ignored)) unittest.TextTestRunner(verbosity=2).run(suite) exit(0) {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6964) error in logs: ByteBuf.release() was not called before it's garbage-collected
[ https://issues.apache.org/jira/browse/CASSANDRA-6964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956224#comment-13956224 ] Benedict commented on CASSANDRA-6964: - Right, I'm suggesting suppressing the errors in the application (assuming they don't turn out to actually be a bug) so that they don't leak to the user. Either way, hopefully will be resolved soon. error in logs: ByteBuf.release() was not called before it's garbage-collected - Key: CASSANDRA-6964 URL: https://issues.apache.org/jira/browse/CASSANDRA-6964 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch Fix For: 2.1 beta3 Attachments: node1.log Running some of our paging tests against 2.1 several of these exceptions are triggered: {noformat} run_tests: [java] SLF4J: Failed to load class org.slf4j.impl.StaticLoggerBinder. [java] SLF4J: Defaulting to no-operation (NOP) logger implementation [java] SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further details. [java] test_zero_page_size_ignored (__main__.TestPagingSize) ... ERROR [java] [java] == [java] ERROR: test_zero_page_size_ignored (__main__.TestPagingSize) [java] -- [java] Traceback (most recent call last): [java] File /home/rhatch/git/knifewine/cassandra-dtest-jython/base.py, line 152, in tearDown [java] raise AssertionError('Unexpected error in %s node log: %s' % (node.name, errors)) [java] AssertionError: Unexpected error in node1 node log: [ERROR [nioEventLoopGroup-3-2] 2014-03-31 17:16:44,337 Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's garbage-collected. Enable advanced leak reporting to find out where the leak occurred. To enable advanced leak reporting, specify the JVM option '-Dio.netty.leakDetectionLevel=advanced' or call ResourceLeakDetector.setLevel()\n] [java] [java] -- [java] Ran 1 test in 37.876s [java] [java] FAILED (errors=1) {noformat} These tests are run through jython with the java driver, so there's a little bit of setup needed (if you have ccm and dtest you are most of the way there): 1. clone and set up https://github.com/riptano/cassandra-dtest-jython . You may need to install ivy and copy ivy.jar to ~/.ant/lib/ivy.jar 2. you should have ccm, and CASSANDRA_DIR should be set in your environment 3. from the root of cassandra-dtest-jython run the tests with 'ant run_tests'. The tests take about 10 minutes run completely. 4. if you don't want to wait for the entire test suite to run, change the bottom of paging_test.py to just run a single test like so: {noformat} if __name__ == '__main__': suite = unittest.TestSuite() suite.addTest(TestPagingSize(test_zero_page_size_ignored)) unittest.TextTestRunner(verbosity=2).run(suite) exit(0) {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
git commit: Fix LIMIT with static columns
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 a90b98e90 - abc6565e2 Fix LIMIT with static columns patch by slebresne; reviewed by iamaleksey for CASSANDRA-6956 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/abc6565e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/abc6565e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/abc6565e Branch: refs/heads/cassandra-2.0 Commit: abc6565e260d07614a400b0a83a56a20a7a0722f Parents: a90b98e Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:02:15 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:02:15 2014 +0200 -- CHANGES.txt | 1 + .../cassandra/db/filter/ColumnCounter.java | 23 2 files changed, 20 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/abc6565e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 3326c6c..8bfc8b9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -37,6 +37,7 @@ * Add QueryHandler interface (CASSANDRA-6659) * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945) * Make sure upgradesstables keeps sstable level (CASSANDRA-6958) + * Fix LIMT with static columns (CASSANDRA-6956) Merged from 1.2: * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816) * add extra SSL cipher suites (CASSANDRA-6613) http://git-wip-us.apache.org/repos/asf/cassandra/blob/abc6565e/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java index c2c0ade..83b138c 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@ -77,7 +77,8 @@ public class ColumnCounter { private final CompositeType type; private final int toGroup; -private ByteBuffer[] last; +private ByteBuffer[] previous; +private boolean previousGroupIsStatic; /** * A column counter that count only 1 for all the columns sharing a @@ -115,12 +116,17 @@ public class ColumnCounter ByteBuffer[] current = type.split(column.name()); assert current.length = toGroup; -if (last != null) +if (previous == null) +{ +// Only the first group can be static +previousGroupIsStatic = type.isStaticName(column.name()); +} +else { boolean isSameGroup = true; for (int i = 0; i toGroup; i++) { -if (ByteBufferUtil.compareUnsigned(last[i], current[i]) != 0) +if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) { isSameGroup = false; break; @@ -129,10 +135,19 @@ public class ColumnCounter if (isSameGroup) return; + +// We want to count the static group as 1 (CQL) row only if it's the only +// group in the partition. So, since we have already counted it at this point, +// just don't count the 2nd group if there is one and the first one was static +if (previousGroupIsStatic) +{ +previousGroupIsStatic = false; +return; +} } live++; -last = current; +previous = current; } } }
[1/2] git commit: Fix LIMIT with static columns
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 26191ca69 - 6776136a6 Fix LIMIT with static columns patch by slebresne; reviewed by iamaleksey for CASSANDRA-6956 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/abc6565e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/abc6565e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/abc6565e Branch: refs/heads/cassandra-2.1 Commit: abc6565e260d07614a400b0a83a56a20a7a0722f Parents: a90b98e Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:02:15 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:02:15 2014 +0200 -- CHANGES.txt | 1 + .../cassandra/db/filter/ColumnCounter.java | 23 2 files changed, 20 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/abc6565e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 3326c6c..8bfc8b9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -37,6 +37,7 @@ * Add QueryHandler interface (CASSANDRA-6659) * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945) * Make sure upgradesstables keeps sstable level (CASSANDRA-6958) + * Fix LIMT with static columns (CASSANDRA-6956) Merged from 1.2: * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816) * add extra SSL cipher suites (CASSANDRA-6613) http://git-wip-us.apache.org/repos/asf/cassandra/blob/abc6565e/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java index c2c0ade..83b138c 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@ -77,7 +77,8 @@ public class ColumnCounter { private final CompositeType type; private final int toGroup; -private ByteBuffer[] last; +private ByteBuffer[] previous; +private boolean previousGroupIsStatic; /** * A column counter that count only 1 for all the columns sharing a @@ -115,12 +116,17 @@ public class ColumnCounter ByteBuffer[] current = type.split(column.name()); assert current.length = toGroup; -if (last != null) +if (previous == null) +{ +// Only the first group can be static +previousGroupIsStatic = type.isStaticName(column.name()); +} +else { boolean isSameGroup = true; for (int i = 0; i toGroup; i++) { -if (ByteBufferUtil.compareUnsigned(last[i], current[i]) != 0) +if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) { isSameGroup = false; break; @@ -129,10 +135,19 @@ public class ColumnCounter if (isSameGroup) return; + +// We want to count the static group as 1 (CQL) row only if it's the only +// group in the partition. So, since we have already counted it at this point, +// just don't count the 2nd group if there is one and the first one was static +if (previousGroupIsStatic) +{ +previousGroupIsStatic = false; +return; +} } live++; -last = current; +previous = current; } } }
[2/2] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: src/java/org/apache/cassandra/db/filter/ColumnCounter.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6776136a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6776136a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6776136a Branch: refs/heads/cassandra-2.1 Commit: 6776136a66a08636b2ddde7e75b39651460fb0ae Parents: 26191ca abc6565 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:09:25 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:09:25 2014 +0200 -- CHANGES.txt | 1 + .../cassandra/db/filter/ColumnCounter.java | 28 ++-- 2 files changed, 21 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6776136a/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6776136a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --cc src/java/org/apache/cassandra/db/filter/ColumnCounter.java index 982cd62,83b138c..b8a9563 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@@ -73,9 -75,10 +73,9 @@@ public class ColumnCounte public static class GroupByPrefix extends ColumnCounter { -private final CompositeType type; +private final CellNameType type; private final int toGroup; - private CellName last; -private ByteBuffer[] previous; -private boolean previousGroupIsStatic; ++private CellName previous; /** * A column counter that count only 1 for all the columns sharing a @@@ -110,23 -113,37 +110,35 @@@ return; } -ByteBuffer[] current = type.split(column.name()); -assert current.length = toGroup; +CellName current = cell.name(); +assert current.size() = toGroup; - if (last != null) -if (previous == null) -{ -// Only the first group can be static -previousGroupIsStatic = type.isStaticName(column.name()); -} -else ++if (previous != null) { --boolean isSameGroup = true; --for (int i = 0; i toGroup; i++) ++boolean isSameGroup = previous.isStatic() == current.isStatic(); ++if (isSameGroup) { - if (type.subtype(i).compare(last.get(i), current.get(i)) != 0) -if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) ++for (int i = 0; i toGroup; i++) { --isSameGroup = false; --break; ++if (type.subtype(i).compare(previous.get(i), current.get(i)) != 0) ++{ ++isSameGroup = false; ++break; ++} } } if (isSameGroup) return; + + // We want to count the static group as 1 (CQL) row only if it's the only + // group in the partition. So, since we have already counted it at this point, + // just don't count the 2nd group if there is one and the first one was static -if (previousGroupIsStatic) ++if (previous.isStatic()) + { -previousGroupIsStatic = false; ++previous = current; + return; + } } live++;
git commit: Fix 6956 patch
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 abc6565e2 - d8c29a3f7 Fix 6956 patch Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d8c29a3f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d8c29a3f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d8c29a3f Branch: refs/heads/cassandra-2.0 Commit: d8c29a3f7e482c99a075f581902c0ed7d142cd6e Parents: abc6565 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:28:48 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:28:48 2014 +0200 -- .../org/apache/cassandra/db/filter/ColumnCounter.java | 14 +- 1 file changed, 9 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8c29a3f/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java index 83b138c..814d8ed 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@ -123,13 +123,16 @@ public class ColumnCounter } else { -boolean isSameGroup = true; -for (int i = 0; i toGroup; i++) +boolean isSameGroup = previousGroupIsStatic == type.isStaticName(column.name()); +if (isSameGroup) { -if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) +for (int i = 0; i toGroup; i++) { -isSameGroup = false; -break; +if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) +{ +isSameGroup = false; +break; +} } } @@ -141,6 +144,7 @@ public class ColumnCounter // just don't count the 2nd group if there is one and the first one was static if (previousGroupIsStatic) { +previous = current; previousGroupIsStatic = false; return; }
[1/2] git commit: Fix 6956 patch
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 6776136a6 - 353b2147b Fix 6956 patch Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d8c29a3f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d8c29a3f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d8c29a3f Branch: refs/heads/cassandra-2.1 Commit: d8c29a3f7e482c99a075f581902c0ed7d142cd6e Parents: abc6565 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:28:48 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:28:48 2014 +0200 -- .../org/apache/cassandra/db/filter/ColumnCounter.java | 14 +- 1 file changed, 9 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8c29a3f/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java index 83b138c..814d8ed 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@ -123,13 +123,16 @@ public class ColumnCounter } else { -boolean isSameGroup = true; -for (int i = 0; i toGroup; i++) +boolean isSameGroup = previousGroupIsStatic == type.isStaticName(column.name()); +if (isSameGroup) { -if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) +for (int i = 0; i toGroup; i++) { -isSameGroup = false; -break; +if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) +{ +isSameGroup = false; +break; +} } } @@ -141,6 +144,7 @@ public class ColumnCounter // just don't count the 2nd group if there is one and the first one was static if (previousGroupIsStatic) { +previous = current; previousGroupIsStatic = false; return; }
[2/2] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: src/java/org/apache/cassandra/db/filter/ColumnCounter.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/353b2147 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/353b2147 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/353b2147 Branch: refs/heads/cassandra-2.1 Commit: 353b2147b80eebab981434e1fd3f0e222428e664 Parents: 6776136 d8c29a3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:29:45 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:29:45 2014 +0200 -- --
[4/5] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: src/java/org/apache/cassandra/db/filter/ColumnCounter.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/353b2147 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/353b2147 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/353b2147 Branch: refs/heads/trunk Commit: 353b2147b80eebab981434e1fd3f0e222428e664 Parents: 6776136 d8c29a3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:29:45 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:29:45 2014 +0200 -- --
[5/5] git commit: Merge branch 'cassandra-2.1' into trunk
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/6660970c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6660970c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6660970c Branch: refs/heads/trunk Commit: 6660970c397a0ac53881422a35920bee2bf2d5f7 Parents: 5cdfa69 353b214 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:30:06 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:30:06 2014 +0200 -- CHANGES.txt | 1 + .../cassandra/db/filter/ColumnCounter.java | 28 ++-- 2 files changed, 21 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6660970c/CHANGES.txt --
[2/5] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: src/java/org/apache/cassandra/db/filter/ColumnCounter.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6776136a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6776136a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6776136a Branch: refs/heads/trunk Commit: 6776136a66a08636b2ddde7e75b39651460fb0ae Parents: 26191ca abc6565 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:09:25 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:09:25 2014 +0200 -- CHANGES.txt | 1 + .../cassandra/db/filter/ColumnCounter.java | 28 ++-- 2 files changed, 21 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6776136a/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6776136a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --cc src/java/org/apache/cassandra/db/filter/ColumnCounter.java index 982cd62,83b138c..b8a9563 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@@ -73,9 -75,10 +73,9 @@@ public class ColumnCounte public static class GroupByPrefix extends ColumnCounter { -private final CompositeType type; +private final CellNameType type; private final int toGroup; - private CellName last; -private ByteBuffer[] previous; -private boolean previousGroupIsStatic; ++private CellName previous; /** * A column counter that count only 1 for all the columns sharing a @@@ -110,23 -113,37 +110,35 @@@ return; } -ByteBuffer[] current = type.split(column.name()); -assert current.length = toGroup; +CellName current = cell.name(); +assert current.size() = toGroup; - if (last != null) -if (previous == null) -{ -// Only the first group can be static -previousGroupIsStatic = type.isStaticName(column.name()); -} -else ++if (previous != null) { --boolean isSameGroup = true; --for (int i = 0; i toGroup; i++) ++boolean isSameGroup = previous.isStatic() == current.isStatic(); ++if (isSameGroup) { - if (type.subtype(i).compare(last.get(i), current.get(i)) != 0) -if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) ++for (int i = 0; i toGroup; i++) { --isSameGroup = false; --break; ++if (type.subtype(i).compare(previous.get(i), current.get(i)) != 0) ++{ ++isSameGroup = false; ++break; ++} } } if (isSameGroup) return; + + // We want to count the static group as 1 (CQL) row only if it's the only + // group in the partition. So, since we have already counted it at this point, + // just don't count the 2nd group if there is one and the first one was static -if (previousGroupIsStatic) ++if (previous.isStatic()) + { -previousGroupIsStatic = false; ++previous = current; + return; + } } live++;
[3/5] git commit: Fix 6956 patch
Fix 6956 patch Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d8c29a3f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d8c29a3f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d8c29a3f Branch: refs/heads/trunk Commit: d8c29a3f7e482c99a075f581902c0ed7d142cd6e Parents: abc6565 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:28:48 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:28:48 2014 +0200 -- .../org/apache/cassandra/db/filter/ColumnCounter.java | 14 +- 1 file changed, 9 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8c29a3f/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java index 83b138c..814d8ed 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@ -123,13 +123,16 @@ public class ColumnCounter } else { -boolean isSameGroup = true; -for (int i = 0; i toGroup; i++) +boolean isSameGroup = previousGroupIsStatic == type.isStaticName(column.name()); +if (isSameGroup) { -if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) +for (int i = 0; i toGroup; i++) { -isSameGroup = false; -break; +if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) +{ +isSameGroup = false; +break; +} } } @@ -141,6 +144,7 @@ public class ColumnCounter // just don't count the 2nd group if there is one and the first one was static if (previousGroupIsStatic) { +previous = current; previousGroupIsStatic = false; return; }
[1/5] git commit: Fix LIMIT with static columns
Repository: cassandra Updated Branches: refs/heads/trunk 5cdfa69ca - 6660970c3 Fix LIMIT with static columns patch by slebresne; reviewed by iamaleksey for CASSANDRA-6956 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/abc6565e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/abc6565e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/abc6565e Branch: refs/heads/trunk Commit: abc6565e260d07614a400b0a83a56a20a7a0722f Parents: a90b98e Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 11:02:15 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 11:02:15 2014 +0200 -- CHANGES.txt | 1 + .../cassandra/db/filter/ColumnCounter.java | 23 2 files changed, 20 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/abc6565e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 3326c6c..8bfc8b9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -37,6 +37,7 @@ * Add QueryHandler interface (CASSANDRA-6659) * Track liveRatio per-memtable, not per-CF (CASSANDRA-6945) * Make sure upgradesstables keeps sstable level (CASSANDRA-6958) + * Fix LIMT with static columns (CASSANDRA-6956) Merged from 1.2: * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816) * add extra SSL cipher suites (CASSANDRA-6613) http://git-wip-us.apache.org/repos/asf/cassandra/blob/abc6565e/src/java/org/apache/cassandra/db/filter/ColumnCounter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java index c2c0ade..83b138c 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java @@ -77,7 +77,8 @@ public class ColumnCounter { private final CompositeType type; private final int toGroup; -private ByteBuffer[] last; +private ByteBuffer[] previous; +private boolean previousGroupIsStatic; /** * A column counter that count only 1 for all the columns sharing a @@ -115,12 +116,17 @@ public class ColumnCounter ByteBuffer[] current = type.split(column.name()); assert current.length = toGroup; -if (last != null) +if (previous == null) +{ +// Only the first group can be static +previousGroupIsStatic = type.isStaticName(column.name()); +} +else { boolean isSameGroup = true; for (int i = 0; i toGroup; i++) { -if (ByteBufferUtil.compareUnsigned(last[i], current[i]) != 0) +if (ByteBufferUtil.compareUnsigned(previous[i], current[i]) != 0) { isSameGroup = false; break; @@ -129,10 +135,19 @@ public class ColumnCounter if (isSameGroup) return; + +// We want to count the static group as 1 (CQL) row only if it's the only +// group in the partition. So, since we have already counted it at this point, +// just don't count the 2nd group if there is one and the first one was static +if (previousGroupIsStatic) +{ +previousGroupIsStatic = false; +return; +} } live++; -last = current; +previous = current; } } }
git commit: Clearer message for assertion error
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 d8c29a3f7 - 55fba6ab0 Clearer message for assertion error Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/55fba6ab Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/55fba6ab Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/55fba6ab Branch: refs/heads/cassandra-2.0 Commit: 55fba6ab0b5521711ed5c33a2d8ddadd2edd8906 Parents: d8c29a3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 12:09:27 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 12:09:27 2014 +0200 -- src/java/org/apache/cassandra/db/filter/ExtendedFilter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/55fba6ab/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java index 5aa1ea9..5c3662b 100644 --- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java @@ -255,7 +255,8 @@ public abstract class ExtendedFilter * 2) We don't yet allow non-indexed range slice with filters in CQL3 (i.e. this will never be * called by CFS.filter() for composites). */ -assert !(cfs.getComparator() instanceof CompositeType); +assert !(cfs.getComparator() instanceof CompositeType) : Sequential scan with filters is not supported (if you just created an index, you + + need to wait for the creation to be propagated to all nodes before querying it); if (!needsExtraQuery(rowKey.key, data)) return null;
[jira] [Commented] (CASSANDRA-6959) Reusing Keyspace and CF names raises assertion errors
[ https://issues.apache.org/jira/browse/CASSANDRA-6959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956338#comment-13956338 ] Sylvain Lebresne commented on CASSANDRA-6959: - I haven't looked at the CommitLog assertion (I'll let somewhat more up to date on the last commit log changes look at it), but regarding the 2nd one, the one in ExtendedFilter, that's because the test tries to use a 2ndary index before it's creation has been propagated to all nodes. More precisely, the {{select count(*) from ks.cf WHERE col1='asdf'}} of the test reaches the coordinator node1, which has seen the index creation and so the query is considered valid, but when the underlying RangeCommand reaches the 2nd node, that one doesn't yet know about the index and so fallback to a sequential scan, but we don't currently support that kind of sequential scan with filters hence the exception (and truly, that was not what the user asked about anyway). If you add a small sleep after the index creation, that exception goes away in particular. The current status quo (CASSANDRA-5886) is that it's the client job to check for schema agreement if need be and not query a table/index that is not fully propagated, and in that sense, it is the test that should be fixed. Note that I'm not against trying to have a clearer error message but I'm not really sure there's a much better place for this, so I did added a somewhat more explicit message to the assertionError which hopefully is good enough. Reusing Keyspace and CF names raises assertion errors - Key: CASSANDRA-6959 URL: https://issues.apache.org/jira/browse/CASSANDRA-6959 Project: Cassandra Issue Type: Bug Reporter: Ryan McGuire The [dtest I introduced|https://github.com/riptano/cassandra-dtest/commit/36960090d219ab8dbc7f108faa91c3ea5cea2bec] to test CASSANDRA-6924 introduces some log errors which I think may be related to CASSANDRA-5202. On 2.1 : {code} ERROR [MigrationStage:1] 2014-03-31 14:36:43,463 CommitLogSegmentManager.java:306 - Failed waiting for a forced recycle of in-use commit log segments java.lang.AssertionError: null at org.apache.cassandra.db.commitlog.CommitLogSegmentManager.forceRecycleAll(CommitLogSegmentManager.java:301) ~[main/:na] at org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:160) [main/:na] at org.apache.cassandra.db.DefsTables.dropColumnFamily(DefsTables.java:497) [main/:na] at org.apache.cassandra.db.DefsTables.mergeColumnFamilies(DefsTables.java:296) [main/:na] at org.apache.cassandra.db.DefsTables.mergeSchema(DefsTables.java:181) [main/:na] at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:49) [main/:na] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) [main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_51] 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] {code} On 2.0: {code} ERROR [ReadStage:3] 2014-03-31 13:28:11,014 CassandraDaemon.java (line 198) Exception in thread Thread[ReadStage:3,5,main] java.lang.AssertionError at org.apache.cassandra.db.filter.ExtendedFilter$WithClauses.getExtraFilter(ExtendedFilter.java:258) at org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1744) at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1699) at org.apache.cassandra.db.PagedRangeCommand.executeLocally(PagedRangeCommand.java:119) at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:39) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) {code} To reproduce, you many need to comment out the assertion in that test, as it is not 100% reproducible on the first try. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6959) Reusing Keyspace and CF names raises assertion errors
[ https://issues.apache.org/jira/browse/CASSANDRA-6959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956353#comment-13956353 ] Benedict commented on CASSANDRA-6959: - The problem with CL is almost certainly that it is attempting to flush all of the keyspaces/CFs that are dirty, but they don't all exist, so when it finishes the flushes there are still segments marked dirty so aren't recycled/retired. On drop of a keyspace/CF we need to mark the CL clean for all CFs dropped Reusing Keyspace and CF names raises assertion errors - Key: CASSANDRA-6959 URL: https://issues.apache.org/jira/browse/CASSANDRA-6959 Project: Cassandra Issue Type: Bug Reporter: Ryan McGuire The [dtest I introduced|https://github.com/riptano/cassandra-dtest/commit/36960090d219ab8dbc7f108faa91c3ea5cea2bec] to test CASSANDRA-6924 introduces some log errors which I think may be related to CASSANDRA-5202. On 2.1 : {code} ERROR [MigrationStage:1] 2014-03-31 14:36:43,463 CommitLogSegmentManager.java:306 - Failed waiting for a forced recycle of in-use commit log segments java.lang.AssertionError: null at org.apache.cassandra.db.commitlog.CommitLogSegmentManager.forceRecycleAll(CommitLogSegmentManager.java:301) ~[main/:na] at org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:160) [main/:na] at org.apache.cassandra.db.DefsTables.dropColumnFamily(DefsTables.java:497) [main/:na] at org.apache.cassandra.db.DefsTables.mergeColumnFamilies(DefsTables.java:296) [main/:na] at org.apache.cassandra.db.DefsTables.mergeSchema(DefsTables.java:181) [main/:na] at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:49) [main/:na] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) [main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_51] 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] {code} On 2.0: {code} ERROR [ReadStage:3] 2014-03-31 13:28:11,014 CassandraDaemon.java (line 198) Exception in thread Thread[ReadStage:3,5,main] java.lang.AssertionError at org.apache.cassandra.db.filter.ExtendedFilter$WithClauses.getExtraFilter(ExtendedFilter.java:258) at org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1744) at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1699) at org.apache.cassandra.db.PagedRangeCommand.executeLocally(PagedRangeCommand.java:119) at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:39) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) {code} To reproduce, you many need to comment out the assertion in that test, as it is not 100% reproducible on the first try. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6957) testNewRepairedSSTable fails intermittently
[ https://issues.apache.org/jira/browse/CASSANDRA-6957?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956369#comment-13956369 ] Marcus Eriksson commented on CASSANDRA-6957: [~mshuler] could you attach the server side logs from cassci when this fails? testNewRepairedSSTable fails intermittently --- Key: CASSANDRA-6957 URL: https://issues.apache.org/jira/browse/CASSANDRA-6957 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 beta2 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6966) Errors with Super Columns, mixup of 1.2 and 2.0
Nicolas Lalevée created CASSANDRA-6966: -- Summary: Errors with Super Columns, mixup of 1.2 and 2.0 Key: CASSANDRA-6966 URL: https://issues.apache.org/jira/browse/CASSANDRA-6966 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée On our test cluster, we tried a upgrade of Cassandra from 1.2.11 to 2.0.6. During the time we were running with 2 different versions of cassandra, there was errors in the logs: ERROR [WRITE-/10.10.0.41] 2014-03-19 11:23:27,523 OutboundTcpConnection.java (line 234) error writing to /10.10.0.41 java.lang.RuntimeException: Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first. at org.apache.cassandra.db.SuperColumns.sliceFilterToSC(SuperColumns.java:357) at org.apache.cassandra.db.SuperColumns.filterToSC(SuperColumns.java:258) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:192) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:134) at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:116) at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251) at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203) at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151) I confirm we do have old style super columns which were designed when cassandra was 1.0.x. Since in our test cluster the replication factor is 1, I can see errors on the client side, since 1 node among 2 was down. So I don't know for sure if this error in cassandra affected the client, the time frame is too short to be sure from the logs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6960) Cassandra requires ALLOW FILTERING for a range scan
[ https://issues.apache.org/jira/browse/CASSANDRA-6960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956406#comment-13956406 ] Sylvain Lebresne commented on CASSANDRA-6960: - bq. It seems that Cassandra should be able to perform the following query without ALLOW FILTERING Actually, it does require filtering, as we need to filter internally any partition that has nothing matching the filter (see part of [that comment|https://issues.apache.org/jira/browse/CASSANDRA-4915] on CASSANDRA-4915 too). More concretely, if you a million partition will each a million rows, and none of rows in the table match the filter, the query will still take a very long time to return no results, which is what ALLOW FILTERING is about warning you about. Cassandra requires ALLOW FILTERING for a range scan --- Key: CASSANDRA-6960 URL: https://issues.apache.org/jira/browse/CASSANDRA-6960 Project: Cassandra Issue Type: Bug Reporter: J.B. Langston Given this table definition: {code} CREATE TABLE metric_log_a ( destination_id text, rate_plan_id int, metric_name text, extraction_date 'org.apache.cassandra.db.marshal.TimestampType', metric_value text, PRIMARY KEY (destination_id, rate_plan_id, metric_name, extraction_date) ); {code} It seems that Cassandra should be able to perform the following query without ALLOW FILTERING: {code} select destination_id, rate_plan_id, metric_name, extraction_date, metric_value from metric_log_a where token(destination_id) ? and token(destination_id) = ? and rate_plan_id=90 and metric_name='minutesOfUse' and extraction_date = '2014-03-05' and extraction_date = '2014-03-05' allow filtering; {code} However, it will refuse to run unless ALLOW FILTERING is specified. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6967) upgradesstables won't upgrade the system-Versions-hf files
Nicolas Lalevée created CASSANDRA-6967: -- Summary: upgradesstables won't upgrade the system-Versions-hf files Key: CASSANDRA-6967 URL: https://issues.apache.org/jira/browse/CASSANDRA-6967 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée In order to properly upgrade our test cluster from 1.2.11 to 2.0.6, we had to perform some upgradesstables, Cassandra 2.0.6 was not starting due to some file in version 'hf'. To prepare the upgrade on prod, I have looked up for some 'hf' files, and I have found some on all nodes, all being /var/lib/cassandra/data/system/Versions/system-Versions-hf-* And I have tried many upgradesstables command, there are still lying there. {noformat} # nodetool upgradesstables system Versions Exception in thread main java.lang.IllegalArgumentException: Unknown table/cf pair (system.Versions) # nodetool upgradesstables system # nodetool upgradesstables # nodetool upgradesstables -a system # ls /var/lib/cassandra/data/system/Versions/*-hf-* | wc -l 15 {noformat} I did not try nodetool upgradesstables -a since we have a lot of data. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6966) Errors with Super Columns, mixup of 1.2 and 2.0
[ https://issues.apache.org/jira/browse/CASSANDRA-6966?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956464#comment-13956464 ] Sylvain Lebresne commented on CASSANDRA-6966: - Did you start accessing your super column families from CQL on the 2.0 nodes before all nodes were upgraded to 2.0? Errors with Super Columns, mixup of 1.2 and 2.0 --- Key: CASSANDRA-6966 URL: https://issues.apache.org/jira/browse/CASSANDRA-6966 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée On our test cluster, we tried a upgrade of Cassandra from 1.2.11 to 2.0.6. During the time we were running with 2 different versions of cassandra, there was errors in the logs: ERROR [WRITE-/10.10.0.41] 2014-03-19 11:23:27,523 OutboundTcpConnection.java (line 234) error writing to /10.10.0.41 java.lang.RuntimeException: Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first. at org.apache.cassandra.db.SuperColumns.sliceFilterToSC(SuperColumns.java:357) at org.apache.cassandra.db.SuperColumns.filterToSC(SuperColumns.java:258) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:192) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:134) at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:116) at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251) at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203) at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151) I confirm we do have old style super columns which were designed when cassandra was 1.0.x. Since in our test cluster the replication factor is 1, I can see errors on the client side, since 1 node among 2 was down. So I don't know for sure if this error in cassandra affected the client, the time frame is too short to be sure from the logs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6966) Errors with Super Columns, mixup of 1.2 and 2.0
[ https://issues.apache.org/jira/browse/CASSANDRA-6966?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956470#comment-13956470 ] Nicolas Lalevée commented on CASSANDRA-6966: No, we have currently no code which use the CQL API, all is done via Thrift (through Hector). Errors with Super Columns, mixup of 1.2 and 2.0 --- Key: CASSANDRA-6966 URL: https://issues.apache.org/jira/browse/CASSANDRA-6966 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée On our test cluster, we tried a upgrade of Cassandra from 1.2.11 to 2.0.6. During the time we were running with 2 different versions of cassandra, there was errors in the logs: ERROR [WRITE-/10.10.0.41] 2014-03-19 11:23:27,523 OutboundTcpConnection.java (line 234) error writing to /10.10.0.41 java.lang.RuntimeException: Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first. at org.apache.cassandra.db.SuperColumns.sliceFilterToSC(SuperColumns.java:357) at org.apache.cassandra.db.SuperColumns.filterToSC(SuperColumns.java:258) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:192) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:134) at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:116) at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251) at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203) at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151) I confirm we do have old style super columns which were designed when cassandra was 1.0.x. Since in our test cluster the replication factor is 1, I can see errors on the client side, since 1 node among 2 was down. So I don't know for sure if this error in cassandra affected the client, the time frame is too short to be sure from the logs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6967) upgradesstables won't upgrade the system-Versions-hf files
[ https://issues.apache.org/jira/browse/CASSANDRA-6967?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6967: Priority: Trivial (was: Major) upgradesstables won't upgrade the system-Versions-hf files -- Key: CASSANDRA-6967 URL: https://issues.apache.org/jira/browse/CASSANDRA-6967 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée Priority: Trivial In order to properly upgrade our test cluster from 1.2.11 to 2.0.6, we had to perform some upgradesstables, Cassandra 2.0.6 was not starting due to some file in version 'hf'. To prepare the upgrade on prod, I have looked up for some 'hf' files, and I have found some on all nodes, all being /var/lib/cassandra/data/system/Versions/system-Versions-hf-* And I have tried many upgradesstables command, there are still lying there. {noformat} # nodetool upgradesstables system Versions Exception in thread main java.lang.IllegalArgumentException: Unknown table/cf pair (system.Versions) # nodetool upgradesstables system # nodetool upgradesstables # nodetool upgradesstables -a system # ls /var/lib/cassandra/data/system/Versions/*-hf-* | wc -l 15 {noformat} I did not try nodetool upgradesstables -a since we have a lot of data. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6967) upgradesstables won't upgrade the system-Versions-hf files
[ https://issues.apache.org/jira/browse/CASSANDRA-6967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956471#comment-13956471 ] Sylvain Lebresne commented on CASSANDRA-6967: - You can just delete those system-Versions files. The Versions system table is an old table that has been removed in C* 1.2. Not sure why you still have sstables around for that old table, but upgradesstables is right to tell that this table is unknown. Maybe we can add some cleanup at starting in 1.2 to remove those sstables? upgradesstables won't upgrade the system-Versions-hf files -- Key: CASSANDRA-6967 URL: https://issues.apache.org/jira/browse/CASSANDRA-6967 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée In order to properly upgrade our test cluster from 1.2.11 to 2.0.6, we had to perform some upgradesstables, Cassandra 2.0.6 was not starting due to some file in version 'hf'. To prepare the upgrade on prod, I have looked up for some 'hf' files, and I have found some on all nodes, all being /var/lib/cassandra/data/system/Versions/system-Versions-hf-* And I have tried many upgradesstables command, there are still lying there. {noformat} # nodetool upgradesstables system Versions Exception in thread main java.lang.IllegalArgumentException: Unknown table/cf pair (system.Versions) # nodetool upgradesstables system # nodetool upgradesstables # nodetool upgradesstables -a system # ls /var/lib/cassandra/data/system/Versions/*-hf-* | wc -l 15 {noformat} I did not try nodetool upgradesstables -a since we have a lot of data. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6967) upgradesstables won't upgrade the system-Versions-hf files
[ https://issues.apache.org/jira/browse/CASSANDRA-6967?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nicolas Lalevée updated CASSANDRA-6967: --- Issue Type: Improvement (was: Bug) OK. Thus setting this issue as Improvement. For me, just some lines about it in the doc or in the release notes would be sufficient. upgradesstables won't upgrade the system-Versions-hf files -- Key: CASSANDRA-6967 URL: https://issues.apache.org/jira/browse/CASSANDRA-6967 Project: Cassandra Issue Type: Improvement Reporter: Nicolas Lalevée Priority: Trivial In order to properly upgrade our test cluster from 1.2.11 to 2.0.6, we had to perform some upgradesstables, Cassandra 2.0.6 was not starting due to some file in version 'hf'. To prepare the upgrade on prod, I have looked up for some 'hf' files, and I have found some on all nodes, all being /var/lib/cassandra/data/system/Versions/system-Versions-hf-* And I have tried many upgradesstables command, there are still lying there. {noformat} # nodetool upgradesstables system Versions Exception in thread main java.lang.IllegalArgumentException: Unknown table/cf pair (system.Versions) # nodetool upgradesstables system # nodetool upgradesstables # nodetool upgradesstables -a system # ls /var/lib/cassandra/data/system/Versions/*-hf-* | wc -l 15 {noformat} I did not try nodetool upgradesstables -a since we have a lot of data. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-6965) sstableloader fails to stream data
[ https://issues.apache.org/jira/browse/CASSANDRA-6965?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams reassigned CASSANDRA-6965: --- Assignee: Yuki Morishita sstableloader fails to stream data -- Key: CASSANDRA-6965 URL: https://issues.apache.org/jira/browse/CASSANDRA-6965 Project: Cassandra Issue Type: Bug Reporter: Ryan McGuire Assignee: Yuki Morishita When loading a snapshot via sstableloader I get this NullPointerException: {code} Established connection to initial hosts Opening sstables and calculating sections to stream Streaming relevant part of /tmp/tmpmusajE/ks/cf/ks-cf-ka-1-Data.db to [/127.0.0.1] Streaming session ID: 276a9d20-b944-11e3-9f60-4191ec2d6f66 Exception in thread STREAM-OUT-/127.0.0.1 java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler$MessageHandler.signalCloseDone(ConnectionHandler.java:249) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:375) at java.lang.Thread.run(Thread.java:744) {code} Which at first I thought meant it was unable to connect, but I do see this in system.log, indicating the connection was successful: {code} INFO [STREAM-INIT-/127.0.0.1:55820] 2014-03-31 22:20:12,265 StreamResultFuture.java:121 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Received streaming plan for Bulk Load INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,272 StreamResultFuture.java:173 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Prepare completed. Receiving 1 files(1726 bytes), sending 0 files(0 bytes) INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,280 CompressedStreamReader.java:65 - reading file from /10.13.37.130, repairedAt = 0 {code} To reproduce this behaviour, run the [snapshot_test.py:TestSnapshot.test_basic_snapshot dtest|https://github.com/riptano/cassandra-dtest/blob/85692be39cb6b1b5ebc96905834ff7bb174de0bc/snapshot_test.py#L40] on 2.1 HEAD. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6937) On large insert, node is being marked down with Commit Log failure
[ https://issues.apache.org/jira/browse/CASSANDRA-6937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956494#comment-13956494 ] jonathan lacefield commented on CASSANDRA-6937: --- recommend closing this one. The client is not able to reproduce. On large insert, node is being marked down with Commit Log failure -- Key: CASSANDRA-6937 URL: https://issues.apache.org/jira/browse/CASSANDRA-6937 Project: Cassandra Issue Type: Bug Environment: 16 vcpu, 480GB SSD storage (used 7GB), 32GB RAM, Cassandra 2.0.5.24, hsha, datastax Java driver 2.0.1, vnodes Reporter: jonathan lacefield Assignee: Benedict Priority: Minor Attachments: cassandra.yaml copy, system.log during large insert, will get specs on insert load, nodes in the cluster are being marked down. creating defect for investigation. error message in log file of one of the down nodes: ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:06,666 StorageService.java (line 364) Stopping gossiper WARN [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:06,666 StorageService.java (line 278) Stopping gossip by operator request INFO [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:06,666 Gossiper.java (line 1251) Announcing shutdown INFO [FlushWriter:421] 2014-03-26 08:28:06,677 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data1/system/compactions_in_progress/system-compactions_in_progress-jb-198-Data.db (166 bytes) for commitlog position ReplayPosition(segmentId=1395644152684, position=9577531) INFO [CompactionExecutor:1880] 2014-03-26 08:28:06,679 CompactionTask.java (line 115) Compacting [SSTableReader(path='/var/lib/cassandra/data2/OpsCenter/rollups60/OpsCenter-rollups60-jb-18-Data.db'), SSTableReader(path='/var/lib/cassandra/data1/OpsCenter/rollups60/OpsCenter-rollups60-jb-19-Data.db'), SSTableReader(path='/var/lib/cassandra/data2/OpsCenter/rollups60/OpsCenter-rollups60-jb-17-Data.db'), SSTableReader(path='/var/lib/cassandra/data1/OpsCenter/rollups60/OpsCenter-rollups60-jb-20-Data.db')] INFO [OptionalTasks:1] 2014-03-26 08:28:07,322 MeteredFlusher.java (line 63) flushing high-traffic column family CFS(Keyspace='system', ColumnFamily='batchlog') (estimated 215899526 bytes) INFO [OptionalTasks:1] 2014-03-26 08:28:07,322 ColumnFamilyStore.java (line 784) Enqueuing flush of Memtable-batchlog@74009414(39242953/215899526 serialized/live bytes, 239 ops) INFO [FlushWriter:422] 2014-03-26 08:28:07,323 Memtable.java (line 331) Writing Memtable-batchlog@74009414(39242953/215899526 serialized/live bytes, 239 ops) INFO [FlushWriter:422] 2014-03-26 08:28:07,370 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data1/system/batchlog/system-batchlog-jb-1-Data.db (204617 bytes) for commitlog position ReplayPosition(segmentId=1395644152684, position=17705519) INFO [MemoryMeter:1] 2014-03-26 08:28:08,296 Memtable.java (line 449) CFS(Keyspace='system', ColumnFamily='batchlog') liveRatio is 3.2515027780484544 (just-counted was 1.0013929201634204). calculation took 0ms for 12 cells ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,667 StorageService.java (line 369) Stopping RPC server INFO [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,667 ThriftServer.java (line 141) Stop listening to thrift clients ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,698 StorageService.java (line 374) Stopping native transport INFO [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,765 Server.java (line 181) Stop listening for CQL clients ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,766 CommitLog.java (line 379) Failed to allocate new commit log segments. Commit disk failure policy is stop; terminating thread java.lang.AssertionError: attempted to delete non-existing file CommitLog-3-1395644152652.log -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000
[ https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956496#comment-13956496 ] Christopher Smith commented on CASSANDRA-6106: -- So, looking at JNA, it seems like shared memory/NIO buffer logic could be done with it as much as it could be done with JNI. Is it possible what I'm talking about could be done through JNA as much as with JNI? QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000 Key: CASSANDRA-6106 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106 Project: Cassandra Issue Type: Improvement Components: Core Environment: DSE Cassandra 3.1, but also HEAD Reporter: Christopher Smith Assignee: Benedict Priority: Minor Labels: timestamps Fix For: 2.1 beta2 Attachments: microtimstamp.patch, microtimstamp_random.patch, microtimstamp_random_rev2.patch I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra mentioned issues with millisecond rounding in timestamps and was able to reproduce the issue. If I specify a timestamp in a mutating query, I get microsecond precision, but if I don't, I get timestamps rounded to the nearest millisecond, at least for my first query on a given connection, which substantially increases the possibilities of collision. I believe I found the offending code, though I am by no means sure this is comprehensive. I think we probably need a fairly comprehensive replacement of all uses of System.currentTimeMillis() with System.nanoTime(). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6962) examine shortening path length post-5202
[ https://issues.apache.org/jira/browse/CASSANDRA-6962?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956514#comment-13956514 ] Yuki Morishita commented on CASSANDRA-6962: --- bq. limit them to say 10 characters each (just for the file name), truncating the name if necessary? We can truncate name to fit within os path limit adaptively with some calculation. How about completely omit Keyspace name but keep ColumnFamily name and adaptively adjust(truncate) its name? examine shortening path length post-5202 Key: CASSANDRA-6962 URL: https://issues.apache.org/jira/browse/CASSANDRA-6962 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brandon Williams Assignee: Yuki Morishita Fix For: 2.1 beta2 From CASSANDRA-5202 discussion: {quote} Did we give up on this? Could we clean up the redundancy a little by moving the ID into the directory name? e.g., ks/cf-uuid/version-generation-component.db I'm worried about path length, which is limited on Windows. Edit: to give a specific example, for KS foo Table bar we now have /var/lib/cassandra/flush/foo/bar-2fbb89709a6911e3b7dc4d7d4e3ca4b4/foo-bar-ka-1-Data.db I'm proposing /var/lib/cassandra/flush/foo/bar-2fbb89709a6911e3b7dc4d7d4e3ca4b4/ka-1-Data.db {quote} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-6966) Errors with Super Columns, mixup of 1.2 and 2.0
[ https://issues.apache.org/jira/browse/CASSANDRA-6966?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne reassigned CASSANDRA-6966: --- Assignee: Sylvain Lebresne Errors with Super Columns, mixup of 1.2 and 2.0 --- Key: CASSANDRA-6966 URL: https://issues.apache.org/jira/browse/CASSANDRA-6966 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée Assignee: Sylvain Lebresne On our test cluster, we tried a upgrade of Cassandra from 1.2.11 to 2.0.6. During the time we were running with 2 different versions of cassandra, there was errors in the logs: ERROR [WRITE-/10.10.0.41] 2014-03-19 11:23:27,523 OutboundTcpConnection.java (line 234) error writing to /10.10.0.41 java.lang.RuntimeException: Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first. at org.apache.cassandra.db.SuperColumns.sliceFilterToSC(SuperColumns.java:357) at org.apache.cassandra.db.SuperColumns.filterToSC(SuperColumns.java:258) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:192) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:134) at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:116) at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251) at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203) at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151) I confirm we do have old style super columns which were designed when cassandra was 1.0.x. Since in our test cluster the replication factor is 1, I can see errors on the client side, since 1 node among 2 was down. So I don't know for sure if this error in cassandra affected the client, the time frame is too short to be sure from the logs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6966) Errors with Super Columns, mixup of 1.2 and 2.0
[ https://issues.apache.org/jira/browse/CASSANDRA-6966?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6966: Attachment: 6966.txt We indeed had a bug (2 in fact) when translating internal filters to thrift SC filters. Attaching a fix, and I've pushed [a dtest|https://github.com/riptano/cassandra-dtest/commit/7f24091e5ba4220d58649cd7e49e5403630ea646] for this. Errors with Super Columns, mixup of 1.2 and 2.0 --- Key: CASSANDRA-6966 URL: https://issues.apache.org/jira/browse/CASSANDRA-6966 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée Assignee: Sylvain Lebresne Fix For: 2.0.7 Attachments: 6966.txt On our test cluster, we tried a upgrade of Cassandra from 1.2.11 to 2.0.6. During the time we were running with 2 different versions of cassandra, there was errors in the logs: ERROR [WRITE-/10.10.0.41] 2014-03-19 11:23:27,523 OutboundTcpConnection.java (line 234) error writing to /10.10.0.41 java.lang.RuntimeException: Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first. at org.apache.cassandra.db.SuperColumns.sliceFilterToSC(SuperColumns.java:357) at org.apache.cassandra.db.SuperColumns.filterToSC(SuperColumns.java:258) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:192) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:134) at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:116) at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251) at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203) at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151) I confirm we do have old style super columns which were designed when cassandra was 1.0.x. Since in our test cluster the replication factor is 1, I can see errors on the client side, since 1 node among 2 was down. So I don't know for sure if this error in cassandra affected the client, the time frame is too short to be sure from the logs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6696) Drive replacement in JBOD can cause data to reappear.
[ https://issues.apache.org/jira/browse/CASSANDRA-6696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956568#comment-13956568 ] Jonathan Ellis commented on CASSANDRA-6696: --- Can you add javadoc for splitranges? Why is it partitioner-dependent? Drive replacement in JBOD can cause data to reappear. -- Key: CASSANDRA-6696 URL: https://issues.apache.org/jira/browse/CASSANDRA-6696 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: Marcus Eriksson Fix For: 3.0 In JBOD, when someone gets a bad drive, the bad drive is replaced with a new empty one and repair is run. This can cause deleted data to come back in some cases. Also this is true for corrupt stables in which we delete the corrupt stable and run repair. Here is an example: Say we have 3 nodes A,B and C and RF=3 and GC grace=10days. row=sankalp col=sankalp is written 20 days back and successfully went to all three nodes. Then a delete/tombstone was written successfully for the same row column 15 days back. Since this tombstone is more than gc grace, it got compacted in Nodes A and B since it got compacted with the actual data. So there is no trace of this row column in node A and B. Now in node C, say the original data is in drive1 and tombstone is in drive2. Compaction has not yet reclaimed the data and tombstone. Drive2 becomes corrupt and was replaced with new empty drive. Due to the replacement, the tombstone in now gone and row=sankalp col=sankalp has come back to life. Now after replacing the drive we run repair. This data will be propagated to all nodes. Note: This is still a problem even if we run repair every gc grace. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-6960) Cassandra requires ALLOW FILTERING for a range scan
[ https://issues.apache.org/jira/browse/CASSANDRA-6960?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-6960. --- Resolution: Not a Problem Cassandra requires ALLOW FILTERING for a range scan --- Key: CASSANDRA-6960 URL: https://issues.apache.org/jira/browse/CASSANDRA-6960 Project: Cassandra Issue Type: Bug Reporter: J.B. Langston Given this table definition: {code} CREATE TABLE metric_log_a ( destination_id text, rate_plan_id int, metric_name text, extraction_date 'org.apache.cassandra.db.marshal.TimestampType', metric_value text, PRIMARY KEY (destination_id, rate_plan_id, metric_name, extraction_date) ); {code} It seems that Cassandra should be able to perform the following query without ALLOW FILTERING: {code} select destination_id, rate_plan_id, metric_name, extraction_date, metric_value from metric_log_a where token(destination_id) ? and token(destination_id) = ? and rate_plan_id=90 and metric_name='minutesOfUse' and extraction_date = '2014-03-05' and extraction_date = '2014-03-05' allow filtering; {code} However, it will refuse to run unless ALLOW FILTERING is specified. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-6937) On large insert, node is being marked down with Commit Log failure
[ https://issues.apache.org/jira/browse/CASSANDRA-6937?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-6937. --- Resolution: Cannot Reproduce On large insert, node is being marked down with Commit Log failure -- Key: CASSANDRA-6937 URL: https://issues.apache.org/jira/browse/CASSANDRA-6937 Project: Cassandra Issue Type: Bug Environment: 16 vcpu, 480GB SSD storage (used 7GB), 32GB RAM, Cassandra 2.0.5.24, hsha, datastax Java driver 2.0.1, vnodes Reporter: jonathan lacefield Assignee: Benedict Priority: Minor Attachments: cassandra.yaml copy, system.log during large insert, will get specs on insert load, nodes in the cluster are being marked down. creating defect for investigation. error message in log file of one of the down nodes: ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:06,666 StorageService.java (line 364) Stopping gossiper WARN [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:06,666 StorageService.java (line 278) Stopping gossip by operator request INFO [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:06,666 Gossiper.java (line 1251) Announcing shutdown INFO [FlushWriter:421] 2014-03-26 08:28:06,677 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data1/system/compactions_in_progress/system-compactions_in_progress-jb-198-Data.db (166 bytes) for commitlog position ReplayPosition(segmentId=1395644152684, position=9577531) INFO [CompactionExecutor:1880] 2014-03-26 08:28:06,679 CompactionTask.java (line 115) Compacting [SSTableReader(path='/var/lib/cassandra/data2/OpsCenter/rollups60/OpsCenter-rollups60-jb-18-Data.db'), SSTableReader(path='/var/lib/cassandra/data1/OpsCenter/rollups60/OpsCenter-rollups60-jb-19-Data.db'), SSTableReader(path='/var/lib/cassandra/data2/OpsCenter/rollups60/OpsCenter-rollups60-jb-17-Data.db'), SSTableReader(path='/var/lib/cassandra/data1/OpsCenter/rollups60/OpsCenter-rollups60-jb-20-Data.db')] INFO [OptionalTasks:1] 2014-03-26 08:28:07,322 MeteredFlusher.java (line 63) flushing high-traffic column family CFS(Keyspace='system', ColumnFamily='batchlog') (estimated 215899526 bytes) INFO [OptionalTasks:1] 2014-03-26 08:28:07,322 ColumnFamilyStore.java (line 784) Enqueuing flush of Memtable-batchlog@74009414(39242953/215899526 serialized/live bytes, 239 ops) INFO [FlushWriter:422] 2014-03-26 08:28:07,323 Memtable.java (line 331) Writing Memtable-batchlog@74009414(39242953/215899526 serialized/live bytes, 239 ops) INFO [FlushWriter:422] 2014-03-26 08:28:07,370 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data1/system/batchlog/system-batchlog-jb-1-Data.db (204617 bytes) for commitlog position ReplayPosition(segmentId=1395644152684, position=17705519) INFO [MemoryMeter:1] 2014-03-26 08:28:08,296 Memtable.java (line 449) CFS(Keyspace='system', ColumnFamily='batchlog') liveRatio is 3.2515027780484544 (just-counted was 1.0013929201634204). calculation took 0ms for 12 cells ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,667 StorageService.java (line 369) Stopping RPC server INFO [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,667 ThriftServer.java (line 141) Stop listening to thrift clients ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,698 StorageService.java (line 374) Stopping native transport INFO [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,765 Server.java (line 181) Stop listening for CQL clients ERROR [COMMIT-LOG-ALLOCATOR] 2014-03-26 08:28:08,766 CommitLog.java (line 379) Failed to allocate new commit log segments. Commit disk failure policy is stop; terminating thread java.lang.AssertionError: attempted to delete non-existing file CommitLog-3-1395644152652.log -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000
[ https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956578#comment-13956578 ] Jonathan Ellis commented on CASSANDRA-6106: --- It's the code that [grabs] the addresses of gtod.wall_time_sec and gtod.wall_time_nsec that I don't know how to do with JNA. QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000 Key: CASSANDRA-6106 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106 Project: Cassandra Issue Type: Improvement Components: Core Environment: DSE Cassandra 3.1, but also HEAD Reporter: Christopher Smith Assignee: Benedict Priority: Minor Labels: timestamps Fix For: 2.1 beta2 Attachments: microtimstamp.patch, microtimstamp_random.patch, microtimstamp_random_rev2.patch I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra mentioned issues with millisecond rounding in timestamps and was able to reproduce the issue. If I specify a timestamp in a mutating query, I get microsecond precision, but if I don't, I get timestamps rounded to the nearest millisecond, at least for my first query on a given connection, which substantially increases the possibilities of collision. I believe I found the offending code, though I am by no means sure this is comprehensive. I think we probably need a fairly comprehensive replacement of all uses of System.currentTimeMillis() with System.nanoTime(). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6477) Partitioned indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-6477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13955260#comment-13955260 ] Jonathan Ellis edited comment on CASSANDRA-6477 at 4/1/14 2:52 PM: --- Sylvain had a different idea: Instead of just writing a {{24, user1}} tombstone, write a tombstone that indicates what the value changed to: {{24, user1 - 25}} for one thread, and {{24, user1 - 26}} for the other. When the tombstones are merged for compaction or read you can say, Wait! 2 people tried to erase that, one with 25 the other with 26, let's check which one has a higher timestamp and delete any obsolete entries. Note that this requires reading the existing data row to get the old indexed value, but in exchange for introducing read-before-write we also get to add the desired denormalization into the index and no longer have to check the original data row for reach read. was (Author: jbellis): Sylvain had a different idea: Instead of just writing a {{24, user1}} tombstone, write a tombstone that indicates what the value changed to: {{24, user1 - 25}} for one thread, and {{24, user1 - 26}} for the other. When the tombstones are merged for compaction or read you can say, Wait! 2 people tried to erase that, one with 25 the other with 26, let's check which one has a higher timestamp and delete any obsolete entries. Partitioned indexes --- Key: CASSANDRA-6477 URL: https://issues.apache.org/jira/browse/CASSANDRA-6477 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Fix For: 3.0 Local indexes are suitable for low-cardinality data, where spreading the index across the cluster is a Good Thing. However, for high-cardinality data, local indexes require querying most nodes in the cluster even if only a handful of rows is returned. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6927) Create a CQL3 based bulk OutputFormat
[ https://issues.apache.org/jira/browse/CASSANDRA-6927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-6927: -- Reviewer: Piotr Kołaczkowski (tagging [~pkolaczk] for review) Create a CQL3 based bulk OutputFormat - Key: CASSANDRA-6927 URL: https://issues.apache.org/jira/browse/CASSANDRA-6927 Project: Cassandra Issue Type: New Feature Components: Hadoop Reporter: Paul Pak Priority: Minor Labels: cql3, hadoop Attachments: trunk-6927.txt This is the CQL compatible version of BulkOutputFormat. CqlOutputFormat exists, but doesn't write SSTables directly, similar to ColumnFamilyOutputFormat for thrift. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6893) Unintended update with conditional statement
[ https://issues.apache.org/jira/browse/CASSANDRA-6893?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6893: Reviewer: Aleksey Yeschenko (was: Tyler Hobbs) Unintended update with conditional statement Key: CASSANDRA-6893 URL: https://issues.apache.org/jira/browse/CASSANDRA-6893 Project: Cassandra Issue Type: Bug Environment: Ubuntu Precise 64bit / Cassandra 2.0.6 Reporter: Suguru Namura Assignee: Sylvain Lebresne Fix For: 2.0.7 Attachments: 6893.txt, ConcurrentCASUpdate.java After updated to 2.0.6, I have encountered the strange behavior of conditional updates. When I executed CQL like UPDATE test SET value = ? WHERE id = ? IF value = ? in concurrent, sometimes cassandra returns true even if value is not satisfied the condition. I have attached the program which reproduce this issue. The program works fine in cassandra 2.0.5. But it seems that resets values while execution in 2.0.6. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6966) Errors with Super Columns, mixup of 1.2 and 2.0
[ https://issues.apache.org/jira/browse/CASSANDRA-6966?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6966: Reviewer: Aleksey Yeschenko Errors with Super Columns, mixup of 1.2 and 2.0 --- Key: CASSANDRA-6966 URL: https://issues.apache.org/jira/browse/CASSANDRA-6966 Project: Cassandra Issue Type: Bug Reporter: Nicolas Lalevée Assignee: Sylvain Lebresne Fix For: 2.0.7 Attachments: 6966.txt On our test cluster, we tried a upgrade of Cassandra from 1.2.11 to 2.0.6. During the time we were running with 2 different versions of cassandra, there was errors in the logs: ERROR [WRITE-/10.10.0.41] 2014-03-19 11:23:27,523 OutboundTcpConnection.java (line 234) error writing to /10.10.0.41 java.lang.RuntimeException: Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first. at org.apache.cassandra.db.SuperColumns.sliceFilterToSC(SuperColumns.java:357) at org.apache.cassandra.db.SuperColumns.filterToSC(SuperColumns.java:258) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:192) at org.apache.cassandra.db.ReadCommandSerializer.serializedSize(ReadCommand.java:134) at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:116) at org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251) at org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203) at org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151) I confirm we do have old style super columns which were designed when cassandra was 1.0.x. Since in our test cluster the replication factor is 1, I can see errors on the client side, since 1 node among 2 was down. So I don't know for sure if this error in cassandra affected the client, the time frame is too short to be sure from the logs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6696) Drive replacement in JBOD can cause data to reappear.
[ https://issues.apache.org/jira/browse/CASSANDRA-6696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956657#comment-13956657 ] Marcus Eriksson commented on CASSANDRA-6696: bq. Why is it partitioner-dependent? if we own all tokens we need to know min and max tokens for the partitioner to be able to split them over the disks. javadoc added for splitRanges in the repo above Drive replacement in JBOD can cause data to reappear. -- Key: CASSANDRA-6696 URL: https://issues.apache.org/jira/browse/CASSANDRA-6696 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: Marcus Eriksson Fix For: 3.0 In JBOD, when someone gets a bad drive, the bad drive is replaced with a new empty one and repair is run. This can cause deleted data to come back in some cases. Also this is true for corrupt stables in which we delete the corrupt stable and run repair. Here is an example: Say we have 3 nodes A,B and C and RF=3 and GC grace=10days. row=sankalp col=sankalp is written 20 days back and successfully went to all three nodes. Then a delete/tombstone was written successfully for the same row column 15 days back. Since this tombstone is more than gc grace, it got compacted in Nodes A and B since it got compacted with the actual data. So there is no trace of this row column in node A and B. Now in node C, say the original data is in drive1 and tombstone is in drive2. Compaction has not yet reclaimed the data and tombstone. Drive2 becomes corrupt and was replaced with new empty drive. Due to the replacement, the tombstone in now gone and row=sankalp col=sankalp has come back to life. Now after replacing the drive we run repair. This data will be propagated to all nodes. Note: This is still a problem even if we run repair every gc grace. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6825) COUNT(*) with WHERE not finding all the matching rows
[ https://issues.apache.org/jira/browse/CASSANDRA-6825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956669#comment-13956669 ] Sylvain Lebresne commented on CASSANDRA-6825: - [~thobbs] Any insights on what it off in the logic exactly? COUNT(*) with WHERE not finding all the matching rows - Key: CASSANDRA-6825 URL: https://issues.apache.org/jira/browse/CASSANDRA-6825 Project: Cassandra Issue Type: Bug Components: Core Environment: quad core Windows7 x64, single node cluster Cassandra 2.0.5 Reporter: Bill Mitchell Assignee: Tyler Hobbs Fix For: 2.0.7, 2.1 beta2 Attachments: cassandra.log, selectpartitions.zip, selectrowcounts.txt, testdb_1395372407904.zip, testdb_1395372407904.zip Investigating another problem, I needed to do COUNT(*) on the several partitions of a table immediately after a test case ran, and I discovered that count(*) on the full table and on each of the partitions returned different counts. In particular case, SELECT COUNT(*) FROM sr LIMIT 100; returned the expected count from the test 9 rows. The composite primary key splits the logical row into six distinct partitions, and when I issue a query asking for the total across all six partitions, the returned result is only 83999. Drilling down, I find that SELECT * from sr WHERE s = 5 AND l = 11 AND partition = 0; returns 30,000 rows, but a SELECT COUNT(*) with the identical WHERE predicate reports only 14,000. This is failing immediately after running a single small test, such that there are only two SSTables, sr-jb-1 and sr-jb-2. Compaction never needed to run. In selectrowcounts.txt is a copy of the cqlsh output showing the incorrect count(*) results. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6477) Partitioned indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-6477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956701#comment-13956701 ] Benedict commented on CASSANDRA-6477: - New suggestion: Since we're performing read-before-write anyway with this suggestion, why not simply perform a _local only_ read-before-write on each of the nodes that owns the main record whilst writing the update - instead of issuing a complex tombstone, we simply issue a delete for whichever value is older on reconcile. Since we always CAS local updates, we will never get missed deletes, however we will issue redundant/duplicate deletes (RF many) - but they should be coalesced in memtable almost always, so it's a network cost only. There are probably tricks we can do to mitigate this cost, though, e.g. having each node (deterministically) pick two of the possible owners of the 2i entry to send the deletes it encounters to, to minimise replication of effort but also ensure message delivery to all nodes. Result is we keep compaction logic exactly the same, and we retain approximately the same consistency guarantees we currently have. Partitioned indexes --- Key: CASSANDRA-6477 URL: https://issues.apache.org/jira/browse/CASSANDRA-6477 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Fix For: 3.0 Local indexes are suitable for low-cardinality data, where spreading the index across the cluster is a Good Thing. However, for high-cardinality data, local indexes require querying most nodes in the cluster even if only a handful of rows is returned. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6957) testNewRepairedSSTable fails intermittently
[ https://issues.apache.org/jira/browse/CASSANDRA-6957?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956718#comment-13956718 ] Michael Shuler commented on CASSANDRA-6957: --- I'll see what I can work up to grab those logs testNewRepairedSSTable fails intermittently --- Key: CASSANDRA-6957 URL: https://issues.apache.org/jira/browse/CASSANDRA-6957 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 beta2 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6825) COUNT(*) with WHERE not finding all the matching rows
[ https://issues.apache.org/jira/browse/CASSANDRA-6825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956721#comment-13956721 ] Tyler Hobbs commented on CASSANDRA-6825: [~slebresne] the logic is primarily broken because it continues checking latter components after it knows that the first component intersects. For example, suppose you have a slice of {{((1, 1), )}}, min column names of {{(0, 2)}}, and max column names of {{(2, 3)}}. The first component of the slice start falls within the min/max range; the second component does not. Although the slice is _starting_ outside of the min/max range for the second component, it should be considered intersecting because we'll accept other values for the second component (for higher values of the first component). The current logic sees that the second component doesn't fall within min/max and considers it non-intersecting. COUNT(*) with WHERE not finding all the matching rows - Key: CASSANDRA-6825 URL: https://issues.apache.org/jira/browse/CASSANDRA-6825 Project: Cassandra Issue Type: Bug Components: Core Environment: quad core Windows7 x64, single node cluster Cassandra 2.0.5 Reporter: Bill Mitchell Assignee: Tyler Hobbs Fix For: 2.0.7, 2.1 beta2 Attachments: cassandra.log, selectpartitions.zip, selectrowcounts.txt, testdb_1395372407904.zip, testdb_1395372407904.zip Investigating another problem, I needed to do COUNT(*) on the several partitions of a table immediately after a test case ran, and I discovered that count(*) on the full table and on each of the partitions returned different counts. In particular case, SELECT COUNT(*) FROM sr LIMIT 100; returned the expected count from the test 9 rows. The composite primary key splits the logical row into six distinct partitions, and when I issue a query asking for the total across all six partitions, the returned result is only 83999. Drilling down, I find that SELECT * from sr WHERE s = 5 AND l = 11 AND partition = 0; returns 30,000 rows, but a SELECT COUNT(*) with the identical WHERE predicate reports only 14,000. This is failing immediately after running a single small test, such that there are only two SSTables, sr-jb-1 and sr-jb-2. Compaction never needed to run. In selectrowcounts.txt is a copy of the cqlsh output showing the incorrect count(*) results. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6968) Reduce Unit Test Times Due to Schema Loading
Tyler Hobbs created CASSANDRA-6968: -- Summary: Reduce Unit Test Times Due to Schema Loading Key: CASSANDRA-6968 URL: https://issues.apache.org/jira/browse/CASSANDRA-6968 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Unit tests which extend SchemaLoader take about 6s longer to run than the others, on average. We could greatly reduce the time it takes to run the tests by improving this. None of the tests require everything that SchemaLoader does. We should change SchemaLoader into a set of test utilities that are run as needed in {{\@BeforeClass}} and {{\@AfterClass}} methods. Additionally, instead of running a full cleanup, most tests could simply use a keyspace or column family with a unique name (class/method name or perhaps class/method name + timestamp). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
Tyler Hobbs created CASSANDRA-6969: -- Summary: Use Unsafe Mutations Where Possible in Unit Tests Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
[ https://issues.apache.org/jira/browse/CASSANDRA-6969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956833#comment-13956833 ] Benedict edited comment on CASSANDRA-6969 at 4/1/14 6:10 PM: - If this is the plan, why not just convert to periodic CL? I thought the reason we used Batch CL in unit tests was to give it a bit of a run in case it happens to catch something? (Admittedly not the most scientific of reasons, but if we're planning on disabling it everywhere, it does seem like using Batch doesn't buy us much) was (Author: benedict): If this is the plan, why not just convert to periodic CL? I thought the reason we used Batch CL in unit tests was to give it a bit of a run in case it happens to catch something? Use Unsafe Mutations Where Possible in Unit Tests - Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
[ https://issues.apache.org/jira/browse/CASSANDRA-6969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956833#comment-13956833 ] Benedict commented on CASSANDRA-6969: - If this is the plan, why not just convert to periodic CL? I thought the reason we used Batch CL in unit tests was to give it a bit of a run in case it happens to catch something? Use Unsafe Mutations Where Possible in Unit Tests - Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6855) Native protocol V3
[ https://issues.apache.org/jira/browse/CASSANDRA-6855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956919#comment-13956919 ] Tyler Hobbs commented on CASSANDRA-6855: I haven't gotten to review this in depth, but one thing I'd like to add support for is warnings. These could be a field on RESULT messages (and possibly others). Mainly we need these for deprecation warnings, but I'm sure there are other cases where these would be handy. Native protocol V3 -- Key: CASSANDRA-6855 URL: https://issues.apache.org/jira/browse/CASSANDRA-6855 Project: Cassandra Issue Type: New Feature Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Fix For: 2.1 beta2 I think we need a V3 of the protocol for 2.1. The things that this could/should includes are: # Adding an optional Serial CL for protocol batches (like we have for QUERY and EXECUTE). It was an oversight of V2 of not adding it, and now that we can batch conditional updates, it's definitively missing. # Proper type codes for UDT. This is not *strictly* needed to be able to support UDT since currently a UDT will be sent as a custom type with his fully class name + arguments. But parsing that is no fun nor convenient for clients. It's also not particular space efficient (though that's probably not a huge concern since with prepared statement you can avoid sending the ResultSet metadata every time). # Serialization format for collections. Currently the serialization format only allow for 65K elements, each of 65K bytes size at most. While collections are not meant to store large amount of data, having the limitation in the protocol serialization format is the wrong way to deal with that. Concretely, the current workaround for CASSANDRA-5428 is ugly. I'll note that the current serialization format is also an obstacle to supporting null inside collections (whether or not we want to support null there is a good question, but here again I'm not sure being limited by the serialization format is a good idea). # CASSANDRA-6178: I continue to believe that in many case it makes somewhat more sense to have the default timestamp provided by the client (this is a necessary condition for true idempotent retries in particular). I'm absolutely fine making that optional and leaving server-side generated timestamps by default, but since client can already provide timestamp in query string anyway, I don't see a big deal in making it easier for client driver to control that without messing with the query string. # Optional names for values in QUERY messages: it has been brought to my attention that while V2 allows to send a query string with values for a one-roundtrip bind-and-execute, a driver can't really support named bind marker with that feature properly without parsing the query. The proposition is thus to make it (optionally) possible to ship the name of the marker each value is supposed to be bound to. I think that 1) and 2) are enough reason to make a V3 (even if there is disagreement on the rest that is). 3) is a little bit more involved tbh but I do think having the current limitations bolted in the protocol serialization format is wrong in the long run, and it turns out that due to UDT we will start storing serialized collections internally so if we want to lift said limitation in the serialization format, we should do it now and everywhere, as doing it afterwards will be a lot more painful. 4) and 5) are probably somewhat more minor, but at the same time, both are completely optional (a driver won't have to support those if he doesn't want). They are really just about making things more flexible for client drivers and they are not particularly hard to support so I don't see too many reasons not to include them. Last but not least, I know that some may find it wrong to do a new protocol version with each major of C*, so let me state my view here: I fully agree that we shouldn't make an habit of that in the long run and that's definitively *not* my objective. However, it would be silly to expect that we could get everything right and forget nothing in the very first version. It shouldn't be surprising that we'll have to burn a few versions (and there might be a few more yet) before getting something more stable and complete and I think that delaying the addition of stuffs that are useful to create some fake notion of stability would be even more silly. On the bright side, the additions of this V3 are comparatively much more simple to implement for a client that those of V2 (in fact, for clients that want to support UDT, it will probably require less effort to add the changes for this new version than to try to support UDT without it), so I do think we make
[jira] [Commented] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000
[ https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956938#comment-13956938 ] Benedict commented on CASSANDRA-6106: - It doesn't look safe to me to simply grab gtod.wall_time_sec anyway, even if we could find its location, as the nanos value gets repaired after reading with another call. We could investigate further, but for the time being I have a reasonably straightforward solution [here|github.com/belliottsmith/cassandra/tree/6106-microstime] I started by simply calling the rt clock_gettime method through JNA, which unfortunately clocks in at a heavy 7 micros; since nanoTime and currentTimeMillis are 0.03 micros, this seemed a little unacceptable. So what I've done is opted to periodically (once per second) grab the latest micros time via the best method possible (clock_gettime if available, currentTimeMillis * 1000 otherwise) and use this to reset the offset, however to ensure we have a smooth transition I: # Cap the rate of change at 50ms per second # Ensure it never leaps back in time, at least on any given thread (no way to guarantee stronger than this) # Only apply a change if it is at least 1ms out, to avoid noise (possibly should tighten this to 100 micros, or dependent on resolution of time library we're using) The result is a method that costs around the same as a raw call to System.nanoTime() but gives pretty decent accuracy. Obviously any method that involves using nanos and calculating an offset from a method that takes ~7micros to return is going to have an inherent inaccuracy, but no more than the 7micros direct method call would itself, and the inaccuracy will be consistent given the jitter reduction I'm applying. At startup we also sample the offset 10k times, derive a 90%ile for elapsed time fetching the offset (we ignore future offsets we calculate that take more than twice this period to sample) and average all of those within the 90%ile. QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000 Key: CASSANDRA-6106 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106 Project: Cassandra Issue Type: Improvement Components: Core Environment: DSE Cassandra 3.1, but also HEAD Reporter: Christopher Smith Assignee: Benedict Priority: Minor Labels: timestamps Fix For: 2.1 beta2 Attachments: microtimstamp.patch, microtimstamp_random.patch, microtimstamp_random_rev2.patch I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra mentioned issues with millisecond rounding in timestamps and was able to reproduce the issue. If I specify a timestamp in a mutating query, I get microsecond precision, but if I don't, I get timestamps rounded to the nearest millisecond, at least for my first query on a given connection, which substantially increases the possibilities of collision. I believe I found the offending code, though I am by no means sure this is comprehensive. I think we probably need a fairly comprehensive replacement of all uses of System.currentTimeMillis() with System.nanoTime(). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000
[ https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956938#comment-13956938 ] Benedict edited comment on CASSANDRA-6106 at 4/1/14 7:34 PM: - It doesn't look safe to me to simply grab gtod.wall_time_sec anyway, even if we could find its location, as the nanos value gets repaired after reading with another call. We could investigate further, but for the time being I have a reasonably straightforward solution [here|http://github.com/belliottsmith/cassandra/tree/6106-microstime] I started by simply calling the rt clock_gettime method through JNA, which unfortunately clocks in at a heavy 7 micros; since nanoTime and currentTimeMillis are 0.03 micros, this seemed a little unacceptable. So what I've done is opted to periodically (once per second) grab the latest micros time via the best method possible (clock_gettime if available, currentTimeMillis * 1000 otherwise) and use this to reset the offset, however to ensure we have a smooth transition I: # Cap the rate of change at 50ms per second # Ensure it never leaps back in time, at least on any given thread (no way to guarantee stronger than this) # Only apply a change if it is at least 1ms out, to avoid noise (possibly should tighten this to 100 micros, or dependent on resolution of time library we're using) The result is a method that costs around the same as a raw call to System.nanoTime() but gives pretty decent accuracy. Obviously any method that involves using nanos and calculating an offset from a method that takes ~7micros to return is going to have an inherent inaccuracy, but no more than the 7micros direct method call would itself, and the inaccuracy will be consistent given the jitter reduction I'm applying. At startup we also sample the offset 10k times, derive a 90%ile for elapsed time fetching the offset (we ignore future offsets we calculate that take more than twice this period to sample) and average all of those within the 90%ile. was (Author: benedict): It doesn't look safe to me to simply grab gtod.wall_time_sec anyway, even if we could find its location, as the nanos value gets repaired after reading with another call. We could investigate further, but for the time being I have a reasonably straightforward solution [here|github.com/belliottsmith/cassandra/tree/6106-microstime] I started by simply calling the rt clock_gettime method through JNA, which unfortunately clocks in at a heavy 7 micros; since nanoTime and currentTimeMillis are 0.03 micros, this seemed a little unacceptable. So what I've done is opted to periodically (once per second) grab the latest micros time via the best method possible (clock_gettime if available, currentTimeMillis * 1000 otherwise) and use this to reset the offset, however to ensure we have a smooth transition I: # Cap the rate of change at 50ms per second # Ensure it never leaps back in time, at least on any given thread (no way to guarantee stronger than this) # Only apply a change if it is at least 1ms out, to avoid noise (possibly should tighten this to 100 micros, or dependent on resolution of time library we're using) The result is a method that costs around the same as a raw call to System.nanoTime() but gives pretty decent accuracy. Obviously any method that involves using nanos and calculating an offset from a method that takes ~7micros to return is going to have an inherent inaccuracy, but no more than the 7micros direct method call would itself, and the inaccuracy will be consistent given the jitter reduction I'm applying. At startup we also sample the offset 10k times, derive a 90%ile for elapsed time fetching the offset (we ignore future offsets we calculate that take more than twice this period to sample) and average all of those within the 90%ile. QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000 Key: CASSANDRA-6106 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106 Project: Cassandra Issue Type: Improvement Components: Core Environment: DSE Cassandra 3.1, but also HEAD Reporter: Christopher Smith Assignee: Benedict Priority: Minor Labels: timestamps Fix For: 2.1 beta2 Attachments: microtimstamp.patch, microtimstamp_random.patch, microtimstamp_random_rev2.patch I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra mentioned issues with millisecond rounding in timestamps and was able to reproduce the issue. If I specify a timestamp in
[jira] [Updated] (CASSANDRA-6841) ConcurrentModificationException in commit-log-writer after local schema reset
[ https://issues.apache.org/jira/browse/CASSANDRA-6841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] J.B. Langston updated CASSANDRA-6841: - Fix Version/s: 1.2.17 ConcurrentModificationException in commit-log-writer after local schema reset - Key: CASSANDRA-6841 URL: https://issues.apache.org/jira/browse/CASSANDRA-6841 Project: Cassandra Issue Type: Bug Environment: Linux 3.2.0 (Debian Wheezy) Cassandra 2.0.6, Oracle JVM 1.7.0_51 Almost default cassandra.yaml (IPs and cluster name changed) This is the 2nd node in a 2-node ring. It has ~2500 keyspaces and very low traffic. (Only new keyspaces see reads and writes.) Reporter: Pas Assignee: Benedict Priority: Minor Fix For: 1.2.17, 2.0.7, 2.1 beta2 {code} INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,013 MigrationManager.java (line 329) Starting local schema reset... INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,016 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@394448776(114/1140 serialized/live bytes, 3 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,016 Memtable.java (line 331) Writing Memtable-local@394448776(114/1140 serialized/live bytes, 3 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,182 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-398-Data.db (145 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33159822) INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,185 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@1087210140(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,185 Memtable.java (line 331) Writing Memtable-local@1087210140(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,357 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-399-Data.db (96 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33159959) INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,361 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@768887091(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,361 Memtable.java (line 331) Writing Memtable-local@768887091(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,516 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-400-Data.db (96 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33160096) INFO [CompactionExecutor:38] 2014-03-12 11:37:54,517 CompactionTask.java (line 115) Compacting [SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-398-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-400-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-399-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-397-Data.db')] INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,519 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@271993477(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,519 Memtable.java (line 331) Writing Memtable-local@271993477(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,794 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-401-Data.db (96 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33160233) INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,799 MigrationManager.java (line 357) Local schema reset is complete. INFO [CompactionExecutor:38] 2014-03-12 11:37:54,848 CompactionTask.java (line 275) Compacted 4 sstables to [/var/lib/cassandra/data/system/local/system-local-jb-402,]. 6,099 bytes to 5,821 (~95% of original) in 330ms = 0.016822MB/s. 4 total partitions merged to 1. Partition merge counts were {4:1, } INFO [OptionalTasks:1] 2014-03-12 11:37:55,110 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-schema_columnfamilies@106276050(181506/509164 serialized/live bytes, 3276 ops) INFO [FlushWriter:6] 2014-03-12 11:37:55,110 Memtable.java (line 331) Writing Memtable-schema_columnfamilies@106276050(181506/509164 serialized/live bytes, 3276 ops) INFO [OptionalTasks:1] 2014-03-12 11:37:55,110 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-schema_columns@252242773(185191/630698 serialized/live bytes, 3614 ops) ERROR [COMMIT-LOG-WRITER]
[jira] [Reopened] (CASSANDRA-6841) ConcurrentModificationException in commit-log-writer after local schema reset
[ https://issues.apache.org/jira/browse/CASSANDRA-6841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] J.B. Langston reopened CASSANDRA-6841: -- Reopening to get a backport for 1.2. ConcurrentModificationException in commit-log-writer after local schema reset - Key: CASSANDRA-6841 URL: https://issues.apache.org/jira/browse/CASSANDRA-6841 Project: Cassandra Issue Type: Bug Environment: Linux 3.2.0 (Debian Wheezy) Cassandra 2.0.6, Oracle JVM 1.7.0_51 Almost default cassandra.yaml (IPs and cluster name changed) This is the 2nd node in a 2-node ring. It has ~2500 keyspaces and very low traffic. (Only new keyspaces see reads and writes.) Reporter: Pas Assignee: Benedict Priority: Minor Fix For: 1.2.17, 2.0.7, 2.1 beta2 {code} INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,013 MigrationManager.java (line 329) Starting local schema reset... INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,016 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@394448776(114/1140 serialized/live bytes, 3 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,016 Memtable.java (line 331) Writing Memtable-local@394448776(114/1140 serialized/live bytes, 3 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,182 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-398-Data.db (145 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33159822) INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,185 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@1087210140(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,185 Memtable.java (line 331) Writing Memtable-local@1087210140(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,357 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-399-Data.db (96 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33159959) INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,361 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@768887091(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,361 Memtable.java (line 331) Writing Memtable-local@768887091(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,516 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-400-Data.db (96 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33160096) INFO [CompactionExecutor:38] 2014-03-12 11:37:54,517 CompactionTask.java (line 115) Compacting [SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-398-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-400-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-399-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-jb-397-Data.db')] INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,519 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-local@271993477(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,519 Memtable.java (line 331) Writing Memtable-local@271993477(62/620 serialized/live bytes, 1 ops) INFO [FlushWriter:6] 2014-03-12 11:37:54,794 Memtable.java (line 371) Completed flushing /var/lib/cassandra/data/system/local/system-local-jb-401-Data.db (96 bytes) for commitlog position ReplayPosition(segmentId=1394620057452, position=33160233) INFO [RMI TCP Connection(38)-192.168.36.171] 2014-03-12 11:37:54,799 MigrationManager.java (line 357) Local schema reset is complete. INFO [CompactionExecutor:38] 2014-03-12 11:37:54,848 CompactionTask.java (line 275) Compacted 4 sstables to [/var/lib/cassandra/data/system/local/system-local-jb-402,]. 6,099 bytes to 5,821 (~95% of original) in 330ms = 0.016822MB/s. 4 total partitions merged to 1. Partition merge counts were {4:1, } INFO [OptionalTasks:1] 2014-03-12 11:37:55,110 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-schema_columnfamilies@106276050(181506/509164 serialized/live bytes, 3276 ops) INFO [FlushWriter:6] 2014-03-12 11:37:55,110 Memtable.java (line 331) Writing Memtable-schema_columnfamilies@106276050(181506/509164 serialized/live bytes, 3276 ops) INFO [OptionalTasks:1] 2014-03-12 11:37:55,110 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-schema_columns@252242773(185191/630698 serialized/live bytes, 3614 ops) ERROR
[jira] [Updated] (CASSANDRA-6961) nodes should go into hibernate when join_ring is false
[ https://issues.apache.org/jira/browse/CASSANDRA-6961?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-6961: Attachment: 6961.txt Patch to enable this. We only advertise tokens if some were already saved, so you can still start with join_ring=false, and then use join to bootstrap later (I don't know why anyone would do that, but that's the behavior we had before.) You can run repair on the node while it's in hibernate, and as a bonus side effect, if you take a blank node and set join_ring=false but don't disable rpc, you have an instant coordinator-only fat client (where before if you did this, you were asking for trouble.) nodes should go into hibernate when join_ring is false -- Key: CASSANDRA-6961 URL: https://issues.apache.org/jira/browse/CASSANDRA-6961 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 2.0.8 Attachments: 6961.txt The impetus here is this: a node that was down for some period and comes back can serve stale information. We know from CASSANDRA-768 that we can't just wait for hints, and know that tangentially related CASSANDRA-3569 prevents us from having the node in a down (from the FD's POV) state handle streaming. We can *almost* set join_ring to false, then repair, and then join the ring to narrow the window (actually, you can do this and everything succeeds because the node doesn't know it's a member yet, which is probably a bit of a bug.) If instead we modified this to put the node in hibernate, like replace_address does, it could work almost like replace, except you could run a repair (manually) while in the hibernate state, and then flip to normal when it's done. This won't prevent the staleness 100%, but it will greatly reduce the chance if the node has been down a significant amount of time. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6961) nodes should go into hibernate when join_ring is false
[ https://issues.apache.org/jira/browse/CASSANDRA-6961?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-6961: Reviewer: Tyler Hobbs nodes should go into hibernate when join_ring is false -- Key: CASSANDRA-6961 URL: https://issues.apache.org/jira/browse/CASSANDRA-6961 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 2.0.8 Attachments: 6961.txt The impetus here is this: a node that was down for some period and comes back can serve stale information. We know from CASSANDRA-768 that we can't just wait for hints, and know that tangentially related CASSANDRA-3569 prevents us from having the node in a down (from the FD's POV) state handle streaming. We can *almost* set join_ring to false, then repair, and then join the ring to narrow the window (actually, you can do this and everything succeeds because the node doesn't know it's a member yet, which is probably a bit of a bug.) If instead we modified this to put the node in hibernate, like replace_address does, it could work almost like replace, except you could run a repair (manually) while in the hibernate state, and then flip to normal when it's done. This won't prevent the staleness 100%, but it will greatly reduce the chance if the node has been down a significant amount of time. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6961) nodes should go into hibernate when join_ring is false
[ https://issues.apache.org/jira/browse/CASSANDRA-6961?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-6961: Tester: Ryan McGuire nodes should go into hibernate when join_ring is false -- Key: CASSANDRA-6961 URL: https://issues.apache.org/jira/browse/CASSANDRA-6961 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 2.0.8 Attachments: 6961.txt The impetus here is this: a node that was down for some period and comes back can serve stale information. We know from CASSANDRA-768 that we can't just wait for hints, and know that tangentially related CASSANDRA-3569 prevents us from having the node in a down (from the FD's POV) state handle streaming. We can *almost* set join_ring to false, then repair, and then join the ring to narrow the window (actually, you can do this and everything succeeds because the node doesn't know it's a member yet, which is probably a bit of a bug.) If instead we modified this to put the node in hibernate, like replace_address does, it could work almost like replace, except you could run a repair (manually) while in the hibernate state, and then flip to normal when it's done. This won't prevent the staleness 100%, but it will greatly reduce the chance if the node has been down a significant amount of time. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6948) After Bootstrap or Replace node startup, EXPIRING_MAP_REAPER is shutdown and cannot be restarted, causing callbacks to collect indefinitely
[ https://issues.apache.org/jira/browse/CASSANDRA-6948?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-6948: Fix Version/s: 2.0.7 After Bootstrap or Replace node startup, EXPIRING_MAP_REAPER is shutdown and cannot be restarted, causing callbacks to collect indefinitely --- Key: CASSANDRA-6948 URL: https://issues.apache.org/jira/browse/CASSANDRA-6948 Project: Cassandra Issue Type: Bug Components: Core Reporter: Keith Wright Assignee: Benedict Fix For: 2.0.7 Attachments: 6948.debug.txt, 6948.txt, Screen Shot 2014-03-28 at 11.27.56 AM.png, Screen Shot 2014-03-28 at 11.29.24 AM.png, cassandra.log.min, cassandra.yaml, logs.old.tar.gz, logs.tar.gz, system.log.1.gz, system.log.gz Since ExpiringMap.shutdown() shuts down the static executor service, it cannot be restarted (and in fact reset() makes no attempt to do so). As such callbacks that receive no response are never removed from the map, and eventually either than server will run out of memory or will loop around the integer space and start reusing messageids that have not been expired, causing assertions to be thrown and messages to fail to be sent. It appears that this situation only arises on bootstrap or node replacement, as MessagingService is shutdown before being attached to the listen address. This can cause the following errors to begin occurring in the log: ERROR [Native-Transport-Requests:7636] 2014-03-28 13:32:10,638 ErrorMessage.java (line 222) Unexpected exception during request java.lang.AssertionError: Callback already exists for id -1665979622! (CallbackInfo(target=/10.106.160.84, callback=org.apache.cassandra.service.WriteResponseHandler@5d36d8ea, serializer=org.apache.cassandra.db.WriteResponse$WriteResponseSerializer@6ed37f0b)) at org.apache.cassandra.net.MessagingService.addCallback(MessagingService.java:549) at org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:601) at org.apache.cassandra.service.StorageProxy.mutateCounter(StorageProxy.java:984) at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:449) at org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:524) at org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:521) at org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:505) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:188) at org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:358) at org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:131) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:304) at org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43) at org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) ERROR [ReplicateOnWriteStage:102766] 2014-03-28 13:32:10,638 CassandraDaemon.java (line 196) Exception in thread Thread[ReplicateOnWriteStage:102766,5,main] java.lang.AssertionError: Callback already exists for id -1665979620! (CallbackInfo(target=/10.106.160.84, callback=org.apache.cassandra.service.WriteResponseHandler@3bdb1a75, serializer=org.apache.cassandra.db.WriteResponse$WriteResponseSerializer@6ed37f0b)) at org.apache.cassandra.net.MessagingService.addCallback(MessagingService.java:549) at org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:601) at org.apache.cassandra.service.StorageProxy.sendToHintedEndpoints(StorageProxy.java:806) at org.apache.cassandra.service.StorageProxy$8$1.runMayThrow(StorageProxy.java:1074) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1896) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6825) Slice Queries Can Skip Intersecting SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-6825?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-6825: --- Summary: Slice Queries Can Skip Intersecting SSTables (was: COUNT(*) with WHERE not finding all the matching rows) Slice Queries Can Skip Intersecting SSTables Key: CASSANDRA-6825 URL: https://issues.apache.org/jira/browse/CASSANDRA-6825 Project: Cassandra Issue Type: Bug Components: Core Environment: quad core Windows7 x64, single node cluster Cassandra 2.0.5 Reporter: Bill Mitchell Assignee: Tyler Hobbs Fix For: 2.0.7, 2.1 beta2 Attachments: cassandra.log, selectpartitions.zip, selectrowcounts.txt, testdb_1395372407904.zip, testdb_1395372407904.zip Investigating another problem, I needed to do COUNT(*) on the several partitions of a table immediately after a test case ran, and I discovered that count(*) on the full table and on each of the partitions returned different counts. In particular case, SELECT COUNT(*) FROM sr LIMIT 100; returned the expected count from the test 9 rows. The composite primary key splits the logical row into six distinct partitions, and when I issue a query asking for the total across all six partitions, the returned result is only 83999. Drilling down, I find that SELECT * from sr WHERE s = 5 AND l = 11 AND partition = 0; returns 30,000 rows, but a SELECT COUNT(*) with the identical WHERE predicate reports only 14,000. This is failing immediately after running a single small test, such that there are only two SSTables, sr-jb-1 and sr-jb-2. Compaction never needed to run. In selectrowcounts.txt is a copy of the cqlsh output showing the incorrect count(*) results. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6970) Support CL=EACH_QUORUM for reads
Oliver Seiler created CASSANDRA-6970: Summary: Support CL=EACH_QUORUM for reads Key: CASSANDRA-6970 URL: https://issues.apache.org/jira/browse/CASSANDRA-6970 Project: Cassandra Issue Type: Wish Environment: SLES 11 SP3 x86_64 Cassandra 2.0.6 Reporter: Oliver Seiler SELECTs done at CL=EACH_QUORUM get an InvalidRequestException: with a message of EACH_QUORUM ConsistencyLevel is only supported for writes. The documentation doesn't indicate this would happen, so at minimum this is inconsistent with the current documentation. I'm aware of CASSANDRA-3272, which indicates this has never worked as expected, and made this obvious with the InvalidRequestException. I'm not sure why this shouldn't work, regardless of Jonathan Ellis commenting I ask because I've never seen it to be the 'right' CL yet. I'd like it because: I want to do my writes at LOCAL_QUORUM and (some) reads at EACH_QUORUM, because I require guaranteed consistency for (some) reads where I can be partition intolerant without affecting clients, rather than becoming partition intolerant on writes... -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6825) Slice Queries Can Skip Intersecting SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-6825?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-6825: --- Attachment: 6825-2.0.txt The attached patch (and [branch|https://github.com/thobbs/cassandra/tree/CASSANDRA-6825]) fixes {{CompositeType.intersects()}} and adds unit tests. I also added some basic debug logging around query paging, since there wasn't anything there. Slice Queries Can Skip Intersecting SSTables Key: CASSANDRA-6825 URL: https://issues.apache.org/jira/browse/CASSANDRA-6825 Project: Cassandra Issue Type: Bug Components: Core Environment: quad core Windows7 x64, single node cluster Cassandra 2.0.5 Reporter: Bill Mitchell Assignee: Tyler Hobbs Fix For: 2.0.7, 2.1 beta2 Attachments: 6825-2.0.txt, cassandra.log, selectpartitions.zip, selectrowcounts.txt, testdb_1395372407904.zip, testdb_1395372407904.zip Investigating another problem, I needed to do COUNT(*) on the several partitions of a table immediately after a test case ran, and I discovered that count(*) on the full table and on each of the partitions returned different counts. In particular case, SELECT COUNT(*) FROM sr LIMIT 100; returned the expected count from the test 9 rows. The composite primary key splits the logical row into six distinct partitions, and when I issue a query asking for the total across all six partitions, the returned result is only 83999. Drilling down, I find that SELECT * from sr WHERE s = 5 AND l = 11 AND partition = 0; returns 30,000 rows, but a SELECT COUNT(*) with the identical WHERE predicate reports only 14,000. This is failing immediately after running a single small test, such that there are only two SSTables, sr-jb-1 and sr-jb-2. Compaction never needed to run. In selectrowcounts.txt is a copy of the cqlsh output showing the incorrect count(*) results. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6961) nodes should go into hibernate when join_ring is false
[ https://issues.apache.org/jira/browse/CASSANDRA-6961?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-6961: Fix Version/s: (was: 2.0.8) 2.0.7 nodes should go into hibernate when join_ring is false -- Key: CASSANDRA-6961 URL: https://issues.apache.org/jira/browse/CASSANDRA-6961 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 2.0.7 Attachments: 6961.txt The impetus here is this: a node that was down for some period and comes back can serve stale information. We know from CASSANDRA-768 that we can't just wait for hints, and know that tangentially related CASSANDRA-3569 prevents us from having the node in a down (from the FD's POV) state handle streaming. We can *almost* set join_ring to false, then repair, and then join the ring to narrow the window (actually, you can do this and everything succeeds because the node doesn't know it's a member yet, which is probably a bit of a bug.) If instead we modified this to put the node in hibernate, like replace_address does, it could work almost like replace, except you could run a repair (manually) while in the hibernate state, and then flip to normal when it's done. This won't prevent the staleness 100%, but it will greatly reduce the chance if the node has been down a significant amount of time. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
[ https://issues.apache.org/jira/browse/CASSANDRA-6969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956970#comment-13956970 ] Tyler Hobbs commented on CASSANDRA-6969: I'm not sure why we use batch CL for the tests. If you're correct and the purpose is to exercise batch CL, we should just make sure it's properly tested elsewhere. I'd argue that having slow unit tests is hurting our stability more than not using batch CL for unit tests would. Use Unsafe Mutations Where Possible in Unit Tests - Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6968) Reduce Unit Test Times Due to Schema Loading
[ https://issues.apache.org/jira/browse/CASSANDRA-6968?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-6968: -- Fix Version/s: 2.1 beta3 Reduce Unit Test Times Due to Schema Loading Key: CASSANDRA-6968 URL: https://issues.apache.org/jira/browse/CASSANDRA-6968 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Fix For: 2.1 beta3 Unit tests which extend SchemaLoader take about 6s longer to run than the others, on average. We could greatly reduce the time it takes to run the tests by improving this. None of the tests require everything that SchemaLoader does. We should change SchemaLoader into a set of test utilities that are run as needed in {{\@BeforeClass}} and {{\@AfterClass}} methods. Additionally, instead of running a full cleanup, most tests could simply use a keyspace or column family with a unique name (class/method name or perhaps class/method name + timestamp). -- This message was sent by Atlassian JIRA (v6.2#6252)
[1/2] git commit: Clearer message for assertion error
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 353b2147b - d6b582614 Clearer message for assertion error Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/55fba6ab Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/55fba6ab Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/55fba6ab Branch: refs/heads/cassandra-2.1 Commit: 55fba6ab0b5521711ed5c33a2d8ddadd2edd8906 Parents: d8c29a3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 12:09:27 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 12:09:27 2014 +0200 -- src/java/org/apache/cassandra/db/filter/ExtendedFilter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/55fba6ab/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java index 5aa1ea9..5c3662b 100644 --- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java @@ -255,7 +255,8 @@ public abstract class ExtendedFilter * 2) We don't yet allow non-indexed range slice with filters in CQL3 (i.e. this will never be * called by CFS.filter() for composites). */ -assert !(cfs.getComparator() instanceof CompositeType); +assert !(cfs.getComparator() instanceof CompositeType) : Sequential scan with filters is not supported (if you just created an index, you + + need to wait for the creation to be propagated to all nodes before querying it); if (!needsExtraQuery(rowKey.key, data)) return null;
[2/2] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: src/java/org/apache/cassandra/db/filter/ExtendedFilter.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d6b58261 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d6b58261 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d6b58261 Branch: refs/heads/cassandra-2.1 Commit: d6b582614f4f2c9cdd391623b340b2732311b98d Parents: 353b214 55fba6a Author: Aleksey Yeschenko alek...@apache.org Authored: Tue Apr 1 23:25:19 2014 +0300 Committer: Aleksey Yeschenko alek...@apache.org Committed: Tue Apr 1 23:25:19 2014 +0300 -- src/java/org/apache/cassandra/db/filter/ExtendedFilter.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d6b58261/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java -- diff --cc src/java/org/apache/cassandra/db/filter/ExtendedFilter.java index 7326d80,5c3662b..6bf25b1 --- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java @@@ -253,6 -255,9 +253,9 @@@ public abstract class ExtendedFilte * 2) We don't yet allow non-indexed range slice with filters in CQL3 (i.e. this will never be * called by CFS.filter() for composites). */ -assert !(cfs.getComparator() instanceof CompositeType) : Sequential scan with filters is not supported (if you just created an index, you - + need to wait for the creation to be propagated to all nodes before querying it); ++assert !(cfs.getComparator().isCompound()) : Sequential scan with filters is not supported (if you just created an index, you ++ + need to wait for the creation to be propagated to all nodes before querying it); + if (!needsExtraQuery(rowKey.key, data)) return null;
[2/3] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: src/java/org/apache/cassandra/db/filter/ExtendedFilter.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d6b58261 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d6b58261 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d6b58261 Branch: refs/heads/trunk Commit: d6b582614f4f2c9cdd391623b340b2732311b98d Parents: 353b214 55fba6a Author: Aleksey Yeschenko alek...@apache.org Authored: Tue Apr 1 23:25:19 2014 +0300 Committer: Aleksey Yeschenko alek...@apache.org Committed: Tue Apr 1 23:25:19 2014 +0300 -- src/java/org/apache/cassandra/db/filter/ExtendedFilter.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d6b58261/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java -- diff --cc src/java/org/apache/cassandra/db/filter/ExtendedFilter.java index 7326d80,5c3662b..6bf25b1 --- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java @@@ -253,6 -255,9 +253,9 @@@ public abstract class ExtendedFilte * 2) We don't yet allow non-indexed range slice with filters in CQL3 (i.e. this will never be * called by CFS.filter() for composites). */ -assert !(cfs.getComparator() instanceof CompositeType) : Sequential scan with filters is not supported (if you just created an index, you - + need to wait for the creation to be propagated to all nodes before querying it); ++assert !(cfs.getComparator().isCompound()) : Sequential scan with filters is not supported (if you just created an index, you ++ + need to wait for the creation to be propagated to all nodes before querying it); + if (!needsExtraQuery(rowKey.key, data)) return null;
[3/3] git commit: Merge branch 'cassandra-2.1' into trunk
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/f94a7af1 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f94a7af1 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f94a7af1 Branch: refs/heads/trunk Commit: f94a7af1737db714fac29330d1e0c0d67bd51c7d Parents: 6660970 d6b5826 Author: Aleksey Yeschenko alek...@apache.org Authored: Tue Apr 1 23:26:19 2014 +0300 Committer: Aleksey Yeschenko alek...@apache.org Committed: Tue Apr 1 23:26:19 2014 +0300 -- src/java/org/apache/cassandra/db/filter/ExtendedFilter.java | 3 +++ 1 file changed, 3 insertions(+) --
[1/3] git commit: Clearer message for assertion error
Repository: cassandra Updated Branches: refs/heads/trunk 6660970c3 - f94a7af17 Clearer message for assertion error Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/55fba6ab Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/55fba6ab Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/55fba6ab Branch: refs/heads/trunk Commit: 55fba6ab0b5521711ed5c33a2d8ddadd2edd8906 Parents: d8c29a3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Apr 1 12:09:27 2014 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Apr 1 12:09:27 2014 +0200 -- src/java/org/apache/cassandra/db/filter/ExtendedFilter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/55fba6ab/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java -- diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java index 5aa1ea9..5c3662b 100644 --- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java @@ -255,7 +255,8 @@ public abstract class ExtendedFilter * 2) We don't yet allow non-indexed range slice with filters in CQL3 (i.e. this will never be * called by CFS.filter() for composites). */ -assert !(cfs.getComparator() instanceof CompositeType); +assert !(cfs.getComparator() instanceof CompositeType) : Sequential scan with filters is not supported (if you just created an index, you + + need to wait for the creation to be propagated to all nodes before querying it); if (!needsExtraQuery(rowKey.key, data)) return null;
[jira] [Commented] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
[ https://issues.apache.org/jira/browse/CASSANDRA-6969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13956977#comment-13956977 ] Jonathan Ellis commented on CASSANDRA-6969: --- That was the idea -- not many people run Batch in the wild, so let's make sure it at least gets tested in the lab. Could we make a dtest profile using batch instead? Use Unsafe Mutations Where Possible in Unit Tests - Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
git commit: Don't break API in minor rev, followup to CASSANDRA-6958
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 55fba6ab0 - 80b3e6cc3 Don't break API in minor rev, followup to CASSANDRA-6958 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/80b3e6cc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/80b3e6cc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/80b3e6cc Branch: refs/heads/cassandra-2.0 Commit: 80b3e6cc3b1d6fdae91193563626f0823c92c528 Parents: 55fba6a Author: Marcus Eriksson marc...@apache.org Authored: Tue Apr 1 22:03:30 2014 +0200 Committer: Marcus Eriksson marc...@apache.org Committed: Tue Apr 1 22:16:10 2014 +0200 -- .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 - .../cassandra/db/compaction/SizeTieredCompactionStrategy.java | 6 -- 2 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index d3339d5..941d3d7 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -167,7 +167,10 @@ public abstract class AbstractCompactionStrategy */ public abstract AbstractCompactionTask getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore); -public abstract AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long maxSSTableBytes); +public AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long maxSSTableBytes) +{ +return new CompactionTask(cfs, sstables, gcBefore); +} /** * @return the number of background tasks estimated to still be needed for this columnfamilystore http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java index fae52e2..0f94918 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java @@ -271,12 +271,6 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy return new CompactionTask(cfs, sstables, gcBefore).setUserDefined(true); } -@Override -public AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, int gcBefore, long maxSSTableBytes) -{ -return new CompactionTask(cfs, sstables, gcBefore); -} - public int getEstimatedRemainingTasks() { return estimatedRemainingTasks;
[2/2] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/92b40fd9 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/92b40fd9 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/92b40fd9 Branch: refs/heads/cassandra-2.1 Commit: 92b40fd90d9d590b738a325ee68ca80acfaa97f9 Parents: d6b5826 80b3e6c Author: Marcus Eriksson marc...@apache.org Authored: Tue Apr 1 22:26:49 2014 +0200 Committer: Marcus Eriksson marc...@apache.org Committed: Tue Apr 1 22:26:49 2014 +0200 -- .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 - .../cassandra/db/compaction/SizeTieredCompactionStrategy.java | 6 -- 2 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/92b40fd9/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/92b40fd9/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java --
[1/2] git commit: Don't break API in minor rev, followup to CASSANDRA-6958
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 d6b582614 - 92b40fd90 Don't break API in minor rev, followup to CASSANDRA-6958 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/80b3e6cc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/80b3e6cc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/80b3e6cc Branch: refs/heads/cassandra-2.1 Commit: 80b3e6cc3b1d6fdae91193563626f0823c92c528 Parents: 55fba6a Author: Marcus Eriksson marc...@apache.org Authored: Tue Apr 1 22:03:30 2014 +0200 Committer: Marcus Eriksson marc...@apache.org Committed: Tue Apr 1 22:16:10 2014 +0200 -- .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 - .../cassandra/db/compaction/SizeTieredCompactionStrategy.java | 6 -- 2 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index d3339d5..941d3d7 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -167,7 +167,10 @@ public abstract class AbstractCompactionStrategy */ public abstract AbstractCompactionTask getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore); -public abstract AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long maxSSTableBytes); +public AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long maxSSTableBytes) +{ +return new CompactionTask(cfs, sstables, gcBefore); +} /** * @return the number of background tasks estimated to still be needed for this columnfamilystore http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java index fae52e2..0f94918 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java @@ -271,12 +271,6 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy return new CompactionTask(cfs, sstables, gcBefore).setUserDefined(true); } -@Override -public AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, int gcBefore, long maxSSTableBytes) -{ -return new CompactionTask(cfs, sstables, gcBefore); -} - public int getEstimatedRemainingTasks() { return estimatedRemainingTasks;
[2/3] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/92b40fd9 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/92b40fd9 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/92b40fd9 Branch: refs/heads/trunk Commit: 92b40fd90d9d590b738a325ee68ca80acfaa97f9 Parents: d6b5826 80b3e6c Author: Marcus Eriksson marc...@apache.org Authored: Tue Apr 1 22:26:49 2014 +0200 Committer: Marcus Eriksson marc...@apache.org Committed: Tue Apr 1 22:26:49 2014 +0200 -- .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 - .../cassandra/db/compaction/SizeTieredCompactionStrategy.java | 6 -- 2 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/92b40fd9/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/92b40fd9/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java --
[1/3] git commit: Don't break API in minor rev, followup to CASSANDRA-6958
Repository: cassandra Updated Branches: refs/heads/trunk f94a7af17 - 2eecd4039 Don't break API in minor rev, followup to CASSANDRA-6958 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/80b3e6cc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/80b3e6cc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/80b3e6cc Branch: refs/heads/trunk Commit: 80b3e6cc3b1d6fdae91193563626f0823c92c528 Parents: 55fba6a Author: Marcus Eriksson marc...@apache.org Authored: Tue Apr 1 22:03:30 2014 +0200 Committer: Marcus Eriksson marc...@apache.org Committed: Tue Apr 1 22:16:10 2014 +0200 -- .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 - .../cassandra/db/compaction/SizeTieredCompactionStrategy.java | 6 -- 2 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index d3339d5..941d3d7 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -167,7 +167,10 @@ public abstract class AbstractCompactionStrategy */ public abstract AbstractCompactionTask getUserDefinedTask(CollectionSSTableReader sstables, final int gcBefore); -public abstract AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long maxSSTableBytes); +public AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, final int gcBefore, long maxSSTableBytes) +{ +return new CompactionTask(cfs, sstables, gcBefore); +} /** * @return the number of background tasks estimated to still be needed for this columnfamilystore http://git-wip-us.apache.org/repos/asf/cassandra/blob/80b3e6cc/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java index fae52e2..0f94918 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java @@ -271,12 +271,6 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy return new CompactionTask(cfs, sstables, gcBefore).setUserDefined(true); } -@Override -public AbstractCompactionTask getCompactionTask(CollectionSSTableReader sstables, int gcBefore, long maxSSTableBytes) -{ -return new CompactionTask(cfs, sstables, gcBefore); -} - public int getEstimatedRemainingTasks() { return estimatedRemainingTasks;
[3/3] git commit: Merge branch 'cassandra-2.1' into trunk
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/2eecd403 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2eecd403 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2eecd403 Branch: refs/heads/trunk Commit: 2eecd40392cd65e211480cd7069fabf5ad6a5950 Parents: f94a7af 92b40fd Author: Marcus Eriksson marc...@apache.org Authored: Tue Apr 1 22:28:11 2014 +0200 Committer: Marcus Eriksson marc...@apache.org Committed: Tue Apr 1 22:28:11 2014 +0200 -- .../cassandra/db/compaction/AbstractCompactionStrategy.java| 5 - .../cassandra/db/compaction/SizeTieredCompactionStrategy.java | 6 -- 2 files changed, 4 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2eecd403/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java --
[jira] [Commented] (CASSANDRA-6892) Cassandra 2.0.x validates Thrift columns incorrectly and causes InvalidRequestException
[ https://issues.apache.org/jira/browse/CASSANDRA-6892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957001#comment-13957001 ] Tyler Hobbs commented on CASSANDRA-6892: [~slebresne] thanks, your approach is nice. I have a few nits about your patch: * Most of the methods in ScrubTest.java are accidentally commented out. * Try to add javadocs for methods instead of normal comments (e.g. {{ColumnDefinition.isPartOfCellName()}}). I know you use vim, but it's helpful with IDEs ;) * In {{Column.validateFields()}}, the cfdef variable is unused. Other than that, +1 Cassandra 2.0.x validates Thrift columns incorrectly and causes InvalidRequestException --- Key: CASSANDRA-6892 URL: https://issues.apache.org/jira/browse/CASSANDRA-6892 Project: Cassandra Issue Type: Bug Components: API Reporter: Christian Spriegel Assignee: Tyler Hobbs Priority: Minor Fix For: 2.0.7 Attachments: 6892-2.0-v2.txt, 6892-2.0.txt, CASSANDRA-6892_V1.patch I just upgrade my local dev machine to Cassandra 2.0, which causes one of my automated tests to fail now. With the latest 1.2.x it was working fine. The Exception I get on my client (using Hector) is: {code} me.prettyprint.hector.api.exceptions.HInvalidRequestException: InvalidRequestException(why:(Expected 8 or 0 byte long (21)) [MDS_0][MasterdataIndex][key2] failed validation) at me.prettyprint.cassandra.service.ExceptionsTranslatorImpl.translate(ExceptionsTranslatorImpl.java:52) at me.prettyprint.cassandra.connection.HConnectionManager.operateWithFailover(HConnectionManager.java:265) at me.prettyprint.cassandra.model.ExecutingKeyspace.doExecuteOperation(ExecutingKeyspace.java:113) at me.prettyprint.cassandra.model.MutatorImpl.execute(MutatorImpl.java:243) at me.prettyprint.cassandra.service.template.AbstractColumnFamilyTemplate.executeBatch(AbstractColumnFamilyTemplate.java:115) at me.prettyprint.cassandra.service.template.AbstractColumnFamilyTemplate.executeIfNotBatched(AbstractColumnFamilyTemplate.java:163) at me.prettyprint.cassandra.service.template.ColumnFamilyTemplate.update(ColumnFamilyTemplate.java:69) at com.mycompany.spring3utils.dataaccess.cassandra.AbstractCassandraDAO.doUpdate(AbstractCassandraDAO.java:482) Caused by: InvalidRequestException(why:(Expected 8 or 0 byte long (21)) [MDS_0][MasterdataIndex][key2] failed validation) at org.apache.cassandra.thrift.Cassandra$batch_mutate_result.read(Cassandra.java:20833) at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:78) at org.apache.cassandra.thrift.Cassandra$Client.recv_batch_mutate(Cassandra.java:964) at org.apache.cassandra.thrift.Cassandra$Client.batch_mutate(Cassandra.java:950) at me.prettyprint.cassandra.model.MutatorImpl$3.execute(MutatorImpl.java:246) at me.prettyprint.cassandra.model.MutatorImpl$3.execute(MutatorImpl.java:1) at me.prettyprint.cassandra.service.Operation.executeAndSetResult(Operation.java:104) at me.prettyprint.cassandra.connection.HConnectionManager.operateWithFailover(HConnectionManager.java:258) ... 46 more {code} The schema of my column family is: {code} create column family MasterdataIndex with compression_options = {sstable_compression:SnappyCompressor, chunk_length_kb:64} and comparator = UTF8Type and key_validation_class = 'CompositeType(UTF8Type,LongType)' and default_validation_class = BytesType; {code} From the error message it looks like Cassandra is trying to validate the value with the key-validator! (My value in this case it 21 bytes long) I studied the Cassandra 2.0 code and found something wrong. It seems in CFMetaData.addDefaultKeyAliases it passes the KeyValidator into ColumnDefinition.partitionKeyDef. Inside ColumnDefinition the validator is expected to be the value validator! In CFMetaData: {code} private ListColumnDefinition addDefaultKeyAliases(ListColumnDefinition pkCols) { for (int i = 0; i pkCols.size(); i++) { if (pkCols.get(i) == null) { Integer idx = null; AbstractType? type = keyValidator; if (keyValidator instanceof CompositeType) { idx = i; type = ((CompositeType)keyValidator).types.get(i); } // For compatibility sake, we call the first alias 'key' rather than 'key1'. This // is inconsistent with column alias, but it's probably not worth risking breaking compatibility now. ByteBuffer name =
[jira] [Commented] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
[ https://issues.apache.org/jira/browse/CASSANDRA-6969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957020#comment-13957020 ] Jonathan Ellis commented on CASSANDRA-6969: --- We should look at it after CASSANDRA-6968, but for now it batch CL doesn't look like a big contributor to the slowness. On my machine it's 12:25 with batch and 12:02 with periodic. Use Unsafe Mutations Where Possible in Unit Tests - Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6969) Use Unsafe Mutations Where Possible in Unit Tests
[ https://issues.apache.org/jira/browse/CASSANDRA-6969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957020#comment-13957020 ] Jonathan Ellis edited comment on CASSANDRA-6969 at 4/1/14 9:15 PM: --- We should look at it after CASSANDRA-6968, but for now batch CL doesn't look like a big contributor to the slowness. On my machine it's 12:25 with batch and 12:02 with periodic. was (Author: jbellis): We should look at it after CASSANDRA-6968, but for now it batch CL doesn't look like a big contributor to the slowness. On my machine it's 12:25 with batch and 12:02 with periodic. Use Unsafe Mutations Where Possible in Unit Tests - Key: CASSANDRA-6969 URL: https://issues.apache.org/jira/browse/CASSANDRA-6969 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Since the test config uses the batch mode for the commitlog, safe writes are quite slow. In tests that don't rely on safe writes, we should use unsafe writes. This mostly consists of converting {{RowMutation.apply()}} calls into {{RowMutation.applyUnsafe()}} calls. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6912) SSTableReader.isReplaced does not allow for safe resource cleanup
[ https://issues.apache.org/jira/browse/CASSANDRA-6912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957076#comment-13957076 ] Tyler Hobbs commented on CASSANDRA-6912: Overall this looks good. The only part I'm not sure about is running the SSTableDeletingTask and deleting the data/index files directly in tidy(). Instead of calling deletingTask.run() directly, deletingTask.schedule() should be used to properly handle failures. I believe there's a similar problem with calling deleteWithConfirm() directly on the files; see the comment near the top of SSTableDeletingTask. Adding a comment to tidy() with your summary would also be good. SSTableReader.isReplaced does not allow for safe resource cleanup - Key: CASSANDRA-6912 URL: https://issues.apache.org/jira/browse/CASSANDRA-6912 Project: Cassandra Issue Type: Bug Reporter: Benedict Assignee: Benedict Fix For: 2.1 beta2 There are a number of possible race conditions on resource cleanup from the use of cloneWithNewSummarySamplingLevel, because the replacement sstable can be itself replaced/obsoleted while the prior sstable is still referenced (this is actually quite easy with compaction, but can happen in other circumstances less commonly). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6912) SSTableReader.isReplaced does not allow for safe resource cleanup
[ https://issues.apache.org/jira/browse/CASSANDRA-6912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957094#comment-13957094 ] Benedict commented on CASSANDRA-6912: - Dagnabit. Good catch - I originally had this bundled up with two other patches, and in separating them I was apparently not as careful as I had thought. The whole of tidy() was scheduled, and I thought I didn't need that after I extracted this (for reasons I won't get into). I'll reintroduce it. SSTableReader.isReplaced does not allow for safe resource cleanup - Key: CASSANDRA-6912 URL: https://issues.apache.org/jira/browse/CASSANDRA-6912 Project: Cassandra Issue Type: Bug Reporter: Benedict Assignee: Benedict Fix For: 2.1 beta2 There are a number of possible race conditions on resource cleanup from the use of cloneWithNewSummarySamplingLevel, because the replacement sstable can be itself replaced/obsoleted while the prior sstable is still referenced (this is actually quite easy with compaction, but can happen in other circumstances less commonly). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6971) nodes not seeing new keyspace
Russ Hatch created CASSANDRA-6971: - Summary: nodes not seeing new keyspace Key: CASSANDRA-6971 URL: https://issues.apache.org/jira/browse/CASSANDRA-6971 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch The dtest suite is running a test which creates a 3 node cluster, then adds a keyspace and column family. For some reason the 3 nodes are not agreeing on the schema version. The problem is intermittent -- either the nodes all agree on schema quickly, or they seem to stay stuck in limbo. The simplest way to reproduce is to run the dtest (simple_increment_test): https://github.com/riptano/cassandra-dtest/blob/master/counter_tests.py using nosetests: {noformat} nosetests -vs counter_tests.py:TestCounters.simple_increment_test {noformat} If the problem is reproduced nose will return this: ProgrammingError: Bad Request: Keyspace 'ks' does not exist I am not yet sure if the bug is reproducible outside of the dtest suite. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6818) SSTable references not released if stream session fails before it starts
[ https://issues.apache.org/jira/browse/CASSANDRA-6818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-6818: -- Attachment: 6818-2.0-v3.txt [~rlow] Thanks for reviewing. I attached updated version that I hope will clear concurrency issue your pointed out. SSTable references not released if stream session fails before it starts Key: CASSANDRA-6818 URL: https://issues.apache.org/jira/browse/CASSANDRA-6818 Project: Cassandra Issue Type: Bug Components: Core Reporter: Richard Low Assignee: Yuki Morishita Fix For: 1.2.16, 2.0.7, 2.1 beta2 Attachments: 6818-1.2.txt, 6818-2.0-v2.txt, 6818-2.0-v3.txt, 6818-2.0.txt I observed a large number of 'orphan' SSTables - SSTables that are in the data directory but not loaded by Cassandra - on a 1.1.12 node that had a large stream fail before it started. These orphan files are particularly dangerous because if the node is restarted and picks up these SSTables it could bring data back to life if tombstones have been GCed. To confirm the SSTables are orphan, I created a snapshot and it didn't contain these files. I can see in the logs that they have been compacted so should have been deleted. The log entries for the stream are: {{INFO [StreamStage:1] 2014-02-21 19:41:48,742 StreamOut.java (line 115) Beginning transfer to /10.0.0.1}} {{INFO [StreamStage:1] 2014-02-21 19:41:48,743 StreamOut.java (line 96) Flushing memtables for [CFS(Keyspace='ks', ColumnFamily='cf1'), CFS(Keyspace='ks', ColumnFamily='cf2')]...}} {{ERROR [GossipTasks:1] 2014-02-21 19:41:49,239 AbstractStreamSession.java (line 113) Stream failed because /10.0.0.1 died or was restarted/removed (streams may still be active in background, but further streams won't be started)}} {{INFO [StreamStage:1] 2014-02-21 19:41:51,783 StreamOut.java (line 161) Stream context metadata [...] 2267 sstables.}} {{INFO [StreamStage:1] 2014-02-21 19:41:51,789 StreamOutSession.java (line 182) Streaming to /10.0.0.1}} {{INFO [Streaming to /10.0.0.1:1] 2014-02-21 19:42:02,218 FileStreamTask.java (line 99) Found no stream out session at end of file stream task - this is expected if the receiver went down}} After digging in the code, here's what I think the issue is: 1. StreamOutSession.transferRanges() creates a streaming session, which is registered with the failure detector in AbstractStreamSession's constructor. 2. Memtables are flushed, potentially taking a long time. 3. The remote node fails, convict() is called and the StreamOutSession is closed. However, at this time StreamOutSession.files is empty because it's still waiting for the memtables to flush. 4. Memtables finish flusing, references are obtained to SSTables to be streamed and the PendingFiles are added to StreamOutSession.files. 5. The first stream fails but the StreamOutSession isn't found so is never closed and the references are never released. This code is more or less the same on 1.2 so I would expect it to reproduce there. I looked at 2.0 and can't even see where SSTable references are released when the stream fails. Some possible fixes for 1.1/1.2: 1. Don't register with the failure detector until after the PendingFiles are set up. I think this is the behaviour in 2.0 but I don't know if it was done like this to avoid this issue. 2. Detect the above case in (e.g.) StreamOutSession.begin() by noticing the session has been closed with care to avoid double frees. 3. Add some synchronization so closeInternal() doesn't race with setting up the session. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6971) nodes not seeing new keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-6971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957136#comment-13957136 ] Russ Hatch commented on CASSANDRA-6971: --- here's the schema output for the created keyspace (which all nodes don't see): {noformat} CREATE KEYSPACE ks WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '3' }; USE ks; CREATE TABLE cf ( key text PRIMARY KEY, c counter ) WITH bloom_filter_fp_chance=0.01 AND caching='KEYS_ONLY' AND comment='test cf' AND dclocal_read_repair_chance=0.00 AND gc_grace_seconds=864000 AND read_repair_chance=0.10 AND replicate_on_write='true' AND populate_io_cache_on_flush='false' AND compaction={'class': 'SizeTieredCompactionStrategy'} AND compression={'sstable_compression': 'SnappyCompressor'}; {noformat} nodes not seeing new keyspace - Key: CASSANDRA-6971 URL: https://issues.apache.org/jira/browse/CASSANDRA-6971 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch The dtest suite is running a test which creates a 3 node cluster, then adds a keyspace and column family. For some reason the 3 nodes are not agreeing on the schema version. The problem is intermittent -- either the nodes all agree on schema quickly, or they seem to stay stuck in limbo. The simplest way to reproduce is to run the dtest (simple_increment_test): https://github.com/riptano/cassandra-dtest/blob/master/counter_tests.py using nosetests: {noformat} nosetests -vs counter_tests.py:TestCounters.simple_increment_test {noformat} If the problem is reproduced nose will return this: ProgrammingError: Bad Request: Keyspace 'ks' does not exist I am not yet sure if the bug is reproducible outside of the dtest suite. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6971) nodes not catching up to
[ https://issues.apache.org/jira/browse/CASSANDRA-6971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russ Hatch updated CASSANDRA-6971: -- Summary: nodes not catching up to (was: nodes not seeing new keyspace) nodes not catching up to Key: CASSANDRA-6971 URL: https://issues.apache.org/jira/browse/CASSANDRA-6971 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch The dtest suite is running a test which creates a 3 node cluster, then adds a keyspace and column family. For some reason the 3 nodes are not agreeing on the schema version. The problem is intermittent -- either the nodes all agree on schema quickly, or they seem to stay stuck in limbo. The simplest way to reproduce is to run the dtest (simple_increment_test): https://github.com/riptano/cassandra-dtest/blob/master/counter_tests.py using nosetests: {noformat} nosetests -vs counter_tests.py:TestCounters.simple_increment_test {noformat} If the problem is reproduced nose will return this: ProgrammingError: Bad Request: Keyspace 'ks' does not exist I am not yet sure if the bug is reproducible outside of the dtest suite. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6971) nodes not catching up to creation of new keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-6971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russ Hatch updated CASSANDRA-6971: -- Summary: nodes not catching up to creation of new keyspace (was: nodes not catching up to) nodes not catching up to creation of new keyspace - Key: CASSANDRA-6971 URL: https://issues.apache.org/jira/browse/CASSANDRA-6971 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch The dtest suite is running a test which creates a 3 node cluster, then adds a keyspace and column family. For some reason the 3 nodes are not agreeing on the schema version. The problem is intermittent -- either the nodes all agree on schema quickly, or they seem to stay stuck in limbo. The simplest way to reproduce is to run the dtest (simple_increment_test): https://github.com/riptano/cassandra-dtest/blob/master/counter_tests.py using nosetests: {noformat} nosetests -vs counter_tests.py:TestCounters.simple_increment_test {noformat} If the problem is reproduced nose will return this: ProgrammingError: Bad Request: Keyspace 'ks' does not exist I am not yet sure if the bug is reproducible outside of the dtest suite. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6957) testNewRepairedSSTable fails intermittently
[ https://issues.apache.org/jira/browse/CASSANDRA-6957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Shuler updated CASSANDRA-6957: -- Attachment: system.log.txt Just got one - log attached. {noformat} [junit] Testsuite: org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest [junit] Tests run: 4, Failures: 1, Errors: 0, Time elapsed: 20.36 sec [junit] [junit] Testcase: testNewRepairedSSTable(org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest):FAILED [junit] expected:1 but was:0 [junit] junit.framework.AssertionFailedError: expected:1 but was:0 [junit] at org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest.testNewRepairedSSTable(LeveledCompactionStrategyTest.java:267) [junit] [junit] [junit] Test org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest FAILED {noformat} testNewRepairedSSTable fails intermittently --- Key: CASSANDRA-6957 URL: https://issues.apache.org/jira/browse/CASSANDRA-6957 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 beta2 Attachments: system.log.txt -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6971) nodes not catching up to creation of new keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-6971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957143#comment-13957143 ] Brandon Williams commented on CASSANDRA-6971: - Sounds a lot like CASSANDRA-6177 to me nodes not catching up to creation of new keyspace - Key: CASSANDRA-6971 URL: https://issues.apache.org/jira/browse/CASSANDRA-6971 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch The dtest suite is running a test which creates a 3 node cluster, then adds a keyspace and column family. For some reason the 3 nodes are not agreeing on the schema version. The problem is intermittent -- either the nodes all agree on schema quickly, or they seem to stay stuck in limbo. The simplest way to reproduce is to run the dtest (simple_increment_test): https://github.com/riptano/cassandra-dtest/blob/master/counter_tests.py using nosetests: {noformat} nosetests -vs counter_tests.py:TestCounters.simple_increment_test {noformat} If the problem is reproduced nose will return this: ProgrammingError: Bad Request: Keyspace 'ks' does not exist I am not yet sure if the bug is reproducible outside of the dtest suite. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-6972) Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds
Darla Baker created CASSANDRA-6972: -- Summary: Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds Key: CASSANDRA-6972 URL: https://issues.apache.org/jira/browse/CASSANDRA-6972 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Darla Baker Priority: Minor Obviously when this condition exists the node will not bootstrap. But it is not obvious from the logs why it is not bootstrapping. Throwing an error would make it obvious and therefore faster to correct. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6971) nodes not catching up to creation of new keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-6971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957166#comment-13957166 ] Russ Hatch commented on CASSANDRA-6971: --- Yeah, I dislike adding pauses to dtests and agree with the idea of CASSANDRA-6177. I would prefer to block until schema agreement in this test rather than blindly sleeping. But even with a manual pause of 2 minutes the 3 nodes don't arrive at agreement. The ks is still not present in some nodes after this lengthy wait, so I don't think the nodes are going to agree at all. nodes not catching up to creation of new keyspace - Key: CASSANDRA-6971 URL: https://issues.apache.org/jira/browse/CASSANDRA-6971 Project: Cassandra Issue Type: Bug Reporter: Russ Hatch The dtest suite is running a test which creates a 3 node cluster, then adds a keyspace and column family. For some reason the 3 nodes are not agreeing on the schema version. The problem is intermittent -- either the nodes all agree on schema quickly, or they seem to stay stuck in limbo. The simplest way to reproduce is to run the dtest (simple_increment_test): https://github.com/riptano/cassandra-dtest/blob/master/counter_tests.py using nosetests: {noformat} nosetests -vs counter_tests.py:TestCounters.simple_increment_test {noformat} If the problem is reproduced nose will return this: ProgrammingError: Bad Request: Keyspace 'ks' does not exist I am not yet sure if the bug is reproducible outside of the dtest suite. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6972) Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-6972?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957184#comment-13957184 ] Brandon Williams commented on CASSANDRA-6972: - The catch with doing this is, now everyone has to go around and put autobootstrap: false in their seed configs. Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds -- Key: CASSANDRA-6972 URL: https://issues.apache.org/jira/browse/CASSANDRA-6972 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Darla Baker Priority: Minor Obviously when this condition exists the node will not bootstrap. But it is not obvious from the logs why it is not bootstrapping. Throwing an error would make it obvious and therefore faster to correct. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6965) sstableloader fails to stream data
[ https://issues.apache.org/jira/browse/CASSANDRA-6965?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957207#comment-13957207 ] Yuki Morishita commented on CASSANDRA-6965: --- It looks like this is regression from CASSANDRA-6596. Root stack trace is: {code} Exception in thread STREAM-OUT-/127.0.0.1 java.lang.NullPointerException at org.apache.cassandra.streaming.StreamManager$StreamRateLimiter.init(StreamManager.java:79) at org.apache.cassandra.streaming.StreamManager.getRateLimiter(StreamManager.java:61) at org.apache.cassandra.streaming.StreamWriter.init(StreamWriter.java:60) at org.apache.cassandra.streaming.compress.CompressedStreamWriter.init(CompressedStreamWriter.java:47) at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:51) at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:40) at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:389) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:359) at java.lang.Thread.run(Thread.java:744) {code} Line 79 of StreamManager.java indicates that DC names should be loaded from cassandra.yaml file, but in sstableloader cassandra.yaml should not be readed. [~vijay2...@yahoo.com] can you take a look? sstableloader fails to stream data -- Key: CASSANDRA-6965 URL: https://issues.apache.org/jira/browse/CASSANDRA-6965 Project: Cassandra Issue Type: Bug Reporter: Ryan McGuire Assignee: Yuki Morishita When loading a snapshot via sstableloader I get this NullPointerException: {code} Established connection to initial hosts Opening sstables and calculating sections to stream Streaming relevant part of /tmp/tmpmusajE/ks/cf/ks-cf-ka-1-Data.db to [/127.0.0.1] Streaming session ID: 276a9d20-b944-11e3-9f60-4191ec2d6f66 Exception in thread STREAM-OUT-/127.0.0.1 java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler$MessageHandler.signalCloseDone(ConnectionHandler.java:249) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:375) at java.lang.Thread.run(Thread.java:744) {code} Which at first I thought meant it was unable to connect, but I do see this in system.log, indicating the connection was successful: {code} INFO [STREAM-INIT-/127.0.0.1:55820] 2014-03-31 22:20:12,265 StreamResultFuture.java:121 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Received streaming plan for Bulk Load INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,272 StreamResultFuture.java:173 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Prepare completed. Receiving 1 files(1726 bytes), sending 0 files(0 bytes) INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,280 CompressedStreamReader.java:65 - reading file from /10.13.37.130, repairedAt = 0 {code} To reproduce this behaviour, run the [snapshot_test.py:TestSnapshot.test_basic_snapshot dtest|https://github.com/riptano/cassandra-dtest/blob/85692be39cb6b1b5ebc96905834ff7bb174de0bc/snapshot_test.py#L40] on 2.1 HEAD. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6965) sstableloader fails to stream data
[ https://issues.apache.org/jira/browse/CASSANDRA-6965?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13957208#comment-13957208 ] Yuki Morishita commented on CASSANDRA-6965: --- Because of this bug, 2.1 sstableloader is not working right now. sstableloader fails to stream data -- Key: CASSANDRA-6965 URL: https://issues.apache.org/jira/browse/CASSANDRA-6965 Project: Cassandra Issue Type: Bug Reporter: Ryan McGuire Assignee: Yuki Morishita When loading a snapshot via sstableloader I get this NullPointerException: {code} Established connection to initial hosts Opening sstables and calculating sections to stream Streaming relevant part of /tmp/tmpmusajE/ks/cf/ks-cf-ka-1-Data.db to [/127.0.0.1] Streaming session ID: 276a9d20-b944-11e3-9f60-4191ec2d6f66 Exception in thread STREAM-OUT-/127.0.0.1 java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler$MessageHandler.signalCloseDone(ConnectionHandler.java:249) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:375) at java.lang.Thread.run(Thread.java:744) {code} Which at first I thought meant it was unable to connect, but I do see this in system.log, indicating the connection was successful: {code} INFO [STREAM-INIT-/127.0.0.1:55820] 2014-03-31 22:20:12,265 StreamResultFuture.java:121 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Received streaming plan for Bulk Load INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,272 StreamResultFuture.java:173 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Prepare completed. Receiving 1 files(1726 bytes), sending 0 files(0 bytes) INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,280 CompressedStreamReader.java:65 - reading file from /10.13.37.130, repairedAt = 0 {code} To reproduce this behaviour, run the [snapshot_test.py:TestSnapshot.test_basic_snapshot dtest|https://github.com/riptano/cassandra-dtest/blob/85692be39cb6b1b5ebc96905834ff7bb174de0bc/snapshot_test.py#L40] on 2.1 HEAD. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6965) sstableloader fails to stream data
[ https://issues.apache.org/jira/browse/CASSANDRA-6965?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-6965: -- Assignee: Vijay (was: Yuki Morishita) sstableloader fails to stream data -- Key: CASSANDRA-6965 URL: https://issues.apache.org/jira/browse/CASSANDRA-6965 Project: Cassandra Issue Type: Bug Reporter: Ryan McGuire Assignee: Vijay When loading a snapshot via sstableloader I get this NullPointerException: {code} Established connection to initial hosts Opening sstables and calculating sections to stream Streaming relevant part of /tmp/tmpmusajE/ks/cf/ks-cf-ka-1-Data.db to [/127.0.0.1] Streaming session ID: 276a9d20-b944-11e3-9f60-4191ec2d6f66 Exception in thread STREAM-OUT-/127.0.0.1 java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler$MessageHandler.signalCloseDone(ConnectionHandler.java:249) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:375) at java.lang.Thread.run(Thread.java:744) {code} Which at first I thought meant it was unable to connect, but I do see this in system.log, indicating the connection was successful: {code} INFO [STREAM-INIT-/127.0.0.1:55820] 2014-03-31 22:20:12,265 StreamResultFuture.java:121 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Received streaming plan for Bulk Load INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,272 StreamResultFuture.java:173 - [Stream #276a9d20-b944-11e3-9f60-4191ec2d6f66] Prepare completed. Receiving 1 files(1726 bytes), sending 0 files(0 bytes) INFO [STREAM-IN-/10.13.37.130] 2014-03-31 22:20:12,280 CompressedStreamReader.java:65 - reading file from /10.13.37.130, repairedAt = 0 {code} To reproduce this behaviour, run the [snapshot_test.py:TestSnapshot.test_basic_snapshot dtest|https://github.com/riptano/cassandra-dtest/blob/85692be39cb6b1b5ebc96905834ff7bb174de0bc/snapshot_test.py#L40] on 2.1 HEAD. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-6972) Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-6972?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis reassigned CASSANDRA-6972: - Assignee: Brandon Williams Yes, the right fix is to just log it explicitly at info. We added that back in CASSANDRA-746 but it got undone at some point. Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds -- Key: CASSANDRA-6972 URL: https://issues.apache.org/jira/browse/CASSANDRA-6972 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Darla Baker Assignee: Brandon Williams Priority: Minor Fix For: 2.0.7 Obviously when this condition exists the node will not bootstrap. But it is not obvious from the logs why it is not bootstrapping. Throwing an error would make it obvious and therefore faster to correct. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-6972) Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-6972?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-6972: -- Fix Version/s: 2.0.7 Throw an ERROR when auto_bootstrap: true and bootstrapping node is listed in seeds -- Key: CASSANDRA-6972 URL: https://issues.apache.org/jira/browse/CASSANDRA-6972 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Darla Baker Assignee: Brandon Williams Priority: Minor Fix For: 2.0.7 Obviously when this condition exists the node will not bootstrap. But it is not obvious from the logs why it is not bootstrapping. Throwing an error would make it obvious and therefore faster to correct. -- This message was sent by Atlassian JIRA (v6.2#6252)