[jira] [Created] (CASSANDRA-6640) Improve custom 2i performance and abstraction
Miguel Angel Fernandez Diaz created CASSANDRA-6640: -- Summary: Improve custom 2i performance and abstraction Key: CASSANDRA-6640 URL: https://issues.apache.org/jira/browse/CASSANDRA-6640 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Miguel Angel Fernandez Diaz Fix For: 2.1 With the current implementation, the method update from SecondaryIndexManager forces to insert and delete a cell. That happens because we assume that we need the value of the old cell in order to locate the cell that we are updating in our custom secondary index implementation. However, depending on the implementation, a insert and a delete operations could have much worse performance than a simple update. Moreover, if our custom secondary index doesn't use inverted indexes, we don't really need the old cell information and the key information is enough. Therefore, a good solution would be to make the update method more abstract. Thus, the update method for PerColumnSecondaryIndex would receive also the old cell information and from that point we could decide if we must carry out the delete+insert operation or just a update operation. I attach a patch that implements this solution. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6640) Improve custom 2i performance and abstraction
[ https://issues.apache.org/jira/browse/CASSANDRA-6640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Miguel Angel Fernandez Diaz updated CASSANDRA-6640: --- Attachment: 6640.diff The solution takes into account the issue 5540 Improve custom 2i performance and abstraction - Key: CASSANDRA-6640 URL: https://issues.apache.org/jira/browse/CASSANDRA-6640 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Miguel Angel Fernandez Diaz Labels: patch, performance Fix For: 2.1 Attachments: 6640.diff With the current implementation, the method update from SecondaryIndexManager forces to insert and delete a cell. That happens because we assume that we need the value of the old cell in order to locate the cell that we are updating in our custom secondary index implementation. However, depending on the implementation, a insert and a delete operations could have much worse performance than a simple update. Moreover, if our custom secondary index doesn't use inverted indexes, we don't really need the old cell information and the key information is enough. Therefore, a good solution would be to make the update method more abstract. Thus, the update method for PerColumnSecondaryIndex would receive also the old cell information and from that point we could decide if we must carry out the delete+insert operation or just a update operation. I attach a patch that implements this solution. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6638) SSTableScanner can Skip Rows with vnodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6638?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6638: Attachment: 6638.txt Attaching simple fix (the patch includes Tyler's unit test). For the records, the original reported of the issue, Ignace Desimpel, provided the following analysis on the mailing list: {quote} To see what is wrong, think of having 3 ranges in the list, and both the first and second range will not produce a valid currentKey. The first time in the loop we get the first range, and then call seekToCurrentRangeStart(). That routine doesn’t do anything in that case, so then the first key is read from the sstable. But this first key does not match the first range, so we loop again. We get the second range and call seekToCurrentRangeStart() again. Again this does not do anything, leaving all file pointers. So then a new currentKey is read from the sstable BUT that should not be the case. We should, in that case, continue to test with the ‘old’ currentKey. {quote} (which is a fair description of what triggers the problem) and proposed a fix. That fix was modifying the logic of KeyScanningIterator.computeNext(), and while the fix itself is probably fine, it complicate the logic a bit and I think it's simpler and cleaner to just fix seekToCurrentRangeStart to really always seek to the first key greater than the current range start (that was clearly the intent given the comment when indexPosition = -1 in that method, but the method wrongly assumed we were always at the beginning of the sstable). So that's what the attached patch does. I'll note that this does mean we might re-read the same key if the sstable have no keys for at least 2 consecutive range, but this really doesn't matter in practice so we should stick to cleaner code. bq. Are there non-cleanup related consequences of this issue? No. This can only ever happen if SSTableScanner is uses with at least 2 ranges and cleanup is the only place we do that (I double-checked). I'll also note that this could affect non-vnode cases though it's a lot less likely, at least provided you use a random partitionner, as you need for a sstable to have no keys for a given local range, which is possible but much less likely without vnodes. Lastly, this might skip at most 1 row per-local-range. SSTableScanner can Skip Rows with vnodes Key: CASSANDRA-6638 URL: https://issues.apache.org/jira/browse/CASSANDRA-6638 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Tyler Hobbs Priority: Blocker Fix For: 2.0.5 Attachments: 6638-repro-test.txt, 6638.txt CASSANDRA-2524 added multiple range support to SSTableScanner, but it looks like there is at least one case where keys can be skipped. This can result in cleanup removing legitimate keys. See the attached patch that adds a unit test to reproduce the case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Issue Comment Deleted] (CASSANDRA-6640) Improve custom 2i performance and abstraction
[ https://issues.apache.org/jira/browse/CASSANDRA-6640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Miguel Angel Fernandez Diaz updated CASSANDRA-6640: --- Comment: was deleted (was: The solution takes into account the issue 5540) Improve custom 2i performance and abstraction - Key: CASSANDRA-6640 URL: https://issues.apache.org/jira/browse/CASSANDRA-6640 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Miguel Angel Fernandez Diaz Labels: patch, performance Fix For: 2.1 Attachments: 6640.diff With the current implementation, the method update from SecondaryIndexManager forces to insert and delete a cell. That happens because we assume that we need the value of the old cell in order to locate the cell that we are updating in our custom secondary index implementation. However, depending on the implementation, a insert and a delete operations could have much worse performance than a simple update. Moreover, if our custom secondary index doesn't use inverted indexes, we don't really need the old cell information and the key information is enough. Therefore, a good solution would be to make the update method more abstract. Thus, the update method for PerColumnSecondaryIndex would receive also the old cell information and from that point we could decide if we must carry out the delete+insert operation or just a update operation. I attach a patch that implements this solution. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6638) SSTableScanner can Skip Rows with vnodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886570#comment-13886570 ] Ignace Desimpel commented on CASSANDRA-6638: Thanks for the simple patch! Related to efficiency : Suppose we have sstable data in every other range. Then the first range gets data matching the range. The second does not, and a 'file' seek is done to the first key greater than the left of the this second range, thus to the first sample key in the third range. And then a loop is started over all the ifile entries until end of ifile file (there is no upper boundary check)! That would repeat itself over and over again if we happen to have such a data and range arrangement and depending on the number of vnodes. Correct? That means a lot of work for nothing? SSTableScanner can Skip Rows with vnodes Key: CASSANDRA-6638 URL: https://issues.apache.org/jira/browse/CASSANDRA-6638 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Blocker Fix For: 2.0.5 Attachments: 6638-repro-test.txt, 6638.txt CASSANDRA-2524 added multiple range support to SSTableScanner, but it looks like there is at least one case where keys can be skipped. This can result in cleanup removing legitimate keys. See the attached patch that adds a unit test to reproduce the case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6640) Improve custom 2i performance and abstraction
[ https://issues.apache.org/jira/browse/CASSANDRA-6640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-6640: Reviewer: Sam Tunnicliffe Assignee: Miguel Angel Fernandez Diaz Improve custom 2i performance and abstraction - Key: CASSANDRA-6640 URL: https://issues.apache.org/jira/browse/CASSANDRA-6640 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Miguel Angel Fernandez Diaz Assignee: Miguel Angel Fernandez Diaz Labels: patch, performance Fix For: 2.1 Attachments: 6640.diff With the current implementation, the method update from SecondaryIndexManager forces to insert and delete a cell. That happens because we assume that we need the value of the old cell in order to locate the cell that we are updating in our custom secondary index implementation. However, depending on the implementation, a insert and a delete operations could have much worse performance than a simple update. Moreover, if our custom secondary index doesn't use inverted indexes, we don't really need the old cell information and the key information is enough. Therefore, a good solution would be to make the update method more abstract. Thus, the update method for PerColumnSecondaryIndex would receive also the old cell information and from that point we could decide if we must carry out the delete+insert operation or just a update operation. I attach a patch that implements this solution. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6638) SSTableScanner can Skip Rows with vnodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886599#comment-13886599 ] Sylvain Lebresne commented on CASSANDRA-6638: - bq. And then a loop is started over all the ifile entries until end of ifile file (there is no upper boundary check)! There is an upper check, we compare each deserialized index key to the range we're seeking to the start of. If we've seek to a key in the third range, that index key will sort after the 2nd range start and we'll exist the loop right away. SSTableScanner can Skip Rows with vnodes Key: CASSANDRA-6638 URL: https://issues.apache.org/jira/browse/CASSANDRA-6638 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Blocker Fix For: 2.0.5 Attachments: 6638-repro-test.txt, 6638.txt CASSANDRA-2524 added multiple range support to SSTableScanner, but it looks like there is at least one case where keys can be skipped. This can result in cleanup removing legitimate keys. See the attached patch that adds a unit test to reproduce the case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6638) SSTableScanner can Skip Rows with vnodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886624#comment-13886624 ] Ignace Desimpel commented on CASSANDRA-6638: Sorry, my mistake. SSTableScanner can Skip Rows with vnodes Key: CASSANDRA-6638 URL: https://issues.apache.org/jira/browse/CASSANDRA-6638 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Blocker Fix For: 2.0.5 Attachments: 6638-repro-test.txt, 6638.txt CASSANDRA-2524 added multiple range support to SSTableScanner, but it looks like there is at least one case where keys can be skipped. This can result in cleanup removing legitimate keys. See the attached patch that adds a unit test to reproduce the case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6630) Replace UnsortedColumns with ArrayBackedSortedColumns
[ https://issues.apache.org/jira/browse/CASSANDRA-6630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886648#comment-13886648 ] Sylvain Lebresne commented on CASSANDRA-6630: - +1 Nit: AtomicBTreeColumns is missing a space between 'insert' and '='. Replace UnsortedColumns with ArrayBackedSortedColumns - Key: CASSANDRA-6630 URL: https://issues.apache.org/jira/browse/CASSANDRA-6630 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.1 Attachments: 6630.txt It's possible for a counter mutation to have several CounterUpdateCell-s with the same cell name. Those should be summed up and not ignored. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6630) Replace UnsortedColumns with ArrayBackedSortedColumns
[ https://issues.apache.org/jira/browse/CASSANDRA-6630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886668#comment-13886668 ] Benedict commented on CASSANDRA-6630: - If I'm reading this correctly, you're just using ArrayList.add() to shift the previously inserted cells around. This could be severely pathological behaviour for large modification statements. Admittedly such large statements should be uncommon, but why not add everything in unsorted order and then sort/merge prior to a call to any of the accessors? Replace UnsortedColumns with ArrayBackedSortedColumns - Key: CASSANDRA-6630 URL: https://issues.apache.org/jira/browse/CASSANDRA-6630 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.1 Attachments: 6630.txt It's possible for a counter mutation to have several CounterUpdateCell-s with the same cell name. Those should be summed up and not ignored. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6107) CQL3 Batch statement memory leak
[ https://issues.apache.org/jira/browse/CASSANDRA-6107?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-6107: -- Reviewer: Jonathan Ellis CQL3 Batch statement memory leak Key: CASSANDRA-6107 URL: https://issues.apache.org/jira/browse/CASSANDRA-6107 Project: Cassandra Issue Type: Bug Components: API, Core Environment: - CASS version: 1.2.8 or 2.0.1, same issue seen in both - Running on OSX MacbookPro - Sun JVM 1.7 - Single local cassandra node - both CMS and G1 GC used - we are using the cass-JDBC driver to submit our batches Reporter: Constance Eustace Assignee: Lyuben Todorov Priority: Minor Fix For: 1.2.11 Attachments: 6107-v4.txt, 6107.patch, 6107_v2.patch, 6107_v3.patch, Screen Shot 2013-10-03 at 17.59.37.png We are doing large volume insert/update tests on a CASS via CQL3. Using 4GB heap, after roughly 750,000 updates create/update 75,000 row keys, we run out of heap, and it never dissipates, and we begin getting this infamous error which many people seem to be encountering: WARN [ScheduledTasks:1] 2013-09-26 16:17:10,752 GCInspector.java (line 142) Heap is 0.9383457210434385 full. You may need to reduce memtable and/or cache sizes. Cassandra will now flush up to the two largest memtables to free up memory. Adjust flush_largest_memtables_at threshold in cassandra.yaml if you don't want Cassandra to do this automatically INFO [ScheduledTasks:1] 2013-09-26 16:17:10,753 StorageService.java (line 3614) Unable to reduce heap usage since there are no dirty column families 8 and 12 GB heaps appear to delay the problem by roughly proportionate amounts of 75,000 - 100,000 rowkeys per 4GB. Each run of 50,000 row key creations sees the heap grow and never shrink again. We have attempted to no effect: - removing all secondary indexes to see if that alleviates overuse of bloom filters - adjusted parameters for compaction throughput - adjusted memtable flush thresholds and other parameters By examining heapdumps, it seems apparent that the problem is perpetual retention of CQL3 BATCH statements. We have even tried dropping the keyspaces after the updates and the CQL3 statement are still visible in the heapdump, and after many many many CMS GC runs. G1 also showed this issue. The 750,000 statements are broken into batches of roughly 200 statements. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6107) CQL3 Batch statement memory leak
[ https://issues.apache.org/jira/browse/CASSANDRA-6107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886716#comment-13886716 ] Jonathan Ellis commented on CASSANDRA-6107: --- Note: this was reverted in 1.2.14 because of CASSANDRA-6592. CQL3 Batch statement memory leak Key: CASSANDRA-6107 URL: https://issues.apache.org/jira/browse/CASSANDRA-6107 Project: Cassandra Issue Type: Bug Components: API, Core Environment: - CASS version: 1.2.8 or 2.0.1, same issue seen in both - Running on OSX MacbookPro - Sun JVM 1.7 - Single local cassandra node - both CMS and G1 GC used - we are using the cass-JDBC driver to submit our batches Reporter: Constance Eustace Assignee: Lyuben Todorov Priority: Minor Fix For: 1.2.11 Attachments: 6107-v4.txt, 6107.patch, 6107_v2.patch, 6107_v3.patch, Screen Shot 2013-10-03 at 17.59.37.png We are doing large volume insert/update tests on a CASS via CQL3. Using 4GB heap, after roughly 750,000 updates create/update 75,000 row keys, we run out of heap, and it never dissipates, and we begin getting this infamous error which many people seem to be encountering: WARN [ScheduledTasks:1] 2013-09-26 16:17:10,752 GCInspector.java (line 142) Heap is 0.9383457210434385 full. You may need to reduce memtable and/or cache sizes. Cassandra will now flush up to the two largest memtables to free up memory. Adjust flush_largest_memtables_at threshold in cassandra.yaml if you don't want Cassandra to do this automatically INFO [ScheduledTasks:1] 2013-09-26 16:17:10,753 StorageService.java (line 3614) Unable to reduce heap usage since there are no dirty column families 8 and 12 GB heaps appear to delay the problem by roughly proportionate amounts of 75,000 - 100,000 rowkeys per 4GB. Each run of 50,000 row key creations sees the heap grow and never shrink again. We have attempted to no effect: - removing all secondary indexes to see if that alleviates overuse of bloom filters - adjusted parameters for compaction throughput - adjusted memtable flush thresholds and other parameters By examining heapdumps, it seems apparent that the problem is perpetual retention of CQL3 BATCH statements. We have even tried dropping the keyspaces after the updates and the CQL3 statement are still visible in the heapdump, and after many many many CMS GC runs. G1 also showed this issue. The 750,000 statements are broken into batches of roughly 200 statements. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[1/3] git commit: revert CASSANDRA-6107, and cut max cached statements to 50k
Updated Branches: refs/heads/cassandra-1.2 6c4333e3d - 852e27f2e refs/heads/cassandra-2.0 39bdb2be9 - cd91cf974 revert CASSANDRA-6107, and cut max cached statements to 50k Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/852e27f2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/852e27f2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/852e27f2 Branch: refs/heads/cassandra-1.2 Commit: 852e27f2e8f8d6d1e785baf0d834ebfd1d60ca27 Parents: 6c4333e Author: Jonathan Ellis jbel...@apache.org Authored: Thu Jan 30 10:16:08 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Thu Jan 30 10:16:15 2014 -0600 -- CHANGES.txt | 1 + .../apache/cassandra/cql3/QueryProcessor.java | 81 +++- 2 files changed, 10 insertions(+), 72 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/852e27f2/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d88bb26..110bf50 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.14 + * Reverted code to limit CQL prepared statement cache by size (CASSANDRA-6592) * add cassandra.default_messaging_version property to allow easier upgrading from 1.1 (CASSANDRA-6619) * Allow executing CREATE statements multiple times (CASSANDRA-6471) http://git-wip-us.apache.org/repos/asf/cassandra/blob/852e27f2/src/java/org/apache/cassandra/cql3/QueryProcessor.java -- diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java index dc94a18..67ecefc 100644 --- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java +++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java @@ -20,12 +20,8 @@ package org.apache.cassandra.cql3; import java.nio.ByteBuffer; import java.util.*; -import com.google.common.primitives.Ints; - import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap; -import com.googlecode.concurrentlinkedhashmap.EntryWeigher; import org.antlr.runtime.*; -import org.github.jamm.MemoryMeter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,56 +41,15 @@ public class QueryProcessor public static final SemanticVersion CQL_VERSION = new SemanticVersion(3.0.5); private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class); -private static final MemoryMeter meter = new MemoryMeter(); -private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256; -private static final int MAX_CACHE_PREPARED_COUNT = 1; - -private static EntryWeigherMD5Digest, CQLStatement cqlMemoryUsageWeigher = new EntryWeigherMD5Digest, CQLStatement() -{ -@Override -public int weightOf(MD5Digest key, CQLStatement value) -{ -return Ints.checkedCast(measure(key) + measure(value)); -} -}; -private static EntryWeigherInteger, CQLStatement thriftMemoryUsageWeigher = new EntryWeigherInteger, CQLStatement() -{ -@Override -public int weightOf(Integer key, CQLStatement value) -{ -return Ints.checkedCast(measure(key) + measure(value)); -} -}; +public static final int MAX_CACHE_PREPARED = 5; // Enough to keep buggy clients from OOM'ing us +private static final MapMD5Digest, CQLStatement preparedStatements = new ConcurrentLinkedHashMap.BuilderMD5Digest, CQLStatement() + .maximumWeightedCapacity(MAX_CACHE_PREPARED) + .build(); -private static final ConcurrentLinkedHashMapMD5Digest, CQLStatement preparedStatements; -private static final ConcurrentLinkedHashMapInteger, CQLStatement thriftPreparedStatements; - -static -{ -if (MemoryMeter.isInitialized()) -{ -preparedStatements = new ConcurrentLinkedHashMap.BuilderMD5Digest, CQLStatement() - .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) - .weigher(cqlMemoryUsageWeigher) - .build(); -thriftPreparedStatements = new ConcurrentLinkedHashMap.BuilderInteger, CQLStatement() - .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) - .weigher(thriftMemoryUsageWeigher) - .build(); -} -else -{ -logger.error(Unable to initialize MemoryMeter (jamm not specified
[3/3] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cd91cf97 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cd91cf97 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cd91cf97 Branch: refs/heads/cassandra-2.0 Commit: cd91cf9746adf93788aa2cca76f8430e33216300 Parents: 39bdb2b 852e27f Author: Jonathan Ellis jbel...@apache.org Authored: Thu Jan 30 10:16:27 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Thu Jan 30 10:16:27 2014 -0600 -- --
[2/3] git commit: revert CASSANDRA-6107, and cut max cached statements to 50k
revert CASSANDRA-6107, and cut max cached statements to 50k Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/852e27f2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/852e27f2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/852e27f2 Branch: refs/heads/cassandra-2.0 Commit: 852e27f2e8f8d6d1e785baf0d834ebfd1d60ca27 Parents: 6c4333e Author: Jonathan Ellis jbel...@apache.org Authored: Thu Jan 30 10:16:08 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Thu Jan 30 10:16:15 2014 -0600 -- CHANGES.txt | 1 + .../apache/cassandra/cql3/QueryProcessor.java | 81 +++- 2 files changed, 10 insertions(+), 72 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/852e27f2/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d88bb26..110bf50 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.14 + * Reverted code to limit CQL prepared statement cache by size (CASSANDRA-6592) * add cassandra.default_messaging_version property to allow easier upgrading from 1.1 (CASSANDRA-6619) * Allow executing CREATE statements multiple times (CASSANDRA-6471) http://git-wip-us.apache.org/repos/asf/cassandra/blob/852e27f2/src/java/org/apache/cassandra/cql3/QueryProcessor.java -- diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java index dc94a18..67ecefc 100644 --- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java +++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java @@ -20,12 +20,8 @@ package org.apache.cassandra.cql3; import java.nio.ByteBuffer; import java.util.*; -import com.google.common.primitives.Ints; - import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap; -import com.googlecode.concurrentlinkedhashmap.EntryWeigher; import org.antlr.runtime.*; -import org.github.jamm.MemoryMeter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,56 +41,15 @@ public class QueryProcessor public static final SemanticVersion CQL_VERSION = new SemanticVersion(3.0.5); private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class); -private static final MemoryMeter meter = new MemoryMeter(); -private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256; -private static final int MAX_CACHE_PREPARED_COUNT = 1; - -private static EntryWeigherMD5Digest, CQLStatement cqlMemoryUsageWeigher = new EntryWeigherMD5Digest, CQLStatement() -{ -@Override -public int weightOf(MD5Digest key, CQLStatement value) -{ -return Ints.checkedCast(measure(key) + measure(value)); -} -}; -private static EntryWeigherInteger, CQLStatement thriftMemoryUsageWeigher = new EntryWeigherInteger, CQLStatement() -{ -@Override -public int weightOf(Integer key, CQLStatement value) -{ -return Ints.checkedCast(measure(key) + measure(value)); -} -}; +public static final int MAX_CACHE_PREPARED = 5; // Enough to keep buggy clients from OOM'ing us +private static final MapMD5Digest, CQLStatement preparedStatements = new ConcurrentLinkedHashMap.BuilderMD5Digest, CQLStatement() + .maximumWeightedCapacity(MAX_CACHE_PREPARED) + .build(); -private static final ConcurrentLinkedHashMapMD5Digest, CQLStatement preparedStatements; -private static final ConcurrentLinkedHashMapInteger, CQLStatement thriftPreparedStatements; - -static -{ -if (MemoryMeter.isInitialized()) -{ -preparedStatements = new ConcurrentLinkedHashMap.BuilderMD5Digest, CQLStatement() - .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) - .weigher(cqlMemoryUsageWeigher) - .build(); -thriftPreparedStatements = new ConcurrentLinkedHashMap.BuilderInteger, CQLStatement() - .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) - .weigher(thriftMemoryUsageWeigher) - .build(); -} -else -{ -logger.error(Unable to initialize MemoryMeter (jamm not specified as javaagent). This means - + Cassandra will be unable to measure object sizes accurately
[jira] [Commented] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886721#comment-13886721 ] Jonathan Ellis commented on CASSANDRA-6592: --- Solved for 1.2.14 by reverting CASSANDRA-6107 (and reducing the statement count from 100k to 50k to try to avoid the original OOM). IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 0001-Remove-concurrent-use-of-MemoryMeter-instance.txt, 0001-Switch-to-adding-fields-manually-in-measureForPrepared.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6638) SSTableScanner can Skip Rows with vnodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886726#comment-13886726 ] Tyler Hobbs commented on CASSANDRA-6638: +1 on the patch with a minor nit: can you remove the this will currently fail comment and mention 6638 in the new test? SSTableScanner can Skip Rows with vnodes Key: CASSANDRA-6638 URL: https://issues.apache.org/jira/browse/CASSANDRA-6638 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Blocker Fix For: 2.0.5 Attachments: 6638-repro-test.txt, 6638.txt CASSANDRA-2524 added multiple range support to SSTableScanner, but it looks like there is at least one case where keys can be skipped. This can result in cleanup removing legitimate keys. See the attached patch that adds a unit test to reproduce the case. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6630) Replace UnsortedColumns with ArrayBackedSortedColumns
[ https://issues.apache.org/jira/browse/CASSANDRA-6630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886734#comment-13886734 ] Sylvain Lebresne commented on CASSANDRA-6630: - bq. This could be severely pathological behaviour for large modification statements. It might not be optimal for really large modifications, true. severely pathological sounds a bit like over-dramatization for effect though imo (especially since really large modifications is not what we particularly optimize for so should be rather uncommon). Anyway. bq. why not add everything in unsorted order and then sort/merge prior to a call to any of the accessors? Why not, though I'll note this could easily back-fire performance wise if some accesses are intermingled with adds. I don't think we do that with ABSC, at least not in place where it would be a big performance problem, but it feels like something that can be easily introduced by accident later. Not a huge deal I guess, but if we're gonna end up with an implementation for which performance is a little bit harder to reason about, it could be worth asserting what actual benefits it gets us? Maybe in a followup ticket. Replace UnsortedColumns with ArrayBackedSortedColumns - Key: CASSANDRA-6630 URL: https://issues.apache.org/jira/browse/CASSANDRA-6630 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.1 Attachments: 6630.txt It's possible for a counter mutation to have several CounterUpdateCell-s with the same cell name. Those should be summed up and not ignored. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6630) Replace UnsortedColumns with ArrayBackedSortedColumns
[ https://issues.apache.org/jira/browse/CASSANDRA-6630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886750#comment-13886750 ] Benedict commented on CASSANDRA-6630: - bq. Why not, though I'll note this could easily back-fire performance wise if some accesses are intermingled with adds I don't see how it would end up worse. If you track how far you've sorted the list to, you just sort the end unsorted portion, and then merge it into the sorted portion. Worst case behaviour is the same as currently, algorithmically. Might need to do something clever to avoid allocating extra memory when merging (like, we might have to manually shift instead of relying on ArrayList), but nothing too dramatic. bq. severely pathological sounds a bit like over-dramatization Well, let's say pathological. If you have a thousand updates, it will require a 500k ops, which may be on the order of 100 cycles each. This is a latency of 25ms just for this portion. Let's say somebody creates a statement with 10k columns (not crazy, but not common); you're now talking 2.5s. This is pathological in my book. Replace UnsortedColumns with ArrayBackedSortedColumns - Key: CASSANDRA-6630 URL: https://issues.apache.org/jira/browse/CASSANDRA-6630 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.1 Attachments: 6630.txt It's possible for a counter mutation to have several CounterUpdateCell-s with the same cell name. Those should be summed up and not ignored. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6630) Replace UnsortedColumns with ArrayBackedSortedColumns
[ https://issues.apache.org/jira/browse/CASSANDRA-6630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886786#comment-13886786 ] Sylvain Lebresne commented on CASSANDRA-6630: - bq. If you track how far you've sorted the list to, you just sort the end unsorted portion, and then merge it into the sorted portion Fair enough. That's just not a 2 lines change so I'd still go ahead committing as is and leave that to another ticket, but I'll let aleksey decide how he wants to do it. Replace UnsortedColumns with ArrayBackedSortedColumns - Key: CASSANDRA-6630 URL: https://issues.apache.org/jira/browse/CASSANDRA-6630 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.1 Attachments: 6630.txt It's possible for a counter mutation to have several CounterUpdateCell-s with the same cell name. Those should be summed up and not ignored. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
git commit: Fix SSTableScanner potentially skipping rows during cleanup
Updated Branches: refs/heads/cassandra-2.0 cd91cf974 - 81fcc33d9 Fix SSTableScanner potentially skipping rows during cleanup patch by slebresne; reviewed by thobbs for CASSANDRA-6638 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/81fcc33d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/81fcc33d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/81fcc33d Branch: refs/heads/cassandra-2.0 Commit: 81fcc33d9d483d27d38348f7aed582f3621cedbd Parents: cd91cf9 Author: Sylvain Lebresne sylv...@datastax.com Authored: Thu Jan 30 18:17:16 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Thu Jan 30 18:19:34 2014 +0100 -- CHANGES.txt | 1 + .../cassandra/io/sstable/SSTableScanner.java| 6 .../io/sstable/SSTableScannerTest.java | 30 3 files changed, 37 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/81fcc33d/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index a059d84..531ac15 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -15,6 +15,7 @@ * Fix ArrayIndexOutOfBoundsException in 2ndary index query (CASSANDRA-6470) * Release sstables upon rebuilding 2i (CASSANDRA-6635) * Add AbstractCompactionStrategy.startup() method (CASSANDRA-6637) + * SSTableScanner may skip rows during cleanup (CASSANDRA-6638) Merged from 1.2: * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) http://git-wip-us.apache.org/repos/asf/cassandra/blob/81fcc33d/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java index 703b6e9..7a87879 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java @@ -116,7 +116,13 @@ public class SSTableScanner implements ICompactionScanner long indexPosition = sstable.getIndexScanPosition(currentRange.left); // -1 means the key is before everything in the sstable. So just start from the beginning. if (indexPosition == -1) +{ +// Note: this method shouldn't assume we're at the start of the sstable already (see #6638) and +// the seeks are no-op anyway if we are. +ifile.seek(0); +dfile.seek(0); return; +} ifile.seek(indexPosition); try http://git-wip-us.apache.org/repos/asf/cassandra/blob/81fcc33d/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java -- diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java index c17fb71..67d9d2c 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java @@ -284,4 +284,34 @@ public class SSTableScannerTest extends SchemaLoader scanner = sstable.getScanner(new ArrayListRangeToken(), null); assertFalse(scanner.hasNext()); } + +@Test +public void testSingleKeyMultipleRanges() +{ +Keyspace keyspace = Keyspace.open(KEYSPACE); +ColumnFamilyStore store = keyspace.getColumnFamilyStore(TABLE); +store.clearUnsafe(); + +// disable compaction while flushing +store.disableAutoCompaction(); + +insertRowWithKey(205); +store.forceBlockingFlush(); + +assertEquals(1, store.getSSTables().size()); +SSTableReader sstable = store.getSSTables().iterator().next(); + +// full range scan +SSTableScanner fullScanner = sstable.getScanner(); +assertScanContainsRanges(fullScanner, 205, 205); + +// scan three ranges separately +ICompactionScanner scanner = sstable.getScanner(makeRanges( +101, 109, +201, 209), +null); + +// Test for #6638 bug +assertScanContainsRanges(scanner, 205, 205); +} }
[2/4] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cd91cf97 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cd91cf97 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cd91cf97 Branch: refs/heads/trunk Commit: cd91cf9746adf93788aa2cca76f8430e33216300 Parents: 39bdb2b 852e27f Author: Jonathan Ellis jbel...@apache.org Authored: Thu Jan 30 10:16:27 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Thu Jan 30 10:16:27 2014 -0600 -- --
[4/4] git commit: Merge branch 'cassandra-2.0' into trunk
Merge branch 'cassandra-2.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b5a3303f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5a3303f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5a3303f Branch: refs/heads/trunk Commit: b5a3303f354492b38a74dea3b49d8e916007ed58 Parents: b1cb582 81fcc33 Author: Sylvain Lebresne sylv...@datastax.com Authored: Thu Jan 30 18:21:53 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Thu Jan 30 18:21:53 2014 +0100 -- CHANGES.txt | 1 + .../cassandra/io/sstable/SSTableScanner.java| 6 .../io/sstable/SSTableScannerTest.java | 30 3 files changed, 37 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5a3303f/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5a3303f/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5a3303f/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java --
[3/4] git commit: Fix SSTableScanner potentially skipping rows during cleanup
Fix SSTableScanner potentially skipping rows during cleanup patch by slebresne; reviewed by thobbs for CASSANDRA-6638 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/81fcc33d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/81fcc33d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/81fcc33d Branch: refs/heads/trunk Commit: 81fcc33d9d483d27d38348f7aed582f3621cedbd Parents: cd91cf9 Author: Sylvain Lebresne sylv...@datastax.com Authored: Thu Jan 30 18:17:16 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Thu Jan 30 18:19:34 2014 +0100 -- CHANGES.txt | 1 + .../cassandra/io/sstable/SSTableScanner.java| 6 .../io/sstable/SSTableScannerTest.java | 30 3 files changed, 37 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/81fcc33d/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index a059d84..531ac15 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -15,6 +15,7 @@ * Fix ArrayIndexOutOfBoundsException in 2ndary index query (CASSANDRA-6470) * Release sstables upon rebuilding 2i (CASSANDRA-6635) * Add AbstractCompactionStrategy.startup() method (CASSANDRA-6637) + * SSTableScanner may skip rows during cleanup (CASSANDRA-6638) Merged from 1.2: * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) http://git-wip-us.apache.org/repos/asf/cassandra/blob/81fcc33d/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java index 703b6e9..7a87879 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java @@ -116,7 +116,13 @@ public class SSTableScanner implements ICompactionScanner long indexPosition = sstable.getIndexScanPosition(currentRange.left); // -1 means the key is before everything in the sstable. So just start from the beginning. if (indexPosition == -1) +{ +// Note: this method shouldn't assume we're at the start of the sstable already (see #6638) and +// the seeks are no-op anyway if we are. +ifile.seek(0); +dfile.seek(0); return; +} ifile.seek(indexPosition); try http://git-wip-us.apache.org/repos/asf/cassandra/blob/81fcc33d/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java -- diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java index c17fb71..67d9d2c 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java @@ -284,4 +284,34 @@ public class SSTableScannerTest extends SchemaLoader scanner = sstable.getScanner(new ArrayListRangeToken(), null); assertFalse(scanner.hasNext()); } + +@Test +public void testSingleKeyMultipleRanges() +{ +Keyspace keyspace = Keyspace.open(KEYSPACE); +ColumnFamilyStore store = keyspace.getColumnFamilyStore(TABLE); +store.clearUnsafe(); + +// disable compaction while flushing +store.disableAutoCompaction(); + +insertRowWithKey(205); +store.forceBlockingFlush(); + +assertEquals(1, store.getSSTables().size()); +SSTableReader sstable = store.getSSTables().iterator().next(); + +// full range scan +SSTableScanner fullScanner = sstable.getScanner(); +assertScanContainsRanges(fullScanner, 205, 205); + +// scan three ranges separately +ICompactionScanner scanner = sstable.getScanner(makeRanges( +101, 109, +201, 209), +null); + +// Test for #6638 bug +assertScanContainsRanges(scanner, 205, 205); +} }
[1/4] git commit: revert CASSANDRA-6107, and cut max cached statements to 50k
Updated Branches: refs/heads/trunk b1cb582a2 - b5a3303f3 revert CASSANDRA-6107, and cut max cached statements to 50k Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/852e27f2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/852e27f2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/852e27f2 Branch: refs/heads/trunk Commit: 852e27f2e8f8d6d1e785baf0d834ebfd1d60ca27 Parents: 6c4333e Author: Jonathan Ellis jbel...@apache.org Authored: Thu Jan 30 10:16:08 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Thu Jan 30 10:16:15 2014 -0600 -- CHANGES.txt | 1 + .../apache/cassandra/cql3/QueryProcessor.java | 81 +++- 2 files changed, 10 insertions(+), 72 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/852e27f2/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d88bb26..110bf50 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.14 + * Reverted code to limit CQL prepared statement cache by size (CASSANDRA-6592) * add cassandra.default_messaging_version property to allow easier upgrading from 1.1 (CASSANDRA-6619) * Allow executing CREATE statements multiple times (CASSANDRA-6471) http://git-wip-us.apache.org/repos/asf/cassandra/blob/852e27f2/src/java/org/apache/cassandra/cql3/QueryProcessor.java -- diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java index dc94a18..67ecefc 100644 --- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java +++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java @@ -20,12 +20,8 @@ package org.apache.cassandra.cql3; import java.nio.ByteBuffer; import java.util.*; -import com.google.common.primitives.Ints; - import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap; -import com.googlecode.concurrentlinkedhashmap.EntryWeigher; import org.antlr.runtime.*; -import org.github.jamm.MemoryMeter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,56 +41,15 @@ public class QueryProcessor public static final SemanticVersion CQL_VERSION = new SemanticVersion(3.0.5); private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class); -private static final MemoryMeter meter = new MemoryMeter(); -private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256; -private static final int MAX_CACHE_PREPARED_COUNT = 1; - -private static EntryWeigherMD5Digest, CQLStatement cqlMemoryUsageWeigher = new EntryWeigherMD5Digest, CQLStatement() -{ -@Override -public int weightOf(MD5Digest key, CQLStatement value) -{ -return Ints.checkedCast(measure(key) + measure(value)); -} -}; -private static EntryWeigherInteger, CQLStatement thriftMemoryUsageWeigher = new EntryWeigherInteger, CQLStatement() -{ -@Override -public int weightOf(Integer key, CQLStatement value) -{ -return Ints.checkedCast(measure(key) + measure(value)); -} -}; +public static final int MAX_CACHE_PREPARED = 5; // Enough to keep buggy clients from OOM'ing us +private static final MapMD5Digest, CQLStatement preparedStatements = new ConcurrentLinkedHashMap.BuilderMD5Digest, CQLStatement() + .maximumWeightedCapacity(MAX_CACHE_PREPARED) + .build(); -private static final ConcurrentLinkedHashMapMD5Digest, CQLStatement preparedStatements; -private static final ConcurrentLinkedHashMapInteger, CQLStatement thriftPreparedStatements; - -static -{ -if (MemoryMeter.isInitialized()) -{ -preparedStatements = new ConcurrentLinkedHashMap.BuilderMD5Digest, CQLStatement() - .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) - .weigher(cqlMemoryUsageWeigher) - .build(); -thriftPreparedStatements = new ConcurrentLinkedHashMap.BuilderInteger, CQLStatement() - .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY) - .weigher(thriftMemoryUsageWeigher) - .build(); -} -else -{ -logger.error(Unable to initialize MemoryMeter (jamm not specified as javaagent). This means - + Cassandra
git commit: sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503
Updated Branches: refs/heads/cassandra-1.2 852e27f2e - 6ad995e8f sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6ad995e8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ad995e8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ad995e8 Branch: refs/heads/cassandra-1.2 Commit: 6ad995e8fa7703e082ea9ce67dc4c1ed0b1fd18a Parents: 852e27f Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 09:51:42 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 09:51:42 2014 -0800 -- CHANGES.txt| 1 + .../cassandra/streaming/IncomingStreamReader.java | 8 .../apache/cassandra/streaming/StreamInSession.java| 13 + 3 files changed, 14 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 110bf50..d85d3a4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -21,6 +21,7 @@ * Fix preparing with batch and delete from collection (CASSANDRA-6607) * Fix ABSC reverse iterator's remove() method (CASSANDRA-6629) * Handle host ID conflicts properly (CASSANDRA-6615) + * sstables from stalled repair sessions can resurrect deleted data (CASSANDRA-6503) 1.2.13 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java -- diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index 0b058fc..940f8de 100644 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@ -119,8 +119,8 @@ public class IncomingStreamReader DataInput dis = new DataInputStream(underliningStream); try { -SSTableReader reader = streamIn(dis, localFile, remoteFile); -session.finished(remoteFile, reader); +SSTableWriter writer = streamIn(dis, localFile, remoteFile); +session.finished(remoteFile, writer); } catch (IOException ex) { @@ -141,7 +141,7 @@ public class IncomingStreamReader /** * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. */ -private SSTableReader streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException +private SSTableWriter streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException { ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname); DecoratedKey key; @@ -197,7 +197,7 @@ public class IncomingStreamReader } StreamingMetrics.totalIncomingBytes.inc(totalBytesRead); metrics.incomingBytes.inc(totalBytesRead); -return writer.closeAndOpenReader(); +return writer; } catch (Throwable e) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/src/java/org/apache/cassandra/streaming/StreamInSession.java -- diff --git a/src/java/org/apache/cassandra/streaming/StreamInSession.java b/src/java/org/apache/cassandra/streaming/StreamInSession.java index 96c31da..e83a5b6 100644 --- a/src/java/org/apache/cassandra/streaming/StreamInSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamInSession.java @@ -24,6 +24,7 @@ import java.net.Socket; import java.util.*; import java.util.concurrent.ConcurrentMap; +import org.apache.cassandra.io.sstable.SSTableWriter; import org.cliffc.high_scale_lib.NonBlockingHashMap; import org.cliffc.high_scale_lib.NonBlockingHashSet; import org.slf4j.Logger; @@ -47,7 +48,7 @@ public class StreamInSession extends AbstractStreamSession private static final ConcurrentMapUUID, StreamInSession sessions = new NonBlockingHashMapUUID, StreamInSession(); private final SetPendingFile files = new NonBlockingHashSetPendingFile(); -private final ListSSTableReader readers = new ArrayListSSTableReader(); +private final ListSSTableWriter writers = new ArrayListSSTableWriter(); private PendingFile current; private Socket socket; private volatile int retries; @@ -106,13 +107,13 @@ public class
[1/2] git commit: sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503
Updated Branches: refs/heads/cassandra-2.0 81fcc33d9 - b5a2b6507 sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6ad995e8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ad995e8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ad995e8 Branch: refs/heads/cassandra-2.0 Commit: 6ad995e8fa7703e082ea9ce67dc4c1ed0b1fd18a Parents: 852e27f Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 09:51:42 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 09:51:42 2014 -0800 -- CHANGES.txt| 1 + .../cassandra/streaming/IncomingStreamReader.java | 8 .../apache/cassandra/streaming/StreamInSession.java| 13 + 3 files changed, 14 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 110bf50..d85d3a4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -21,6 +21,7 @@ * Fix preparing with batch and delete from collection (CASSANDRA-6607) * Fix ABSC reverse iterator's remove() method (CASSANDRA-6629) * Handle host ID conflicts properly (CASSANDRA-6615) + * sstables from stalled repair sessions can resurrect deleted data (CASSANDRA-6503) 1.2.13 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java -- diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index 0b058fc..940f8de 100644 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@ -119,8 +119,8 @@ public class IncomingStreamReader DataInput dis = new DataInputStream(underliningStream); try { -SSTableReader reader = streamIn(dis, localFile, remoteFile); -session.finished(remoteFile, reader); +SSTableWriter writer = streamIn(dis, localFile, remoteFile); +session.finished(remoteFile, writer); } catch (IOException ex) { @@ -141,7 +141,7 @@ public class IncomingStreamReader /** * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. */ -private SSTableReader streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException +private SSTableWriter streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException { ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname); DecoratedKey key; @@ -197,7 +197,7 @@ public class IncomingStreamReader } StreamingMetrics.totalIncomingBytes.inc(totalBytesRead); metrics.incomingBytes.inc(totalBytesRead); -return writer.closeAndOpenReader(); +return writer; } catch (Throwable e) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/src/java/org/apache/cassandra/streaming/StreamInSession.java -- diff --git a/src/java/org/apache/cassandra/streaming/StreamInSession.java b/src/java/org/apache/cassandra/streaming/StreamInSession.java index 96c31da..e83a5b6 100644 --- a/src/java/org/apache/cassandra/streaming/StreamInSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamInSession.java @@ -24,6 +24,7 @@ import java.net.Socket; import java.util.*; import java.util.concurrent.ConcurrentMap; +import org.apache.cassandra.io.sstable.SSTableWriter; import org.cliffc.high_scale_lib.NonBlockingHashMap; import org.cliffc.high_scale_lib.NonBlockingHashSet; import org.slf4j.Logger; @@ -47,7 +48,7 @@ public class StreamInSession extends AbstractStreamSession private static final ConcurrentMapUUID, StreamInSession sessions = new NonBlockingHashMapUUID, StreamInSession(); private final SetPendingFile files = new NonBlockingHashSetPendingFile(); -private final ListSSTableReader readers = new ArrayListSSTableReader(); +private final ListSSTableWriter writers = new ArrayListSSTableWriter(); private PendingFile current; private Socket socket; private volatile int retries; @@ -106,13 +107,13 @@ public class
[2/2] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b5a2b650 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5a2b650 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5a2b650 Branch: refs/heads/cassandra-2.0 Commit: b5a2b65076f86ee03ccd6083a6ccafae0c4c6f1d Parents: 81fcc33 6ad995e Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 09:58:36 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 09:58:36 2014 -0800 -- --
[jira] [Updated] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6592: Attachment: 6592-2.0.txt Summing up where we are: * Tyler tested the 2nd patch which seems to fix the issue. * It seems MemoryMeter can return pretty funky results (~100MB for a statement that Tyler confirmed as really small) when measuring CFMetaData. While there is some theory, it's no yet entirely clear why it can produce such big error. * While the 2nd patch is likely fine, given the previous point and to take 0 risk with 1.2, we're reverted CASSANDRA-6107 there. Now, for 2.0, attaching a rebase of the patch with correct handling of null values (which was missing from the first patch -- could maybe be slightly cleaner to make MemoryMeter.measureDeep handle null on the long run). I'll note that while the patch seems to properly fix this issue, it might be worth digging a bit more separatly to understand what was confusing MemoryMeter exactly, since we use it in other places. IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 0001-Remove-concurrent-use-of-MemoryMeter-instance.txt, 0001-Switch-to-adding-fields-manually-in-measureForPrepared.txt, 6592-2.0.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[1/3] git commit: sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503
Updated Branches: refs/heads/trunk b5a3303f3 - 00fc31868 sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6ad995e8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ad995e8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ad995e8 Branch: refs/heads/trunk Commit: 6ad995e8fa7703e082ea9ce67dc4c1ed0b1fd18a Parents: 852e27f Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 09:51:42 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 09:51:42 2014 -0800 -- CHANGES.txt| 1 + .../cassandra/streaming/IncomingStreamReader.java | 8 .../apache/cassandra/streaming/StreamInSession.java| 13 + 3 files changed, 14 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 110bf50..d85d3a4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -21,6 +21,7 @@ * Fix preparing with batch and delete from collection (CASSANDRA-6607) * Fix ABSC reverse iterator's remove() method (CASSANDRA-6629) * Handle host ID conflicts properly (CASSANDRA-6615) + * sstables from stalled repair sessions can resurrect deleted data (CASSANDRA-6503) 1.2.13 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java -- diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index 0b058fc..940f8de 100644 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@ -119,8 +119,8 @@ public class IncomingStreamReader DataInput dis = new DataInputStream(underliningStream); try { -SSTableReader reader = streamIn(dis, localFile, remoteFile); -session.finished(remoteFile, reader); +SSTableWriter writer = streamIn(dis, localFile, remoteFile); +session.finished(remoteFile, writer); } catch (IOException ex) { @@ -141,7 +141,7 @@ public class IncomingStreamReader /** * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. */ -private SSTableReader streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException +private SSTableWriter streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException { ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname); DecoratedKey key; @@ -197,7 +197,7 @@ public class IncomingStreamReader } StreamingMetrics.totalIncomingBytes.inc(totalBytesRead); metrics.incomingBytes.inc(totalBytesRead); -return writer.closeAndOpenReader(); +return writer; } catch (Throwable e) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ad995e8/src/java/org/apache/cassandra/streaming/StreamInSession.java -- diff --git a/src/java/org/apache/cassandra/streaming/StreamInSession.java b/src/java/org/apache/cassandra/streaming/StreamInSession.java index 96c31da..e83a5b6 100644 --- a/src/java/org/apache/cassandra/streaming/StreamInSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamInSession.java @@ -24,6 +24,7 @@ import java.net.Socket; import java.util.*; import java.util.concurrent.ConcurrentMap; +import org.apache.cassandra.io.sstable.SSTableWriter; import org.cliffc.high_scale_lib.NonBlockingHashMap; import org.cliffc.high_scale_lib.NonBlockingHashSet; import org.slf4j.Logger; @@ -47,7 +48,7 @@ public class StreamInSession extends AbstractStreamSession private static final ConcurrentMapUUID, StreamInSession sessions = new NonBlockingHashMapUUID, StreamInSession(); private final SetPendingFile files = new NonBlockingHashSetPendingFile(); -private final ListSSTableReader readers = new ArrayListSSTableReader(); +private final ListSSTableWriter writers = new ArrayListSSTableWriter(); private PendingFile current; private Socket socket; private volatile int retries; @@ -106,13 +107,13 @@ public class StreamInSession extends
[2/3] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b5a2b650 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5a2b650 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5a2b650 Branch: refs/heads/trunk Commit: b5a2b65076f86ee03ccd6083a6ccafae0c4c6f1d Parents: 81fcc33 6ad995e Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 09:58:36 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 09:58:36 2014 -0800 -- --
[jira] [Commented] (CASSANDRA-6503) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data
[ https://issues.apache.org/jira/browse/CASSANDRA-6503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886836#comment-13886836 ] Jason Brown commented on CASSANDRA-6503: Committed the 1.2 patch to 1.2, have a few questions for [~yukim] that I'll here in a minute (after coffee) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data --- Key: CASSANDRA-6503 URL: https://issues.apache.org/jira/browse/CASSANDRA-6503 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Jason Brown Priority: Minor Fix For: 1.2.14, 2.0.5 Attachments: 6503_2.0-v2.diff, 6503_2.0-v3.diff, 6503_c1.2-v1.patch The sstables streamed in during a repair session don't become active until the session finishes. If something causes the repair session to hang for some reason, those sstables will hang around until the next reboot, and become active then. If you don't reboot for 3 months, this can cause data to resurrect, as GC grace has expired, so tombstones for the data in those sstables may have already been collected. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[3/3] git commit: Merge branch 'cassandra-2.0' into trunk
Merge branch 'cassandra-2.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/00fc3186 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/00fc3186 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/00fc3186 Branch: refs/heads/trunk Commit: 00fc31868aa4b002b9f6acbc18b861f6e8f11175 Parents: b5a3303 b5a2b65 Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 10:00:34 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 10:00:34 2014 -0800 -- --
[jira] [Updated] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6592: Attachment: (was: 0001-Switch-to-adding-fields-manually-in-measureForPrepared.txt) IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 6592-2.0.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6592: Reviewer: Tyler Hobbs IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 6592-2.0.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6503) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data
[ https://issues.apache.org/jira/browse/CASSANDRA-6503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886836#comment-13886836 ] Jason Brown edited comment on CASSANDRA-6503 at 1/30/14 6:11 PM: - Committed the 1.2 patch to 1.2, have a few questions for [~yukim] about 2.0 patch that I'll add here in a minute (after coffee) was (Author: jasobrown): Committed the 1.2 patch to 1.2, have a few questions for [~yukim] about 2.0 patch that I'll here in a minute (after coffee) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data --- Key: CASSANDRA-6503 URL: https://issues.apache.org/jira/browse/CASSANDRA-6503 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Jason Brown Priority: Minor Fix For: 1.2.14, 2.0.5 Attachments: 6503_2.0-v2.diff, 6503_2.0-v3.diff, 6503_c1.2-v1.patch The sstables streamed in during a repair session don't become active until the session finishes. If something causes the repair session to hang for some reason, those sstables will hang around until the next reboot, and become active then. If you don't reboot for 3 months, this can cause data to resurrect, as GC grace has expired, so tombstones for the data in those sstables may have already been collected. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6503) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data
[ https://issues.apache.org/jira/browse/CASSANDRA-6503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886836#comment-13886836 ] Jason Brown edited comment on CASSANDRA-6503 at 1/30/14 6:11 PM: - Committed the 1.2 patch to 1.2, have a few questions for [~yukim] about 2.0 patch that I'll here in a minute (after coffee) was (Author: jasobrown): Committed the 1.2 patch to 1.2, have a few questions for [~yukim] that I'll here in a minute (after coffee) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data --- Key: CASSANDRA-6503 URL: https://issues.apache.org/jira/browse/CASSANDRA-6503 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Jason Brown Priority: Minor Fix For: 1.2.14, 2.0.5 Attachments: 6503_2.0-v2.diff, 6503_2.0-v3.diff, 6503_c1.2-v1.patch The sstables streamed in during a repair session don't become active until the session finishes. If something causes the repair session to hang for some reason, those sstables will hang around until the next reboot, and become active then. If you don't reboot for 3 months, this can cause data to resurrect, as GC grace has expired, so tombstones for the data in those sstables may have already been collected. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6592: Attachment: (was: 0001-Remove-concurrent-use-of-MemoryMeter-instance.txt) IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 6592-2.0.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Assigned] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne reassigned CASSANDRA-6592: --- Assignee: Sylvain Lebresne (was: Lyuben Todorov) IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 6592-2.0.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5631) NPE when creating column family shortly after multinode startup
[ https://issues.apache.org/jira/browse/CASSANDRA-5631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886886#comment-13886886 ] Tym Altman commented on CASSANDRA-5631: --- I'm just seeing the exact same thing on my 2 node cluster (cassandra 2.0.4). NPE when creating column family shortly after multinode startup --- Key: CASSANDRA-5631 URL: https://issues.apache.org/jira/browse/CASSANDRA-5631 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.2.0 Reporter: Martin Serrano Assignee: Aleksey Yeschenko I'm testing a 2-node cluster and creating a column family right after the nodes startup. I am using the Astyanax client. Sometimes column family creation fails and I see NPEs on the cassandra server: {noformat} 2013-06-12 14:55:31,773 ERROR CassandraDaemon [MigrationStage:1] - Exception in thread Thread[MigrationStage:1,5,main] java.lang.NullPointerException at org.apache.cassandra.db.DefsTable.addColumnFamily(DefsTable.java:510) at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:444) at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:354) at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:55) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334) at java.util.concurrent.FutureTask.run(FutureTask.java:166) 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:722) {noformat} {noformat} 2013-06-12 14:55:31,880 ERROR CassandraDaemon [MigrationStage:1] - Exception in thread Thread[MigrationStage:1,5,main] java.lang.NullPointerException at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:475) at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:354) at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:55) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334) at java.util.concurrent.FutureTask.run(FutureTask.java:166) 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:722) {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6614) 2 hours loop flushing+compacting system/{schema_keyspaces,schema_columnfamilies,schema_columns} when upgrading
[ https://issues.apache.org/jira/browse/CASSANDRA-6614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886947#comment-13886947 ] Cyril Scetbon commented on CASSANDRA-6614: -- I found that using nodetool drain prevents from having this issue. However nodetool drain should only prevent overcounts of counter data and make the restart faster. And why only with the first node and not with others (maybe system column families have been flushed on others ??) [~jasobrown], did you use nodetool drain ? 2 hours loop flushing+compacting system/{schema_keyspaces,schema_columnfamilies,schema_columns} when upgrading -- Key: CASSANDRA-6614 URL: https://issues.apache.org/jira/browse/CASSANDRA-6614 Project: Cassandra Issue Type: Bug Components: Core Environment: ubuntu 12.04 Reporter: Cyril Scetbon It happens when we upgrade one node to 1.2.13 on a 1.2.2 cluster see http://pastebin.com/YZKUQLXz If I grep for only InternalResponseStage logs I get http://pastebin.com/htnXZCiT which always displays same account of ops and serialized/live bytes per column family. When I upgrade one node from 1.2.2 to 1.2.13, for 2h I get the previous messages with a raise of CPU (as it flushes and compacts continually) on all nodes http://picpaste.com/pics/Screen_Shot_2014-01-24_at_09.18.50-ggcCDVqd.1390587562.png After that, everything is fine and I can upgrade other nodes without any raise of cpus load. when I start the upgrade, the more nodes I upgrade at the same time (at the beginning), the higher the cpu load is http://picpaste.com/pics/Screen_Shot_2014-01-23_at_17.45.56-I3fdEQ2T.1390587597.png -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6503) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data
[ https://issues.apache.org/jira/browse/CASSANDRA-6503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13886961#comment-13886961 ] Jason Brown commented on CASSANDRA-6503: bq. How about using SSTable directory itself? I think that is legit as StreamReceiveTask is specific to a CF. sstables from stalled repair sessions become live after a reboot and can resurrect deleted data --- Key: CASSANDRA-6503 URL: https://issues.apache.org/jira/browse/CASSANDRA-6503 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Jason Brown Priority: Minor Fix For: 1.2.14, 2.0.5 Attachments: 6503_2.0-v2.diff, 6503_2.0-v3.diff, 6503_c1.2-v1.patch The sstables streamed in during a repair session don't become active until the session finishes. If something causes the repair session to hang for some reason, those sstables will hang around until the next reboot, and become active then. If you don't reboot for 3 months, this can cause data to resurrect, as GC grace has expired, so tombstones for the data in those sstables may have already been collected. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6636) sstableloader fails when attempting to load data from a single node into a multi-node cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-6636?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887088#comment-13887088 ] Yuki Morishita commented on CASSANDRA-6636: --- So the root cause I tracked down is actually NPE when showing progress. BulkLoader expects to receive StreamEvent PREPARE to setup progress indicator. But when you get the exception, event listener is registered after streaming emits PREPARE event. I think this is more likely to happen in dev environment like ccm. sstableloader fails when attempting to load data from a single node into a multi-node cluster - Key: CASSANDRA-6636 URL: https://issues.apache.org/jira/browse/CASSANDRA-6636 Project: Cassandra Issue Type: Bug Environment: java version 1.7.0_51 cassandra from cassandra-2.0 branch (0be424...) Reporter: Russ Hatch Assignee: Yuki Morishita I'm running into this exception when trying to use sstableloader to bring in data from another cluster: {noformat} rhatch@whatup:~/.ccm/test_cluster_1391031988/node1$ bin/sstableloader -d 127.0.0.1 ~/tmp/Keyspace1/Standard1 Established connection to initial hosts Opening sstables and calculating sections to stream Streaming relevant part of /home/rhatch/tmp/Keyspace1/Standard1/Keyspace1-Standard1-jb-5-Data.db /home/rhatch/tmp/Keyspace1/Standard1/Keyspace1-Standard1-jb-6-Data.db to [/127.0.0.1, /127.0.0.2, /127.0.0.3] 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) {noformat} This is what I see in the node system.log: {noformat} == ./test_cluster_1391031988/node1/logs/system.log == INFO [STREAM-INIT-/127.0.0.1:60971] 2014-01-29 14:57:25,375 StreamResultFuture.java (line 116) [Stream #564ded70-8930-11e3-84e9-2766c3cc4197] Received streaming plan for Bulk Load INFO [STREAM-IN-/127.0.1.1] 2014-01-29 14:57:25,375 StreamResultFuture.java (line 168) [Stream #564ded70-8930-11e3-84e9-2766c3cc4197] Prepare completed. Receiving 2 files(91047224 bytes), sending 0 files(0 bytes) {noformat} steps to reproduce: create a 3 node cluster with ccm run stress on one node with 'ccm node1 stress' copy the node's data from the data/Keyspace1/Standard1 directory to save it for re-loading (preserve the directory structure for sstableloader) remove the cluster, and create a new 3 node cluster pick a node and run bin/nodetool -d localhost ~/saved_data_location/Keyspace1/Standard1 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6636) sstableloader fails when attempting to load data from a single node into a multi-node cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-6636?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-6636: -- Priority: Minor (was: Major) sstableloader fails when attempting to load data from a single node into a multi-node cluster - Key: CASSANDRA-6636 URL: https://issues.apache.org/jira/browse/CASSANDRA-6636 Project: Cassandra Issue Type: Bug Environment: java version 1.7.0_51 cassandra from cassandra-2.0 branch (0be424...) Reporter: Russ Hatch Assignee: Yuki Morishita Priority: Minor I'm running into this exception when trying to use sstableloader to bring in data from another cluster: {noformat} rhatch@whatup:~/.ccm/test_cluster_1391031988/node1$ bin/sstableloader -d 127.0.0.1 ~/tmp/Keyspace1/Standard1 Established connection to initial hosts Opening sstables and calculating sections to stream Streaming relevant part of /home/rhatch/tmp/Keyspace1/Standard1/Keyspace1-Standard1-jb-5-Data.db /home/rhatch/tmp/Keyspace1/Standard1/Keyspace1-Standard1-jb-6-Data.db to [/127.0.0.1, /127.0.0.2, /127.0.0.3] 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) {noformat} This is what I see in the node system.log: {noformat} == ./test_cluster_1391031988/node1/logs/system.log == INFO [STREAM-INIT-/127.0.0.1:60971] 2014-01-29 14:57:25,375 StreamResultFuture.java (line 116) [Stream #564ded70-8930-11e3-84e9-2766c3cc4197] Received streaming plan for Bulk Load INFO [STREAM-IN-/127.0.1.1] 2014-01-29 14:57:25,375 StreamResultFuture.java (line 168) [Stream #564ded70-8930-11e3-84e9-2766c3cc4197] Prepare completed. Receiving 2 files(91047224 bytes), sending 0 files(0 bytes) {noformat} steps to reproduce: create a 3 node cluster with ccm run stress on one node with 'ccm node1 stress' copy the node's data from the data/Keyspace1/Standard1 directory to save it for re-loading (preserve the directory structure for sstableloader) remove the cluster, and create a new 3 node cluster pick a node and run bin/nodetool -d localhost ~/saved_data_location/Keyspace1/Standard1 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6641) Switch stress to use ITransportFactory to obtain client connections
Sam Tunnicliffe created CASSANDRA-6641: -- Summary: Switch stress to use ITransportFactory to obtain client connections Key: CASSANDRA-6641 URL: https://issues.apache.org/jira/browse/CASSANDRA-6641 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Priority: Minor CASSANDRA-6062 CASSANDRA-6378 changed cassandra-cli and sstableloader respectively to use o.a.c.thrift.ITransportFactory rather than o.a.thrift.transport.TTransportFactory implementations. This ticket is to do likewise for cassandra-stress, so that users can have more control over the connection options when running stress. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6641) Switch stress to use ITransportFactory to obtain client connections
[ https://issues.apache.org/jira/browse/CASSANDRA-6641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-6641: --- Attachment: 0001-cassandra-stress-uses-ITransportFactory-to-provide-c.patch 0002-Set-supplied-encryption-options-on-transport-factory.patch First patch contains the mods to cassandra-stress, the second adds support for encryption options to be passed on the command line to cassandra-cli (which is already supported in sstableloader). Switch stress to use ITransportFactory to obtain client connections --- Key: CASSANDRA-6641 URL: https://issues.apache.org/jira/browse/CASSANDRA-6641 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Priority: Minor Attachments: 0001-cassandra-stress-uses-ITransportFactory-to-provide-c.patch, 0002-Set-supplied-encryption-options-on-transport-factory.patch CASSANDRA-6062 CASSANDRA-6378 changed cassandra-cli and sstableloader respectively to use o.a.c.thrift.ITransportFactory rather than o.a.thrift.transport.TTransportFactory implementations. This ticket is to do likewise for cassandra-stress, so that users can have more control over the connection options when running stress. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6641) Switch stress to use ITransportFactory to obtain client connections
[ https://issues.apache.org/jira/browse/CASSANDRA-6641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887183#comment-13887183 ] Jonathan Ellis commented on CASSANDRA-6641: --- did you check stress-ng? we're likely to put oldstress out of its misery soon. Switch stress to use ITransportFactory to obtain client connections --- Key: CASSANDRA-6641 URL: https://issues.apache.org/jira/browse/CASSANDRA-6641 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Priority: Minor Attachments: 0001-cassandra-stress-uses-ITransportFactory-to-provide-c.patch, 0002-Set-supplied-encryption-options-on-transport-factory.patch CASSANDRA-6062 CASSANDRA-6378 changed cassandra-cli and sstableloader respectively to use o.a.c.thrift.ITransportFactory rather than o.a.thrift.transport.TTransportFactory implementations. This ticket is to do likewise for cassandra-stress, so that users can have more control over the connection options when running stress. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6641) Switch stress to use ITransportFactory to obtain client connections
[ https://issues.apache.org/jira/browse/CASSANDRA-6641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887191#comment-13887191 ] Sam Tunnicliffe commented on CASSANDRA-6641: I didn't as I was targetting this at 2.0, I'll check it out submit a further patch if necessary Switch stress to use ITransportFactory to obtain client connections --- Key: CASSANDRA-6641 URL: https://issues.apache.org/jira/browse/CASSANDRA-6641 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Priority: Minor Attachments: 0001-cassandra-stress-uses-ITransportFactory-to-provide-c.patch, 0002-Set-supplied-encryption-options-on-transport-factory.patch CASSANDRA-6062 CASSANDRA-6378 changed cassandra-cli and sstableloader respectively to use o.a.c.thrift.ITransportFactory rather than o.a.thrift.transport.TTransportFactory implementations. This ticket is to do likewise for cassandra-stress, so that users can have more control over the connection options when running stress. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6642) line 297 of CQL.textile needs updated
Kristine Hahn created CASSANDRA-6642: Summary: line 297 of CQL.textile needs updated Key: CASSANDRA-6642 URL: https://issues.apache.org/jira/browse/CASSANDRA-6642 Project: Cassandra Issue Type: Bug Components: Documentation website Environment: [CQL 3.1.4 doc, cassandra-trunk|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#partition-key-and-clustering] Reporter: Kristine Hahn Priority: Minor The behavior changed in 1.2.6.1 or earlier, but [the doc|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#partition-key-and-clustering] still says: {quote} The restriction for table with COMPACT STORAGE is that they support one and only one column outside of the ones part of the PRIMARY KEY. {quote} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6641) Switch stress to use ITransportFactory to obtain client connections
[ https://issues.apache.org/jira/browse/CASSANDRA-6641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-6641: - Reviewer: Aleksey Yeschenko Switch stress to use ITransportFactory to obtain client connections --- Key: CASSANDRA-6641 URL: https://issues.apache.org/jira/browse/CASSANDRA-6641 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Priority: Minor Attachments: 0001-cassandra-stress-uses-ITransportFactory-to-provide-c.patch, 0002-Set-supplied-encryption-options-on-transport-factory.patch CASSANDRA-6062 CASSANDRA-6378 changed cassandra-cli and sstableloader respectively to use o.a.c.thrift.ITransportFactory rather than o.a.thrift.transport.TTransportFactory implementations. This ticket is to do likewise for cassandra-stress, so that users can have more control over the connection options when running stress. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[1/6] remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549
Updated Branches: refs/heads/trunk 00fc31868 - 4b54b8acd http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java -- diff --git a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java index 185d3b3..cd837c8 100644 --- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java +++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java @@ -35,8 +35,8 @@ import org.apache.cassandra.config.Schema; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.db.composites.*; import org.apache.cassandra.db.filter.ColumnSlice; +import org.apache.cassandra.utils.memory.HeapAllocator; import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.utils.HeapAllocator; public class ArrayBackedSortedColumnsTest extends SchemaLoader { http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java -- diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java index 42f6066..cd79217 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java @@ -37,7 +37,7 @@ import static org.apache.cassandra.Util.cellname; import static org.junit.Assert.assertEquals; import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.HeapAllocator; +import org.apache.cassandra.utils.memory.HeapAllocator; public class ColumnFamilyTest extends SchemaLoader http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/db/CommitLogTest.java -- diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java index b51c17e..577692d 100644 --- a/test/unit/org/apache/cassandra/db/CommitLogTest.java +++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java @@ -117,7 +117,7 @@ public class CommitLogTest extends SchemaLoader assert CommitLog.instance.activeSegments() == 2 : Expecting 2 segments, got + CommitLog.instance.activeSegments(); UUID cfid2 = rm2.getColumnFamilyIds().iterator().next(); -CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext().get()); +CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext()); // Assert we still have both our segment assert CommitLog.instance.activeSegments() == 2 : Expecting 2 segments, got + CommitLog.instance.activeSegments(); @@ -141,7 +141,7 @@ public class CommitLogTest extends SchemaLoader // Flush: this won't delete anything UUID cfid1 = rm.getColumnFamilyIds().iterator().next(); CommitLog.instance.sync(true); -CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext().get()); +CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext()); assert CommitLog.instance.activeSegments() == 1 : Expecting 1 segment, got + CommitLog.instance.activeSegments(); @@ -160,7 +160,7 @@ public class CommitLogTest extends SchemaLoader // didn't write anything on cf1 since last flush (and we flush cf2) UUID cfid2 = rm2.getColumnFamilyIds().iterator().next(); -CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext().get()); +CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext()); // Assert we still have both our segment assert CommitLog.instance.activeSegments() == 1 : Expecting 1 segment, got + CommitLog.instance.activeSegments(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/db/CounterCellTest.java -- diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java index 24d8f1c..6949340 100644 --- a/test/unit/org/apache/cassandra/db/CounterCellTest.java +++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java @@ -27,6 +27,8 @@ import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.Arrays; +import org.apache.cassandra.utils.memory.AbstractAllocator; +import org.apache.cassandra.utils.memory.HeapAllocator; import org.junit.Assert; import org.junit.Test; @@ -87,7 +89,7 @@ public class CounterCellTest extends SchemaLoader ByteBuffer context; -Allocator allocator = HeapAllocator.instance; +AbstractAllocator
[jira] [Commented] (CASSANDRA-5549) Remove Table.switchLock
[ https://issues.apache.org/jira/browse/CASSANDRA-5549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887225#comment-13887225 ] Jonathan Ellis commented on CASSANDRA-5549: --- Committed! Remove Table.switchLock --- Key: CASSANDRA-5549 URL: https://issues.apache.org/jira/browse/CASSANDRA-5549 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Benedict Labels: performance Fix For: 2.1 Attachments: 5549-removed-switchlock.png, 5549-sunnyvale.png As discussed in CASSANDRA-5422, Table.switchLock is a bottleneck on the write path. ReentrantReadWriteLock is not lightweight, even if there is no contention per se between readers and writers of the lock (in Cassandra, memtable updates and switches). -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5549) Remove Table.switchLock
[ https://issues.apache.org/jira/browse/CASSANDRA-5549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887229#comment-13887229 ] Jonathan Ellis commented on CASSANDRA-5549: --- (Please submit a pull request for jamm-0.2.6. Note that jamm uses normal brace-on-same-line convention.) Remove Table.switchLock --- Key: CASSANDRA-5549 URL: https://issues.apache.org/jira/browse/CASSANDRA-5549 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Benedict Labels: performance Fix For: 2.1 Attachments: 5549-removed-switchlock.png, 5549-sunnyvale.png As discussed in CASSANDRA-5422, Table.switchLock is a bottleneck on the write path. ReentrantReadWriteLock is not lightweight, even if there is no contention per se between readers and writers of the lock (in Cassandra, memtable updates and switches). -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6639) Update Guava to version 16
[ https://issues.apache.org/jira/browse/CASSANDRA-6639?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mikhail Stepura updated CASSANDRA-6639: --- Attachment: trunk-6639.patch Patch. Had to remove {{airline}} from {{build-deps-pom}} (it for sure remains in {{dependencyManagement}} ), because it pulled Guava 12.0. Update Guava to version 16 -- Key: CASSANDRA-6639 URL: https://issues.apache.org/jira/browse/CASSANDRA-6639 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Mikhail Mazursky Assignee: Mikhail Stepura Priority: Trivial Fix For: 2.1 Attachments: trunk-6639.patch Currently C* uses Guava 15. I tried to update my code to use Guava 16 and my integration tests, that use C*, started to produce the following traces: {noformat} [INFO ] 10:00:12.600 [CompactionExecutor:2][][] ERROR CassandraDaemon:187 - Exception in thread Thread[CompactionExecutor:2,1,main] [INFO ] java.lang.NoSuchMethodError: com.google.common.util.concurrent.RateLimiter.acquire(I)V [INFO ] at org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:40) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:280) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.computeNext(SSTableScanner.java:256) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.computeNext(SSTableScanner.java:197) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na] [INFO ] at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na] [INFO ] at org.apache.cassandra.io.sstable.SSTableScanner.hasNext(SSTableScanner.java:177) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:144) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.utils.MergeIterator$ManyToOne.init(MergeIterator.java:87) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:46) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.db.compaction.CompactionIterable.iterator(CompactionIterable.java:47) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:129) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197) ~[cassandra-all-2.0.4.jar:2.0.4] [INFO ] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_51] [INFO ] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51] [INFO ] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_51] [INFO ] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51] [INFO ] at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51] {noformat} Exception does not influence the tests and they run ok, however this is disturbing. The cause is that Guava changed the signature of the mentioned method to return double instead of void in 16 release. So, can the dependency be updated to avoid the inconvenience? Thanks. p.s. I found a workaround for integration tests - just add the Guava 15 dependency to the cassandra-maven-plugin configuration as follows: {code:xml} plugin groupIdorg.codehaus.mojo/groupId artifactIdcassandra-maven-plugin/artifactId version2.0.0-1/version dependencies dependency groupIdcom.google.guava/groupId artifactIdguava/artifactId version15.0/version /dependency /dependencies plugin {code} -- This message was sent
[jira] [Updated] (CASSANDRA-6642) line 297 of CQL.textile needs updated
[ https://issues.apache.org/jira/browse/CASSANDRA-6642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kristine Hahn updated CASSANDRA-6642: - Description: [the doc|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#partition-key-and-clustering] says: {quote} The restriction for table with COMPACT STORAGE is that they support one and only one column outside of the ones part of the PRIMARY KEY. {quote} Shouldn't it say, ... outside of the ones that are part of a compound primary key? was: The behavior changed in 1.2.6.1 or earlier, but [the doc|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#partition-key-and-clustering] still says: {quote} The restriction for table with COMPACT STORAGE is that they support one and only one column outside of the ones part of the PRIMARY KEY. {quote} line 297 of CQL.textile needs updated - Key: CASSANDRA-6642 URL: https://issues.apache.org/jira/browse/CASSANDRA-6642 Project: Cassandra Issue Type: Bug Components: Documentation website Environment: [CQL 3.1.4 doc, cassandra-trunk|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#partition-key-and-clustering] Reporter: Kristine Hahn Priority: Minor [the doc|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#partition-key-and-clustering] says: {quote} The restriction for table with COMPACT STORAGE is that they support one and only one column outside of the ones part of the PRIMARY KEY. {quote} Shouldn't it say, ... outside of the ones that are part of a compound primary key? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5549) Remove Table.switchLock
[ https://issues.apache.org/jira/browse/CASSANDRA-5549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887295#comment-13887295 ] Benedict commented on CASSANDRA-5549: - Awesome! I've fixed the formatting, added a couple of comments and opened a pull request. Remove Table.switchLock --- Key: CASSANDRA-5549 URL: https://issues.apache.org/jira/browse/CASSANDRA-5549 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Benedict Labels: performance Fix For: 2.1 Attachments: 5549-removed-switchlock.png, 5549-sunnyvale.png As discussed in CASSANDRA-5422, Table.switchLock is a bottleneck on the write path. ReentrantReadWriteLock is not lightweight, even if there is no contention per se between readers and writers of the lock (in Cassandra, memtable updates and switches). -- This message was sent by Atlassian JIRA (v6.1.5#6160)
git commit: sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503
Updated Branches: refs/heads/cassandra-2.0 b5a2b6507 - 36af40925 sstables from stalled repair sessions become live after a reboot and can resurrect deleted data patch by jasobrown, reviewed by yukim for CASSANDRA-6503 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/36af4092 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/36af4092 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/36af4092 Branch: refs/heads/cassandra-2.0 Commit: 36af40925b3e8e01ede5ff6d7ed9a16046409fe3 Parents: b5a2b65 Author: Jason Brown jasedbr...@gmail.com Authored: Thu Jan 30 16:17:01 2014 -0800 Committer: Jason Brown jasedbr...@gmail.com Committed: Thu Jan 30 16:17:01 2014 -0800 -- CHANGES.txt | 1 + .../apache/cassandra/db/ColumnFamilyStore.java | 28 - .../apache/cassandra/io/sstable/Descriptor.java | 20 ++- .../apache/cassandra/io/sstable/SSTable.java| 2 +- .../org/apache/cassandra/io/util/FileUtils.java | 2 +- .../cassandra/streaming/StreamLockfile.java | 121 +++ .../cassandra/streaming/StreamReader.java | 5 +- .../cassandra/streaming/StreamReceiveTask.java | 72 --- .../cassandra/streaming/StreamSession.java | 16 ++- .../cassandra/streaming/StreamTransferTask.java | 10 +- .../compress/CompressedStreamReader.java| 4 +- .../streaming/messages/FileMessage.java | 112 - .../streaming/messages/IncomingFileMessage.java | 78 .../streaming/messages/OutgoingFileMessage.java | 92 ++ .../streaming/messages/StreamMessage.java | 18 ++- 15 files changed, 429 insertions(+), 152 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36af4092/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 531ac15..56a72ef 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -16,6 +16,7 @@ * Release sstables upon rebuilding 2i (CASSANDRA-6635) * Add AbstractCompactionStrategy.startup() method (CASSANDRA-6637) * SSTableScanner may skip rows during cleanup (CASSANDRA-6638) + * sstables from stalled repair sessions can resurrect deleted data (CASSANDRA-6503) Merged from 1.2: * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) http://git-wip-us.apache.org/repos/asf/cassandra/blob/36af4092/src/java/org/apache/cassandra/db/ColumnFamilyStore.java -- diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 8d09453..8750026 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db; import java.io.File; +import java.io.FileFilter; import java.io.IOException; import java.lang.management.ManagementFactory; import java.nio.ByteBuffer; @@ -66,6 +67,7 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.metrics.ColumnFamilyMetrics; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.StreamLockfile; import org.apache.cassandra.thrift.IndexExpression; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.*; @@ -424,9 +426,33 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean */ public static void scrubDataDirectories(String keyspaceName, String columnFamily) { +Directories directories = Directories.create(keyspaceName, columnFamily); + +// remove any left-behind SSTables from failed/stalled streaming +FileFilter filter = new FileFilter() +{ +public boolean accept(File pathname) +{ +return pathname.toString().endsWith(StreamLockfile.FILE_EXT); +} +}; +for (File dir : directories.getCFDirectories()) +{ +File[] lockfiles = dir.listFiles(filter); +if (lockfiles.length == 0) +continue; +logger.info(Removing SSTables from failed streaming session. Found {} files to cleanup., lockfiles.length); + +for (File lockfile : lockfiles) +{ +StreamLockfile streamLockfile = new StreamLockfile(lockfile); +streamLockfile.cleanup(); +streamLockfile.delete(); +} +} + logger.debug(Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable), columnFamily); -
[jira] [Commented] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887330#comment-13887330 ] Tyler Hobbs commented on CASSANDRA-6592: +1 on the rebase of the 2.0 patch. I've run my repro script for 5x the max time it took me to reproduce before with no errors. Just to add more details for posterity, this was the schema: {noformat} CREATE TABLE duration_test.ints (key int, copy int, value int, RIMARY KEY (key, copy)) {noformat} The statement that triggered the IllegalArgumentException was this: {noformat} DELETE FROM duration_test.ints USING TIMESTAMP 123456 WHERE key=? AND copy=?; {noformat} with the internal representation: {noformat} DeleteStatement(name=duration_test.ints, columns=[], keys=[key EQ ?, copy EQ ?] {noformat} Here are some of the negative sizes that statement got: -20936, -25872, -19856, -4784, -16152. (When reproducing to test this ticket, I never saw the large positive sizes, so that might have been a driver error, I suppose.) Typically the IAE's would come in bursts of ~5 within a second, and then it would be several hours before the next round of errors. The only other activity while this happened was preparation of some insert statements and execution of various selects, inserts, and deletes (both prepared and non-prepared). There were no schema changes or anything like that. IllegalArgumentException when Preparing Statements -- Key: CASSANDRA-6592 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Sylvain Lebresne Priority: Critical Fix For: 1.2.14, 2.0.5 Attachments: 6592-2.0.txt When preparing a lot of statements with the python native driver, I occasionally get an error response with an error that corresponds to the following stacktrace in the cassandra logs: {noformat} ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 ErrorMessage.java (line 210) Unexpected exception during request java.lang.IllegalArgumentException at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764) at com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743) at org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255) at org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221) at org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) 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$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {noformat} Looking at the CLHM source, this means we're giving the statement a weight that's less than 1. I'll also note that these errors frequently happen in clumps of 2 or 3 at a time. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6643) ALTER TYPE RENAME allows renaming to another keyspace
Russ Hatch created CASSANDRA-6643: - Summary: ALTER TYPE RENAME allows renaming to another keyspace Key: CASSANDRA-6643 URL: https://issues.apache.org/jira/browse/CASSANDRA-6643 Project: Cassandra Issue Type: Bug Environment: java version 1.7.0_51 cassandra from trunk, 4b54b8... Reporter: Russ Hatch Priority: Minor I'm not 100% certain this is a bug. The current syntax for alter type rename requires the keyspace on the old and new table name (if a keyspace is not active). So, to rename the type 'foo' to 'bar', you have to issue this statement: ALTER TYPE ks.foo rename to ks.bar . As a result, this syntax will also allow renaming the type into another existing keyspace, which updates the metadata in system.schema_usertypes. I'm wondering if perhaps we can omit the second keyspace prefix and implicitly rename into the same keyspace. To reproduce: {noformat} cqlsh create keyspace user_types with replication = {'class':'SimpleStrategy', 'replication_factor':3} ; cqlsh create keyspace user_types2 with replication = {'class':'SimpleStrategy', 'replication_factor':3} ; cqlsh CREATE TYPE user_types.simple_type (user_number int); cqlsh alter type user_types.simple_type rename to user_types2.simple_type; {noformat} Renaming to another keyspace is also possible when a keyspace is active, like so: {noformat} cqlsh:user_types alter type simple_type rename to user_types2.simple_type; {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6643) ALTER TYPE RENAME allows renaming to another keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-6643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887350#comment-13887350 ] Russ Hatch commented on CASSANDRA-6643: --- I'm working on dtests for user types, so I can add one for this scenario when we know what the behavior ought to be. ALTER TYPE RENAME allows renaming to another keyspace - Key: CASSANDRA-6643 URL: https://issues.apache.org/jira/browse/CASSANDRA-6643 Project: Cassandra Issue Type: Bug Environment: java version 1.7.0_51 cassandra from trunk, 4b54b8... Reporter: Russ Hatch Priority: Minor I'm not 100% certain this is a bug. The current syntax for alter type rename requires the keyspace on the old and new table name (if a keyspace is not active). So, to rename the type 'foo' to 'bar', you have to issue this statement: ALTER TYPE ks.foo rename to ks.bar . As a result, this syntax will also allow renaming the type into another existing keyspace, which updates the metadata in system.schema_usertypes. I'm wondering if perhaps we can omit the second keyspace prefix and implicitly rename into the same keyspace. To reproduce: {noformat} cqlsh create keyspace user_types with replication = {'class':'SimpleStrategy', 'replication_factor':3} ; cqlsh create keyspace user_types2 with replication = {'class':'SimpleStrategy', 'replication_factor':3} ; cqlsh CREATE TYPE user_types.simple_type (user_number int); cqlsh alter type user_types.simple_type rename to user_types2.simple_type; {noformat} Renaming to another keyspace is also possible when a keyspace is active, like so: {noformat} cqlsh:user_types alter type simple_type rename to user_types2.simple_type; {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6503) sstables from stalled repair sessions become live after a reboot and can resurrect deleted data
[ https://issues.apache.org/jira/browse/CASSANDRA-6503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-6503: -- Attachment: 6503-2.0-followup.txt [~jasobrown] Looks like we need to make a little tweak around complete message since we moved adding received SSTables to different thread. It is breaking StreamingTransferTest. Patch attach for fix. sstables from stalled repair sessions become live after a reboot and can resurrect deleted data --- Key: CASSANDRA-6503 URL: https://issues.apache.org/jira/browse/CASSANDRA-6503 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Jason Brown Priority: Minor Fix For: 1.2.14, 2.0.5 Attachments: 6503-2.0-followup.txt, 6503_2.0-v2.diff, 6503_2.0-v3.diff, 6503_c1.2-v1.patch The sstables streamed in during a repair session don't become active until the session finishes. If something causes the repair session to hang for some reason, those sstables will hang around until the next reboot, and become active then. If you don't reboot for 3 months, this can cause data to resurrect, as GC grace has expired, so tombstones for the data in those sstables may have already been collected. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6643) ALTER TYPE RENAME allows renaming to another keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-6643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887412#comment-13887412 ] Mikhail Stepura commented on CASSANDRA-6643: [~rhatch] It doesn't work if there is a table which uses the type, does it? ALTER TYPE RENAME allows renaming to another keyspace - Key: CASSANDRA-6643 URL: https://issues.apache.org/jira/browse/CASSANDRA-6643 Project: Cassandra Issue Type: Bug Environment: java version 1.7.0_51 cassandra from trunk, 4b54b8... Reporter: Russ Hatch Priority: Minor I'm not 100% certain this is a bug. The current syntax for alter type rename requires the keyspace on the old and new table name (if a keyspace is not active). So, to rename the type 'foo' to 'bar', you have to issue this statement: ALTER TYPE ks.foo rename to ks.bar . As a result, this syntax will also allow renaming the type into another existing keyspace, which updates the metadata in system.schema_usertypes. I'm wondering if perhaps we can omit the second keyspace prefix and implicitly rename into the same keyspace. To reproduce: {noformat} cqlsh create keyspace user_types with replication = {'class':'SimpleStrategy', 'replication_factor':3} ; cqlsh create keyspace user_types2 with replication = {'class':'SimpleStrategy', 'replication_factor':3} ; cqlsh CREATE TYPE user_types.simple_type (user_number int); cqlsh alter type user_types.simple_type rename to user_types2.simple_type; {noformat} Renaming to another keyspace is also possible when a keyspace is active, like so: {noformat} cqlsh:user_types alter type simple_type rename to user_types2.simple_type; {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
git commit: Switch stress to use ITransportFactory
Updated Branches: refs/heads/cassandra-2.0 36af40925 - e7b23cc74 Switch stress to use ITransportFactory patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for CASSANDRA-6641 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e7b23cc7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e7b23cc7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e7b23cc7 Branch: refs/heads/cassandra-2.0 Commit: e7b23cc741622ccf8b1487d8622cca47dcb9cc34 Parents: 36af409 Author: Aleksey Yeschenko alek...@apache.org Authored: Fri Jan 31 01:38:54 2014 -0500 Committer: Aleksey Yeschenko alek...@apache.org Committed: Fri Jan 31 01:38:54 2014 -0500 -- CHANGES.txt | 1 + .../org/apache/cassandra/cli/CliOptions.java| 45 +++-- .../org/apache/cassandra/stress/Session.java| 69 +--- 3 files changed, 88 insertions(+), 27 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7b23cc7/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 56a72ef..94b21d4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -17,6 +17,7 @@ * Add AbstractCompactionStrategy.startup() method (CASSANDRA-6637) * SSTableScanner may skip rows during cleanup (CASSANDRA-6638) * sstables from stalled repair sessions can resurrect deleted data (CASSANDRA-6503) + * Switch stress to use ITransportFactory (CASSANDRA-6641) Merged from 1.2: * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e7b23cc7/src/java/org/apache/cassandra/cli/CliOptions.java -- diff --git a/src/java/org/apache/cassandra/cli/CliOptions.java b/src/java/org/apache/cassandra/cli/CliOptions.java index 68f17c9..7894bf9 100644 --- a/src/java/org/apache/cassandra/cli/CliOptions.java +++ b/src/java/org/apache/cassandra/cli/CliOptions.java @@ -17,9 +17,15 @@ */ package org.apache.cassandra.cli; +import java.util.HashMap; +import java.util.Map; + +import com.google.common.base.Joiner; import org.apache.commons.cli.*; +import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.thrift.ITransportFactory; +import org.apache.cassandra.thrift.SSLTransportFactory; /** * @@ -114,11 +120,6 @@ public class CliOptions css.hostName = DEFAULT_HOST; } -if (cmd.hasOption(TRANSPORT_FACTORY)) -{ -css.transportFactory = validateAndSetTransportFactory(cmd.getOptionValue(TRANSPORT_FACTORY)); -} - if (cmd.hasOption(DEBUG_OPTION)) { css.debug = true; @@ -217,6 +218,12 @@ public class CliOptions css.encOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(,); } +if (cmd.hasOption(TRANSPORT_FACTORY)) +{ +css.transportFactory = validateAndSetTransportFactory(cmd.getOptionValue(TRANSPORT_FACTORY)); +configureTransportFactory(css.transportFactory, css.encOptions); +} + // Abort if there are any unrecognized arguments left if (cmd.getArgs().length 0) { @@ -281,4 +288,32 @@ public class CliOptions throw new IllegalArgumentException(String.format(Cannot create a transport factory '%s'., transportFactory), e); } } + +private static void configureTransportFactory(ITransportFactory transportFactory, EncryptionOptions encOptions) +{ +MapString, String options = new HashMap(); +// If the supplied factory supports the same set of options as our SSL impl, set those +if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE)) +options.put(SSLTransportFactory.TRUSTSTORE, encOptions.truststore); +if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE_PASSWORD)) +options.put(SSLTransportFactory.TRUSTSTORE_PASSWORD, encOptions.truststore_password); +if (transportFactory.supportedOptions().contains(SSLTransportFactory.PROTOCOL)) +options.put(SSLTransportFactory.PROTOCOL, encOptions.protocol); +if (transportFactory.supportedOptions().contains(SSLTransportFactory.CIPHER_SUITES)) +options.put(SSLTransportFactory.CIPHER_SUITES, Joiner.on(',').join(encOptions.cipher_suites)); + +if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE) + encOptions.require_client_auth) +
[jira] [Commented] (CASSANDRA-6641) Switch stress to use ITransportFactory to obtain client connections
[ https://issues.apache.org/jira/browse/CASSANDRA-6641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13887529#comment-13887529 ] Aleksey Yeschenko commented on CASSANDRA-6641: -- Committed to cassandra-2.0 w/ imports and whitespace fixed. Switch stress to use ITransportFactory to obtain client connections --- Key: CASSANDRA-6641 URL: https://issues.apache.org/jira/browse/CASSANDRA-6641 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Sam Tunnicliffe Assignee: Sam Tunnicliffe Priority: Minor Attachments: 0001-cassandra-stress-uses-ITransportFactory-to-provide-c.patch, 0002-Set-supplied-encryption-options-on-transport-factory.patch CASSANDRA-6062 CASSANDRA-6378 changed cassandra-cli and sstableloader respectively to use o.a.c.thrift.ITransportFactory rather than o.a.thrift.transport.TTransportFactory implementations. This ticket is to do likewise for cassandra-stress, so that users can have more control over the connection options when running stress. -- This message was sent by Atlassian JIRA (v6.1.5#6160)