[jira] [Commented] (CASSANDRA-8938) Full Row Scan does not count towards Reads
[ https://issues.apache.org/jira/browse/CASSANDRA-8938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386292#comment-14386292 ] Marcus Eriksson commented on CASSANDRA-8938: depends I guess, one could consider a range scan an analytics workload where it shouldn't count towards hotness of sstables (ie, you have regular real queries and then periodically you do range queries and you don't want them to affect the real queries) [~thobbs] do you remember if this was on purpose? Full Row Scan does not count towards Reads -- Key: CASSANDRA-8938 URL: https://issues.apache.org/jira/browse/CASSANDRA-8938 Project: Cassandra Issue Type: Bug Components: API, Core, Tools Environment: Unix, Cassandra 2.0.3 Reporter: Amit Singh Chowdhery Assignee: Marcus Eriksson Priority: Minor Labels: none When a CQL SELECT statement is executed with WHERE clause, Read Count is incremented in cfstats of the column family. But, when a full row scan is done using SELECT statement without WHERE clause, Read Count is not incremented. Similarly, when using Size Tiered Compaction, if we do a full row scan using Hector RangeslicesQuery, Read Count is not incremented in cfstats, Cassandra still considers all sstables as cold and does not trigger compaction for them. If we fire MultigetSliceQuery, Read Count is incremented and sstables becomes hot, triggering compaction of these sstables. Expected Behavior: 1. Read Count must be incremented by number of rows read during a full row scan done using CQL SELECT statement or Hector RangeslicesQuery. 2. Size Tiered compaction must consider all sstables as Hot after a full row scan. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7976) Changes to index_interval table properties revert after subsequent modifications
[ https://issues.apache.org/jira/browse/CASSANDRA-7976?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386275#comment-14386275 ] Stefania commented on CASSANDRA-7976: - Verified {{index_interval}} on cassandra-2.0 and reproduced the problem, added unit test. Verified {{min_index_interval}} and {{max_index_interval}} on trunk and cassandra-2.1 but could not reproduce. The problem for 2.0 is that the index interval is not updated in {{CFMetadata.apply()}}. As a consequence, the index interval was never changed, this is clearly visible in the log file, despite what the cqlsh DESC command shows. The reason why the initial DESC command shows an updated index interval is that the migration manager pushes a schema change that was not correctly applied. When the index interval was changed into min and max on cassandra-2.1, {{CFMetadata.apply()}} was fixed (verified on trunk). The patch for 2.0 is here: https://github.com/stef1927/cassandra/commits/7976. Changes to index_interval table properties revert after subsequent modifications Key: CASSANDRA-7976 URL: https://issues.apache.org/jira/browse/CASSANDRA-7976 Project: Cassandra Issue Type: Bug Components: Config Environment: cqlsh 4.1.1, Cassandra 2.0.9-SNAPSHOT (built w/ `ccm` on Mac OS X 10.9.4 with Java 1.7.0_67 - more detail below) $ java -version java version 1.7.0_67 Java(TM) SE Runtime Environment (build 1.7.0_67-b01) Java HotSpot(TM) 64-Bit Server VM (build 24.65-b04, mixed mode) $ mvn --version Apache Maven 3.2.3 (33f8c3e1027c3ddde99d3cdebad2656a31e8fdf4; 2014-08-11T13:58:10-07:00) Maven home: /usr/local/Cellar/maven/3.2.3/libexec Java version: 1.7.0_67, vendor: Oracle Corporation Java home: /Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre Default locale: en_US, platform encoding: UTF-8 OS name: mac os x, version: 10.9.4, arch: x86_64, family: mac Reporter: Andrew Lenards Assignee: Stefania Labels: cql3, metadata It appears that if you want to increase the sampling in *-Summary.db files, you would change the default for {{index_interval}} table property from the {{128}} default value to {{256}} on a given CQL {{TABLE}}. However, if you {{ALTER TABLE}} after setting the value, {{index_interval}} returns to the default, {{128}}. This is unexpected behavior. I would expect the value for {{index_interval}} to not be affected by subsequent {{ALTER TABLE}} statements. As noted in Environment, this was seen with a 2.0.9-SNAPSHOT built w/ `ccm`. If I just use a table from one of DataStax documentation tutorials (musicdb as mdb): {noformat} cqlsh:mdb DESC TABLE songs; CREATE TABLE songs ( id uuid, album text, artist text, data blob, reviews listtext, tags settext, title text, venue maptimestamp, text, PRIMARY KEY ((id)) ) WITH bloom_filter_fp_chance=0.01 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND replicate_on_write='true' AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'SizeTieredCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {noformat} We've got {{128}} as expected. We alter it: {noformat} cqlsh:mdb ALTER TABLE songs WITH index_interval = 256; {noformat} And the change appears: {noformat} cqlsh:mdb DESC TABLE songs; CREATE TABLE songs ( id uuid, album text, artist text, data blob, reviews listtext, tags settext, title text, venue maptimestamp, text, PRIMARY KEY ((id)) ) WITH bloom_filter_fp_chance=0.01 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=256 AND read_repair_chance=0.00 AND replicate_on_write='true' AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'SizeTieredCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {noformat} But if do another {{ALTER TABLE}}, say, change the caching or comment, the {{index_interval}} will revert back to {{128}}. {noformat} cqlsh:mdb ALTER TABLE songs WITH caching = 'none'; cqlsh:mdb DESC TABLE songs; CREATE TABLE songs ( id uuid, album text, artist text, data blob, reviews listtext, tags settext, title text, venue maptimestamp, text, PRIMARY KEY ((id)) ) WITH bloom_filter_fp_chance=0.01
[jira] [Resolved] (CASSANDRA-6363) CAS not applied on rows containing an expired ttl column
[ https://issues.apache.org/jira/browse/CASSANDRA-6363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefania resolved CASSANDRA-6363. - Resolution: Cannot Reproduce Thanks, closed. CAS not applied on rows containing an expired ttl column Key: CASSANDRA-6363 URL: https://issues.apache.org/jira/browse/CASSANDRA-6363 Project: Cassandra Issue Type: Bug Components: Core Environment: Linux/x64 2.0.2 4-node cluster Reporter: Michał Ziemski Assignee: Stefania CREATE TABLE session ( id text, usr text, valid int, PRIMARY KEY (id) ); insert into session (id, usr) values ('abc', 'abc'); update session using ttl 1 set valid = 1 where id = 'abc'; (wait 1 sec) And delete from session where id = 'DSYUCTCLSOEKVLAQWNWYLVQMEQGGXD' if usr ='demo'; Yields: [applied] | usr ---+- False | abc Rather than applying the delete. Executing: update session set valid = null where id = 'abc'; and again delete from session where id = 'DSYUCTCLSOEKVLAQWNWYLVQMEQGGXD' if usr ='demo'; Positively deletes the row. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386271#comment-14386271 ] Marcus Eriksson commented on CASSANDRA-9045: [~philipthompson] yes, do that, with this patch you can set it to 0 even: http://aep.appspot.com/display/wSaOmJhJ6IGh0NYSe8-gY0sM4Yg/ Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-6363) CAS not applied on rows containing an expired ttl column
[ https://issues.apache.org/jira/browse/CASSANDRA-6363?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386114#comment-14386114 ] Stefania edited comment on CASSANDRA-6363 at 3/30/15 6:38 AM: -- [~thobbs]: - tested manually and with dtest below in cassandra-2.0 and cannot reproduce. - tested with dtest below in cassandra-2.1 and cannot reproduce - tested with dtest below in trunk and cannot reproduce The dtest: https://github.com/stef1927/cassandra-dtest/commit/eaf56385405db4702d869699e80ad4d00b41cec4 {code} def delete_with_ttl_expired_test(self): Updating a row with a ttl does not prevent deletion, test for CASSANDRA-6363 self.cursor1.execute(DROP TABLE IF EXISTS session) self.cursor1.execute(CREATE TABLE session (id text, usr text, valid int, PRIMARY KEY (id))) self.cursor1.execute(insert into session (id, usr) values ('abc', 'abc')) self.cursor1.execute(update session using ttl 1 set valid = 1 where id = 'abc') self.smart_sleep(time.time(), 1) self.cursor1.execute(delete from session where id = 'abc' if usr ='abc') assert_row_count(self.cursor1, 'session', 0) {code} Please confirm it's OK to close. was (Author: stefania): [~thobbs]: - tested manually and with dtest below in cassandra-2.0 and cannot reproduce. - tested with dtest below in cassandra-2.1 and cannot reproduce - tested with dtest below in trunk and cannot reproduce The dtest: https://github.com/stef1927/cassandra-dtest/commit/eaf56385405db4702d869699e80ad4d00b41cec4 {code} def delete_with_ttl_expired_test(self): Updating a row with a ttl does not prevent deletion, test for CASSANDRA-6363 self.cursor1.execute(DROP TABLE IF EXISTS session) self.cursor1.execute(CREATE TABLE session (id text, usr text, valid int, PRIMARY KEY (id))) self.cursor1.execute(insert into session (id, usr) values ('abc', 'abc')) self.cursor1.execute(update session using ttl 1 set valid = 1 where id = 'abc') self.smart_sleep(time.time(), 1) self.cursor1.execute(delete from session where id = 'abc' if usr ='abc') assert_row_count(self.cursor1, 'session', 0) {code} CAS not applied on rows containing an expired ttl column Key: CASSANDRA-6363 URL: https://issues.apache.org/jira/browse/CASSANDRA-6363 Project: Cassandra Issue Type: Bug Components: Core Environment: Linux/x64 2.0.2 4-node cluster Reporter: Michał Ziemski Assignee: Stefania CREATE TABLE session ( id text, usr text, valid int, PRIMARY KEY (id) ); insert into session (id, usr) values ('abc', 'abc'); update session using ttl 1 set valid = 1 where id = 'abc'; (wait 1 sec) And delete from session where id = 'DSYUCTCLSOEKVLAQWNWYLVQMEQGGXD' if usr ='demo'; Yields: [applied] | usr ---+- False | abc Rather than applying the delete. Executing: update session set valid = null where id = 'abc'; and again delete from session where id = 'DSYUCTCLSOEKVLAQWNWYLVQMEQGGXD' if usr ='demo'; Positively deletes the row. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6363) CAS not applied on rows containing an expired ttl column
[ https://issues.apache.org/jira/browse/CASSANDRA-6363?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386306#comment-14386306 ] Aleksey Yeschenko commented on CASSANDRA-6363: -- Given that it's been a while without seeing any similar reports, and that the bug was only visible in then unstable CAS implementation in 2.0.2, I say go ahead and close. CAS not applied on rows containing an expired ttl column Key: CASSANDRA-6363 URL: https://issues.apache.org/jira/browse/CASSANDRA-6363 Project: Cassandra Issue Type: Bug Components: Core Environment: Linux/x64 2.0.2 4-node cluster Reporter: Michał Ziemski Assignee: Stefania CREATE TABLE session ( id text, usr text, valid int, PRIMARY KEY (id) ); insert into session (id, usr) values ('abc', 'abc'); update session using ttl 1 set valid = 1 where id = 'abc'; (wait 1 sec) And delete from session where id = 'DSYUCTCLSOEKVLAQWNWYLVQMEQGGXD' if usr ='demo'; Yields: [applied] | usr ---+- False | abc Rather than applying the delete. Executing: update session set valid = null where id = 'abc'; and again delete from session where id = 'DSYUCTCLSOEKVLAQWNWYLVQMEQGGXD' if usr ='demo'; Positively deletes the row. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8845) sorted CQLSSTableWriter accept unsorted clustering keys
[ https://issues.apache.org/jira/browse/CASSANDRA-8845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386330#comment-14386330 ] Benjamin Lerer commented on CASSANDRA-8845: --- LGTM sorted CQLSSTableWriter accept unsorted clustering keys --- Key: CASSANDRA-8845 URL: https://issues.apache.org/jira/browse/CASSANDRA-8845 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Assignee: Carl Yeksigian Fix For: 2.1.4 Attachments: 8845-2.1.txt, TestSorted.java The javadoc says : {quote} The SSTable sorted order means that rows are added such that their partition key respect the partitioner order and for a given partition, that *the rows respect the clustering columns order*. public Builder sorted() {quote} It throw an ex when partition key are in incorrect order, however, it doesn't throw an ex when rows are inserted with incorrect clustering keys order. It buffer them and sort them in correct order. {code} writer.addRow(1, 3); writer.addRow(1, 1); writer.addRow(1, 2); {code} {code} $ sstable2json sorted/ks/t1/ks-t1-ka-1-Data.db [ {key: 1, cells: [[\u\u\u\u0001:,,1424524149557000], [\u\u\u\u0002:,,1424524149557000], [\u\u\u\u0003:,,142452414955]]} ] {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Reopened] (CASSANDRA-9060) Anticompaction hangs on bloom filter bitset serialization
[ https://issues.apache.org/jira/browse/CASSANDRA-9060?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reopened CASSANDRA-9060: attaching another small tweak to this Anticompaction hangs on bloom filter bitset serialization -- Key: CASSANDRA-9060 URL: https://issues.apache.org/jira/browse/CASSANDRA-9060 Project: Cassandra Issue Type: Bug Reporter: Gustav Munkby Assignee: Gustav Munkby Priority: Minor Fix For: 2.1.4 Attachments: 2.1-9060-simple.patch, trunk-9060.patch I tried running an incremental repair against a 15-node vnode-cluster with roughly 500GB data running on 2.1.3-SNAPSHOT, without performing the suggested migration steps. I manually chose a small range for the repair (using --start/end-token). The actual repair part took almost no time at all, but the anticompactions took a lot of time (not surprisingly). Obviously, this might not be the ideal way to run incremental repairs, but I wanted to look into what made the whole process so slow. The results were rather surprising. The majority of the time was spent serializing bloom filters. The reason seemed to be two-fold. First, the bloom-filters generated were huge (probably because the original SSTables were large). With a proper migration to incremental repairs, I'm guessing this would not happen. Secondly, however, the bloom filters were being written to the output one byte at a time (with quite a few type-conversions on the way) to transform the little-endian in-memory representation to the big-endian on-disk representation. I have implemented a solution where big-endian is used in-memory as well as on-disk, which obviously makes de-/serialization much, much faster. This introduces some slight overhead when checking the bloom filter, but I can't see how that would be problematic. An obvious alternative would be to still perform the serialization/deserialization using a byte array, but perform the byte-order swap there. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9060) Anticompaction hangs on bloom filter bitset serialization
[ https://issues.apache.org/jira/browse/CASSANDRA-9060?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-9060: --- Attachment: 0001-another-tweak-to-9060.patch Anticompaction hangs on bloom filter bitset serialization -- Key: CASSANDRA-9060 URL: https://issues.apache.org/jira/browse/CASSANDRA-9060 Project: Cassandra Issue Type: Bug Reporter: Gustav Munkby Assignee: Gustav Munkby Priority: Minor Fix For: 2.1.4 Attachments: 0001-another-tweak-to-9060.patch, 2.1-9060-simple.patch, trunk-9060.patch I tried running an incremental repair against a 15-node vnode-cluster with roughly 500GB data running on 2.1.3-SNAPSHOT, without performing the suggested migration steps. I manually chose a small range for the repair (using --start/end-token). The actual repair part took almost no time at all, but the anticompactions took a lot of time (not surprisingly). Obviously, this might not be the ideal way to run incremental repairs, but I wanted to look into what made the whole process so slow. The results were rather surprising. The majority of the time was spent serializing bloom filters. The reason seemed to be two-fold. First, the bloom-filters generated were huge (probably because the original SSTables were large). With a proper migration to incremental repairs, I'm guessing this would not happen. Secondly, however, the bloom filters were being written to the output one byte at a time (with quite a few type-conversions on the way) to transform the little-endian in-memory representation to the big-endian on-disk representation. I have implemented a solution where big-endian is used in-memory as well as on-disk, which obviously makes de-/serialization much, much faster. This introduces some slight overhead when checking the bloom filter, but I can't see how that would be problematic. An obvious alternative would be to still perform the serialization/deserialization using a byte array, but perform the byte-order swap there. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6434) Repair-aware gc grace period
[ https://issues.apache.org/jira/browse/CASSANDRA-6434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386284#comment-14386284 ] Marcus Eriksson commented on CASSANDRA-6434: [~kohlisankalp] no updates, I'll do some more research in what we can actually do here Repair-aware gc grace period - Key: CASSANDRA-6434 URL: https://issues.apache.org/jira/browse/CASSANDRA-6434 Project: Cassandra Issue Type: New Feature Components: Core Reporter: sankalp kohli Assignee: Marcus Eriksson Fix For: 3.0 Since the reason for gcgs is to ensure that we don't purge tombstones until every replica has been notified, it's redundant in a world where we're tracking repair times per sstable (and repairing frequentily), i.e., a world where we default to incremental repair a la CASSANDRA-5351. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9060) Anticompaction hangs on bloom filter bitset serialization
[ https://issues.apache.org/jira/browse/CASSANDRA-9060?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386320#comment-14386320 ] Gustav Munkby commented on CASSANDRA-9060: -- I might be horribly mistaken, but my reading of estimatedKeysForRanges in SSTableReader is that it is based on the number of keys that are definitely in the range, given the sample in the index summary. Thus the estimate should be guaranteed to be an under-approximation. If my understanding above is correct, it seems the repaired table would typically have a too small Bloom filter with the added tweak. Similarly, the unrepaired table will typically have a slightly too big Bloom filter. Given that the Bloom filter is only an optimisation, I'm not sure either of those things really matter that much. I guess it depends on whether any other pieces of the code assume the Bloom-filter sizes to be over- or under sized. Anticompaction hangs on bloom filter bitset serialization -- Key: CASSANDRA-9060 URL: https://issues.apache.org/jira/browse/CASSANDRA-9060 Project: Cassandra Issue Type: Bug Reporter: Gustav Munkby Assignee: Gustav Munkby Priority: Minor Fix For: 2.1.4 Attachments: 0001-another-tweak-to-9060.patch, 2.1-9060-simple.patch, trunk-9060.patch I tried running an incremental repair against a 15-node vnode-cluster with roughly 500GB data running on 2.1.3-SNAPSHOT, without performing the suggested migration steps. I manually chose a small range for the repair (using --start/end-token). The actual repair part took almost no time at all, but the anticompactions took a lot of time (not surprisingly). Obviously, this might not be the ideal way to run incremental repairs, but I wanted to look into what made the whole process so slow. The results were rather surprising. The majority of the time was spent serializing bloom filters. The reason seemed to be two-fold. First, the bloom-filters generated were huge (probably because the original SSTables were large). With a proper migration to incremental repairs, I'm guessing this would not happen. Secondly, however, the bloom filters were being written to the output one byte at a time (with quite a few type-conversions on the way) to transform the little-endian in-memory representation to the big-endian on-disk representation. I have implemented a solution where big-endian is used in-memory as well as on-disk, which obviously makes de-/serialization much, much faster. This introduces some slight overhead when checking the bloom filter, but I can't see how that would be problematic. An obvious alternative would be to still perform the serialization/deserialization using a byte array, but perform the byte-order swap there. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-8728) Unit test support for native protocol
[ https://issues.apache.org/jira/browse/CASSANDRA-8728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-8728. - Resolution: Not a Problem I'm sure no-one will object. Unit test support for native protocol - Key: CASSANDRA-8728 URL: https://issues.apache.org/jira/browse/CASSANDRA-8728 Project: Cassandra Issue Type: Improvement Components: Tests Reporter: Robert Stupp Priority: Minor Currently we do not have any chance to directly test the native protocol „on the wire“. Especially when coding new protocol features, it’s more a ”blind flight” and basically ”hope” that the code is correct. Purpose of this ticket is to provide unit test code that allows testing of native protocol as is. It’s purpose is not to test any CQL stuff as it. Since the native protocol is multiplexed and as such is intended to be used concurrently, unit test support should allow concurrent access to a single connection. Native protocol test code should work against a single node (for unit testing inside C* source tree) but also be able to handle/emit notifications. Test code should be very strict and fail for features/values that are not specified in a specific protocol version. Code used in the test classes should be separate to production code to be able to identify possible bugs in production code. (Following code should be considered as ”pseudo code”/idea and not as _the_ way to go) Establishing a connection: {noformat} NativeConnection connection = new NativeConnection(host, port, minClientVersion, maxClientVersion); connection.setAuthentication(…); connection.establish(timeout); {noformat} (There could be some support to provide host+port in unit tests that start an embedded server.) Sending (building) frames should be possible using a single class that builds the frame’s byte buffer like this. Additionally it should be able to construct corrupt/invalid frames to check failure-resistance of the server. {noformat} NativeStream stream = connection.newStream(); // create new stream NativeFrame frame = new NativeFrame(version, flags, stream.getId(), opcode); frame.addLong(longValue); frame.addStringList(str1, str2, …); nativeConnection.send(frame); {noformat} Stream handling: Each frame received for a stream goes into the stream’s received-frame-queue and can be polled from it. {noformat} NativeStream stream = connection.getEventStream(); // or getStream(-1); NativeFrame frame = stream.poll(); // get next frame {noformat} Native protocol events: {noformat} NativeStream stream = connection.getEventStream(); // or getStream(-1); NativeFrame frame = stream.poll(); // get next frame // OR NativeFrame frame = stream.pollForOpcode(opcode); // get frame with specific opcode // OR NativeFrame frame = stream.pollForEvent(eventCode); // get frame with specific event code {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8238) NPE in SizeTieredCompactionStrategy.filterColdSSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-8238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386279#comment-14386279 ] Marcus Eriksson commented on CASSANDRA-8238: dunno, my thinking was that if anyone has enabled this in 2.0, they probably know what they are doing and removing this would change behavior a bit too much in 2.0 NPE in SizeTieredCompactionStrategy.filterColdSSTables -- Key: CASSANDRA-8238 URL: https://issues.apache.org/jira/browse/CASSANDRA-8238 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Marcus Eriksson Fix For: 2.1.4 Attachments: 0001-assert-that-readMeter-is-not-null.patch, 0001-dont-always-set-client-mode-for-sstable-loader.patch {noformat} ERROR [CompactionExecutor:15] 2014-10-31 15:28:32,318 CassandraDaemon.java:153 - Exception in thread Thread[CompactionExecutor:15,1,main] java.lang.NullPointerException: null at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.filterColdSSTables(SizeTieredCompactionStrategy.java:181) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundSSTables(SizeTieredCompactionStrategy.java:83) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundTask(SizeTieredCompactionStrategy.java:267) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:226) ~[apache-cassandra-2.1.1.jar:2.1.1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_72] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_72] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_72] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_72] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_72] {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8809) Remove 'throws CassandraException'
[ https://issues.apache.org/jira/browse/CASSANDRA-8809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386374#comment-14386374 ] Robert Stupp commented on CASSANDRA-8809: - Would also like to cleanup {{TransportException}} which is an interface the following way: # let {{ServerError}} extend {{CassandraException}} # let {{ProtocolException}} extend {{CassandraException}} # replace usages of {{TransportException}} to {{CassandraException}} Remove 'throws CassandraException' -- Key: CASSANDRA-8809 URL: https://issues.apache.org/jira/browse/CASSANDRA-8809 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Robert Stupp Assignee: Robert Stupp Priority: Minor Fix For: 3.0 Follow-up to CASSANDRA-8528. Once CASSANDRA-8099 has landed and we're close to 3.0 release, we can remove code occurences of {{throws CassandraException}} and inherited and document them using {{@throws}} where neccessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7807) Push notification when tracing completes for an operation
[ https://issues.apache.org/jira/browse/CASSANDRA-7807?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-7807: Attachment: 7807-v4.txt Proabilistic tracing: Nope - that was my first thought (check on {{connection==null}}). But it goes via code paths like this in {{QueryMessage}}: {code} if (isTracingRequested()) { tracingId = UUIDGen.getTimeUUID(); state.prepareTracingSession(tracingId); } if (state.traceNextQuery()) { state.createTracingSession(connection); {code} And with {{state.createTracingSession(connection);}} it has a connection reference. Problem is that {{isTracingRequested}} returns true for both probabilistic and explicit tracing. We could add an _if (probabilisticTracing) { pass null connection reference }_ (and omit the additional boolean) but it looks ugly to me - and the meaning of _connection==null_ for trace-complete event wouldn’t be clear. TransportException: Reason for relying on {{RuntimeException}} is CASSANDRA-8560 in which we made {{CassandraException}} an unchecked exception. But you’re right - better leave it as is for now and clean it up in CASSANDRA-8809. debug-cql: Yes - exactly this exception. Its origin in somewhere in the JMX implementation code. (No clue why the JMX code just cannot say, that it failed to bind.) Opened CASSANDRA-9069 for this. It doesn’t fail with 2.0 and 2.1 - so it looks like a change on trunk broke it. Push notification when tracing completes for an operation - Key: CASSANDRA-7807 URL: https://issues.apache.org/jira/browse/CASSANDRA-7807 Project: Cassandra Issue Type: Sub-task Components: Core Reporter: Tyler Hobbs Assignee: Robert Stupp Priority: Minor Labels: client-impacting, protocolv4 Fix For: 3.0 Attachments: 7807-v2.txt, 7807-v3.txt, 7807-v4.txt, 7807.txt Tracing is an asynchronous operation, and drivers currently poll to determine when the trace is complete (in a loop with sleeps). Instead, the server could push a notification to the driver when the trace completes. I'm guessing that most of the work for this will be around pushing notifications to a single connection instead of all connections that have registered listeners for a particular event type. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/2] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cc0247b1 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cc0247b1 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cc0247b1 Branch: refs/heads/trunk Commit: cc0247b1252bf83cbd14057f60fcf18fc10cfacc Parents: 04389ad 6ee4b09 Author: Aleksey Yeschenko alek...@apache.org Authored: Mon Mar 30 11:00:40 2015 +0300 Committer: Aleksey Yeschenko alek...@apache.org Committed: Mon Mar 30 11:00:40 2015 +0300 -- src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc0247b1/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java --
cassandra git commit: Fix sorted docs in CQLSSTableWriter
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 d3258f615 - 6ee4b0989 Fix sorted docs in CQLSSTableWriter Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6ee4b098 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ee4b098 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ee4b098 Branch: refs/heads/cassandra-2.1 Commit: 6ee4b0989d9a3ae3e704918622024fa57fdf63e7 Parents: d3258f6 Author: Carl Yeksigian c...@apache.org Authored: Thu Mar 26 13:19:38 2015 -0400 Committer: Aleksey Yeschenko alek...@apache.org Committed: Mon Mar 30 10:59:55 2015 +0300 -- src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ee4b098/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java index b4bef7d..6a4ea04 100644 --- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java @@ -469,8 +469,7 @@ public class CQLSSTableWriter implements Closeable * added in SSTable sorted order (and an exception will be thrown if that * is not the case during insertion). The SSTable sorted order means that * rows are added such that their partition key respect the partitioner - * order and for a given partition, that the rows respect the clustering - * columns order. + * order. * p * You should thus only use this option is you know that you can provide * the rows in order, which is rarely the case. If you can provide the
[1/2] cassandra git commit: Fix sorted docs in CQLSSTableWriter
Repository: cassandra Updated Branches: refs/heads/trunk 04389ad5e - cc0247b12 Fix sorted docs in CQLSSTableWriter Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6ee4b098 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ee4b098 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ee4b098 Branch: refs/heads/trunk Commit: 6ee4b0989d9a3ae3e704918622024fa57fdf63e7 Parents: d3258f6 Author: Carl Yeksigian c...@apache.org Authored: Thu Mar 26 13:19:38 2015 -0400 Committer: Aleksey Yeschenko alek...@apache.org Committed: Mon Mar 30 10:59:55 2015 +0300 -- src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ee4b098/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java index b4bef7d..6a4ea04 100644 --- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java @@ -469,8 +469,7 @@ public class CQLSSTableWriter implements Closeable * added in SSTable sorted order (and an exception will be thrown if that * is not the case during insertion). The SSTable sorted order means that * rows are added such that their partition key respect the partitioner - * order and for a given partition, that the rows respect the clustering - * columns order. + * order. * p * You should thus only use this option is you know that you can provide * the rows in order, which is rarely the case. If you can provide the
[jira] [Created] (CASSANDRA-9069) debug-cql broken in trunk
Robert Stupp created CASSANDRA-9069: --- Summary: debug-cql broken in trunk Key: CASSANDRA-9069 URL: https://issues.apache.org/jira/browse/CASSANDRA-9069 Project: Cassandra Issue Type: Bug Reporter: Robert Stupp Priority: Minor {{debug-cql}} is broken on trunk. At startup it just says: {code} Error: Exception thrown by the agent : java.lang.NullPointerException {code} That exception originates from JMX agent (which cannot bind). It can be reproduced by starting C* locally and starting {{debug-cql}}. Workaround is to comment out sourcing of {{cassandra-env.sh}}. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9060) Anticompaction hangs on bloom filter bitset serialization
[ https://issues.apache.org/jira/browse/CASSANDRA-9060?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386417#comment-14386417 ] Benedict commented on CASSANDRA-9060: - bq. If my understanding above is correct, it seems the repaired table would typically have a too small Bloom filter with the added tweak Agreed. I filed CASSANDRA-9068 to address this as a follow up. Depending on how many ranges are provided, and how they overlap with the index summary, it is possible for the over-estimation to yield a value greater than the no-args estimatedKeys(), which I think we should also address (or if we know for sure the range collection provided won't ever suffer this problem significantly, we could comment this fact). Anticompaction hangs on bloom filter bitset serialization -- Key: CASSANDRA-9060 URL: https://issues.apache.org/jira/browse/CASSANDRA-9060 Project: Cassandra Issue Type: Bug Reporter: Gustav Munkby Assignee: Gustav Munkby Priority: Minor Fix For: 2.1.4 Attachments: 0001-another-tweak-to-9060.patch, 2.1-9060-simple.patch, trunk-9060.patch I tried running an incremental repair against a 15-node vnode-cluster with roughly 500GB data running on 2.1.3-SNAPSHOT, without performing the suggested migration steps. I manually chose a small range for the repair (using --start/end-token). The actual repair part took almost no time at all, but the anticompactions took a lot of time (not surprisingly). Obviously, this might not be the ideal way to run incremental repairs, but I wanted to look into what made the whole process so slow. The results were rather surprising. The majority of the time was spent serializing bloom filters. The reason seemed to be two-fold. First, the bloom-filters generated were huge (probably because the original SSTables were large). With a proper migration to incremental repairs, I'm guessing this would not happen. Secondly, however, the bloom filters were being written to the output one byte at a time (with quite a few type-conversions on the way) to transform the little-endian in-memory representation to the big-endian on-disk representation. I have implemented a solution where big-endian is used in-memory as well as on-disk, which obviously makes de-/serialization much, much faster. This introduces some slight overhead when checking the bloom filter, but I can't see how that would be problematic. An obvious alternative would be to still perform the serialization/deserialization using a byte array, but perform the byte-order swap there. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7807) Push notification when tracing completes for an operation
[ https://issues.apache.org/jira/browse/CASSANDRA-7807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386480#comment-14386480 ] Stefania commented on CASSANDRA-7807: - bq. Problem is that isTracingRequested returns true for both probabilistic and explicit tracing. Where do you see this? Push notification when tracing completes for an operation - Key: CASSANDRA-7807 URL: https://issues.apache.org/jira/browse/CASSANDRA-7807 Project: Cassandra Issue Type: Sub-task Components: Core Reporter: Tyler Hobbs Assignee: Robert Stupp Priority: Minor Labels: client-impacting, protocolv4 Fix For: 3.0 Attachments: 7807-v2.txt, 7807-v3.txt, 7807-v4.txt, 7807.txt Tracing is an asynchronous operation, and drivers currently poll to determine when the trace is complete (in a loop with sleeps). Instead, the server could push a notification to the driver when the trace completes. I'm guessing that most of the work for this will be around pushing notifications to a single connection instead of all connections that have registered listeners for a particular event type. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8845) sorted CQLSSTableWriter accept unsorted clustering keys
[ https://issues.apache.org/jira/browse/CASSANDRA-8845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386345#comment-14386345 ] Aleksey Yeschenko commented on CASSANDRA-8845: -- Committed. sorted CQLSSTableWriter accept unsorted clustering keys --- Key: CASSANDRA-8845 URL: https://issues.apache.org/jira/browse/CASSANDRA-8845 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Assignee: Carl Yeksigian Fix For: 2.1.4 Attachments: 8845-2.1.txt, TestSorted.java The javadoc says : {quote} The SSTable sorted order means that rows are added such that their partition key respect the partitioner order and for a given partition, that *the rows respect the clustering columns order*. public Builder sorted() {quote} It throw an ex when partition key are in incorrect order, however, it doesn't throw an ex when rows are inserted with incorrect clustering keys order. It buffer them and sort them in correct order. {code} writer.addRow(1, 3); writer.addRow(1, 1); writer.addRow(1, 2); {code} {code} $ sstable2json sorted/ks/t1/ks-t1-ka-1-Data.db [ {key: 1, cells: [[\u\u\u\u0001:,,1424524149557000], [\u\u\u\u0002:,,1424524149557000], [\u\u\u\u0003:,,142452414955]]} ] {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7272) Add Major Compaction to LCS
[ https://issues.apache.org/jira/browse/CASSANDRA-7272?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386465#comment-14386465 ] Marcus Eriksson commented on CASSANDRA-7272: pushed a branch with fixes here: https://github.com/krummas/cassandra/commits/marcuse/7272-2 Add Major Compaction to LCS -- Key: CASSANDRA-7272 URL: https://issues.apache.org/jira/browse/CASSANDRA-7272 Project: Cassandra Issue Type: Improvement Components: Core Reporter: T Jake Luciani Assignee: Marcus Eriksson Priority: Minor Labels: compaction, docs-impacting Fix For: 3.0 LCS has a number of minor issues (maybe major depending on your perspective). LCS is primarily used for wide rows so for instance when you repair data in LCS you end up with a copy of an entire repaired row in L0. Over time if you repair you end up with multiple copies of a row in L0 - L5. This can make predicting disk usage confusing. Another issue is cleaning up tombstoned data. If a tombstone lives in level 1 and data for the cell lives in level 5 the data will not be reclaimed from disk until the tombstone reaches level 5. I propose we add a major compaction for LCS that forces consolidation of data to level 5 to address these. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-8534) The default configuration URL does not have the required file:// prefix and throws an exception if cassandra.config is not set.
[ https://issues.apache.org/jira/browse/CASSANDRA-8534?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson resolved CASSANDRA-8534. Resolution: Not a Problem The default configuration URL does not have the required file:// prefix and throws an exception if cassandra.config is not set. --- Key: CASSANDRA-8534 URL: https://issues.apache.org/jira/browse/CASSANDRA-8534 Project: Cassandra Issue Type: Bug Components: Config, Core Environment: Ubuntu 14.04 64-bit C* 2.1.2 Reporter: Andrew Trimble Priority: Minor Fix For: 2.1.4 Attachments: error.txt In the class org.apache.cassandra.config.YamlConfigurationLoader, the DEFAULT_CONFIGURATION is set to cassandra.yaml. This is improperly formatted as it does not contain the prefix file://. If this value is used, a ConfigurationException is thrown (see line 73 of the same class). A solution is to set the cassandra.config system property, but this does not solve the underlying problem. A vanilla Cassandra installation will throw this error. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8534) The default configuration URL does not have the required file:// prefix and throws an exception if cassandra.config is not set.
[ https://issues.apache.org/jira/browse/CASSANDRA-8534?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386580#comment-14386580 ] Mike Adamson commented on CASSANDRA-8534: - No, it can't be a problem. The default configuration URL does not have the required file:// prefix and throws an exception if cassandra.config is not set. --- Key: CASSANDRA-8534 URL: https://issues.apache.org/jira/browse/CASSANDRA-8534 Project: Cassandra Issue Type: Bug Components: Config, Core Environment: Ubuntu 14.04 64-bit C* 2.1.2 Reporter: Andrew Trimble Priority: Minor Fix For: 2.1.4 Attachments: error.txt In the class org.apache.cassandra.config.YamlConfigurationLoader, the DEFAULT_CONFIGURATION is set to cassandra.yaml. This is improperly formatted as it does not contain the prefix file://. If this value is used, a ConfigurationException is thrown (see line 73 of the same class). A solution is to set the cassandra.config system property, but this does not solve the underlying problem. A vanilla Cassandra installation will throw this error. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9069) debug-cql broken in trunk
[ https://issues.apache.org/jira/browse/CASSANDRA-9069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9069: --- Fix Version/s: 3.0 debug-cql broken in trunk - Key: CASSANDRA-9069 URL: https://issues.apache.org/jira/browse/CASSANDRA-9069 Project: Cassandra Issue Type: Bug Reporter: Robert Stupp Priority: Minor Fix For: 3.0 {{debug-cql}} is broken on trunk. At startup it just says: {code} Error: Exception thrown by the agent : java.lang.NullPointerException {code} That exception originates from JMX agent (which cannot bind). It can be reproduced by starting C* locally and starting {{debug-cql}}. Workaround is to comment out sourcing of {{cassandra-env.sh}}. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9046) Allow Cassandra config to be updated to restart Daemon without unloading classes
[ https://issues.apache.org/jira/browse/CASSANDRA-9046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386672#comment-14386672 ] Emmanuel Hugonnet commented on CASSANDRA-9046: -- It also fixes the pom produced as the artifactId has changed for reporter-config. Allow Cassandra config to be updated to restart Daemon without unloading classes Key: CASSANDRA-9046 URL: https://issues.apache.org/jira/browse/CASSANDRA-9046 Project: Cassandra Issue Type: Improvement Components: Config Reporter: Emmanuel Hugonnet Fix For: 3.0 Attachments: 0001-CASSANDRA-9046-Making-applyConfig-public-so-it-may-b.patch Make applyConfig public in DatabaseDescriptor so that if we embed C* we can restart it after some configuration change without having to stop the whole application to unload the class which is configured once and for all in a static block. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6096) Look into a Pig Macro to url encode URLs passed to CqlStorage
[ https://issues.apache.org/jira/browse/CASSANDRA-6096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386701#comment-14386701 ] Brandon Williams commented on CASSANDRA-6096: - [~philipthompson] it doesn't apply, but with some cut and paste work you could make a new one that would. Look into a Pig Macro to url encode URLs passed to CqlStorage - Key: CASSANDRA-6096 URL: https://issues.apache.org/jira/browse/CASSANDRA-6096 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Jeremy Hanna Priority: Minor Labels: lhf Attachments: trunk-6096.txt In the evolution of CqlStorage, the URL went from non-encoded to encoded. It would be great to somehow keep the URL readable, perhaps using the Pig macro interface to do expansion: http://pig.apache.org/docs/r0.9.2/cont.html#macros See also CASSANDRA-6073 and CASSANDRA-5867 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9071) CQLSSTableWriter gives java.lang.AssertionError: Empty partition
Ajit Joglekar created CASSANDRA-9071: Summary: CQLSSTableWriter gives java.lang.AssertionError: Empty partition Key: CASSANDRA-9071 URL: https://issues.apache.org/jira/browse/CASSANDRA-9071 Project: Cassandra Issue Type: Bug Components: Tools Environment: java 7 / 8 cassandra 2.1.3 snapshot build locally with last commit https://github.com/apache/cassandra/commit/6ee4b0989d9a3ae3e704918622024fa57fdf63e7 macos Yosemite 10.10.2 Reporter: Ajit Joglekar I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Is there a work around, quick fix that I can try out locally? Thanks, -Ajit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Delay node up and node added notifications until native protocol server is started
Repository: cassandra Updated Branches: refs/heads/trunk cc0247b12 - ff5ed7a03 Delay node up and node added notifications until native protocol server is started Patch by brandonwilliams and stefania, reviewed by brandonwilliams for CASSANDRA-8236 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ff5ed7a0 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ff5ed7a0 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ff5ed7a0 Branch: refs/heads/trunk Commit: ff5ed7a03f7b9968c0156b05226af67882e5670e Parents: cc0247b Author: Brandon Williams brandonwilli...@apache.org Authored: Mon Mar 30 09:18:26 2015 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Mon Mar 30 09:18:26 2015 -0500 -- CHANGES.txt | 1 + .../apache/cassandra/gms/ApplicationState.java | 1 + .../org/apache/cassandra/gms/EndpointState.java | 17 src/java/org/apache/cassandra/gms/Gossiper.java | 15 ++-- .../apache/cassandra/gms/VersionedValue.java| 5 ++ .../cassandra/service/CassandraDaemon.java | 2 + .../cassandra/service/StorageService.java | 83 +++--- .../org/apache/cassandra/transport/Server.java | 88 +--- 8 files changed, 186 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ff5ed7a0/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index e66b724..8b95fb3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0 + * Delay node up and node added notifications until native protocol server is started (CASSANDRA-8236) * Compressed Commit Log (CASSANDRA-6809) * Optimise IntervalTree (CASSANDRA-8988) * Add a key-value payload for third party usage (CASSANDRA-8553) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ff5ed7a0/src/java/org/apache/cassandra/gms/ApplicationState.java -- diff --git a/src/java/org/apache/cassandra/gms/ApplicationState.java b/src/java/org/apache/cassandra/gms/ApplicationState.java index 777dfc5..ade9208 100644 --- a/src/java/org/apache/cassandra/gms/ApplicationState.java +++ b/src/java/org/apache/cassandra/gms/ApplicationState.java @@ -33,6 +33,7 @@ public enum ApplicationState NET_VERSION, HOST_ID, TOKENS, +RPC_READY, // pad to allow adding new states to existing cluster X1, X2, http://git-wip-us.apache.org/repos/asf/cassandra/blob/ff5ed7a0/src/java/org/apache/cassandra/gms/EndpointState.java -- diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java index 1029374..0e6985a 100644 --- a/src/java/org/apache/cassandra/gms/EndpointState.java +++ b/src/java/org/apache/cassandra/gms/EndpointState.java @@ -114,6 +114,23 @@ public class EndpointState isAlive = false; } +public boolean isRpcReady() +{ +VersionedValue rpcState = getApplicationState(ApplicationState.RPC_READY); +return rpcState != null Boolean.parseBoolean(rpcState.value); +} + +public String getStatus() +{ +VersionedValue status = getApplicationState(ApplicationState.STATUS); +if (status == null) +return ; + +String[] pieces = status.value.split(VersionedValue.DELIMITER_STR, -1); +assert (pieces.length 0); +return pieces[0]; +} + public String toString() { return EndpointState: HeartBeatState = + hbState + , AppStateMap = + applicationState; http://git-wip-us.apache.org/repos/asf/cassandra/blob/ff5ed7a0/src/java/org/apache/cassandra/gms/Gossiper.java -- diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index ff1240a..07f2615 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -979,14 +979,19 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean subscriber.onJoin(ep, epState); } +public boolean isAlive(InetAddress endpoint) +{ +EndpointState epState = getEndpointStateForEndpoint(endpoint); +if (epState == null) +return false; +return epState.isAlive() !isDeadState(epState); +} + public boolean isDeadState(EndpointState epState) { -if (epState.getApplicationState(ApplicationState.STATUS) == null) +String state = epState.getStatus(); +if (state.isEmpty()) return false; -String value
[jira] [Resolved] (CASSANDRA-8791) 'nodetool status' reports nodes in undocumented '?' state
[ https://issues.apache.org/jira/browse/CASSANDRA-8791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson resolved CASSANDRA-8791. Resolution: Not a Problem 'nodetool status' reports nodes in undocumented '?' state - Key: CASSANDRA-8791 URL: https://issues.apache.org/jira/browse/CASSANDRA-8791 Project: Cassandra Issue Type: Bug Reporter: Stuart Bishop Priority: Minor Fix For: 2.1.4 'nodetool status' self documents the node status with the statement 'Status=Up/Down' which corresponds to the 'U' and 'D' statuses in the first column of the node listing. There is also a very important '?' status that is not documented, which seems to appear when a node is bootstrapping. Automation and deployment tools need to cope with nodes in this transition state, and the first step is to inform them about it in the documentation. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6096) Look into a Pig Macro to url encode URLs passed to CqlStorage
[ https://issues.apache.org/jira/browse/CASSANDRA-6096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386752#comment-14386752 ] Philip Thompson commented on CASSANDRA-6096: [~ssinghg], could you rebase your patch? Look into a Pig Macro to url encode URLs passed to CqlStorage - Key: CASSANDRA-6096 URL: https://issues.apache.org/jira/browse/CASSANDRA-6096 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Jeremy Hanna Priority: Minor Labels: lhf Attachments: trunk-6096.txt In the evolution of CqlStorage, the URL went from non-encoded to encoded. It would be great to somehow keep the URL readable, perhaps using the Pig macro interface to do expansion: http://pig.apache.org/docs/r0.9.2/cont.html#macros See also CASSANDRA-6073 and CASSANDRA-5867 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8619) using CQLSSTableWriter gives ConcurrentModificationException
[ https://issues.apache.org/jira/browse/CASSANDRA-8619?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386753#comment-14386753 ] Ajit Joglekar commented on CASSANDRA-8619: -- Opened a new bug https://issues.apache.org/jira/browse/CASSANDRA-9071 using CQLSSTableWriter gives ConcurrentModificationException Key: CASSANDRA-8619 URL: https://issues.apache.org/jira/browse/CASSANDRA-8619 Project: Cassandra Issue Type: Bug Components: Tools Environment: sun jdk 7 linux - ubuntu Reporter: Igor Berman Assignee: Benedict Fix For: 2.1.3, 2.0.13 Attachments: 8619, TimeSeriesCassandraLoaderTest.java Using CQLSSTableWriter gives ConcurrentModificationException I'm trying to load many timeseries into cassandra 2.0.11-1 using 'org.apache.cassandra:cassandra-all:2.0.11' {noformat} java.util.ConcurrentModificationException at java.util.TreeMap$PrivateEntryIterator.nextEntry(TreeMap.java:1115) at java.util.TreeMap$ValueIterator.next(TreeMap.java:1160) at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:126) at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:202) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:187) at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:215) schema CREATE TABLE test.sample (ts_id bigint, yr int, t timestamp, v double, tgs setvarchar, PRIMARY KEY((ts_id,yr), t)) WITH CLUSTERING ORDER BY (t DESC) AND COMPRESSION = {'sstable_compression': 'LZ4Compressor'}; statement: INSERT INTO test.sample(ts_id, yr, t, v) VALUES (?,?,?,?) {noformat} with .withBufferSizeInMB(128); it happens more than with .withBufferSizeInMB(256); code based on http://planetcassandra.org/blog/using-the-cassandra-bulk-loader-updated/ writer.addRow(tsId, year, new Date(time), value); Any suggestions will be highly appreciated -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8619) using CQLSSTableWriter gives ConcurrentModificationException
[ https://issues.apache.org/jira/browse/CASSANDRA-8619?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386726#comment-14386726 ] Ajit Joglekar commented on CASSANDRA-8619: -- I am using the latest snapshot built from 2.1 branch (2.1.3-SNAPSHOT) and I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Is there a work around, quick fix that I can try out locally? Thanks, -Ajit using CQLSSTableWriter gives ConcurrentModificationException Key: CASSANDRA-8619 URL: https://issues.apache.org/jira/browse/CASSANDRA-8619 Project: Cassandra Issue Type: Bug Components: Tools Environment: sun jdk 7 linux - ubuntu Reporter: Igor Berman Assignee: Benedict Fix For: 2.1.3, 2.0.13 Attachments: 8619, TimeSeriesCassandraLoaderTest.java Using CQLSSTableWriter gives ConcurrentModificationException I'm trying to load many timeseries into cassandra 2.0.11-1 using 'org.apache.cassandra:cassandra-all:2.0.11' {noformat} java.util.ConcurrentModificationException at java.util.TreeMap$PrivateEntryIterator.nextEntry(TreeMap.java:1115) at java.util.TreeMap$ValueIterator.next(TreeMap.java:1160) at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:126) at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:202) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:187) at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:215) schema CREATE TABLE test.sample (ts_id bigint, yr int, t timestamp, v double, tgs setvarchar, PRIMARY KEY((ts_id,yr), t)) WITH CLUSTERING ORDER BY (t DESC) AND COMPRESSION = {'sstable_compression': 'LZ4Compressor'}; statement: INSERT INTO test.sample(ts_id, yr, t, v) VALUES (?,?,?,?) {noformat} with .withBufferSizeInMB(128); it happens more than with .withBufferSizeInMB(256); code based on http://planetcassandra.org/blog/using-the-cassandra-bulk-loader-updated/ writer.addRow(tsId, year, new Date(time), value); Any suggestions will be highly appreciated -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9071) CQLSSTableWriter gives java.lang.AssertionError: Empty partition
[ https://issues.apache.org/jira/browse/CASSANDRA-9071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ajit Joglekar updated CASSANDRA-9071: - Description: I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit was: I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Is there a work around, quick fix that I can try out locally? Thanks, -Ajit CQLSSTableWriter gives java.lang.AssertionError: Empty partition Key: CASSANDRA-9071 URL: https://issues.apache.org/jira/browse/CASSANDRA-9071 Project: Cassandra Issue Type: Bug Components: Tools Environment: java 7 / 8 cassandra 2.1.3 snapshot build locally with last commit https://github.com/apache/cassandra/commit/6ee4b0989d9a3ae3e704918622024fa57fdf63e7 macos Yosemite 10.10.2 Reporter: Ajit Joglekar I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9072) MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7
Ariel Weisberg created CASSANDRA-9072: - Summary: MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7 Key: CASSANDRA-9072 URL: https://issues.apache.org/jira/browse/CASSANDRA-9072 Project: Cassandra Issue Type: Bug Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fails for me on Linux and OS X {noformat}java.lang.RuntimeException: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:91) Caused by: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at java.lang.reflect.AccessibleObject.slowCheckMemberAccess(AccessibleObject.java:296) at java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:288) at java.lang.reflect.Field.setInt(Field.java:946) at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:86) {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8481) ghost node in gossip
[ https://issues.apache.org/jira/browse/CASSANDRA-8481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-8481: --- Assignee: Brandon Williams ghost node in gossip Key: CASSANDRA-8481 URL: https://issues.apache.org/jira/browse/CASSANDRA-8481 Project: Cassandra Issue Type: Bug Reporter: Alexey Larkov Assignee: Brandon Williams Priority: Minor Fix For: 2.0.14 After inaccurate removing nodes from cluster nodetool gossipinfo and jmx org.apache.cassandra.net.FailureDetector.AllEndpointsStates shows the node status is LEFT. Name Value TypeDisplay NameUpdate Interval Description /192.168.58.75 generation:3 heartbeat:0 REMOVAL_COORDINATOR:REMOVER,f9a28f8c-3244-42d1-986e-592aafe1406c STATUS:LEFT,-3361705224534889554,141446785 jmx org.apache.cassandra.net.FailureDetector.DownEndpointCount is 1 node 58.75 is absent in nodetool status and system.peers table. Before node got LEFT status it was in REMOVING state. I've done unsafeassassinateendpoint and it's status became LEFT, but DownEndpointCount is still 1. And org.apache.cassandra.net.FailureDetector.SimpleStates is still DOWN. How to remove this node from gossip? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8791) 'nodetool status' reports nodes in undocumented '?' state
[ https://issues.apache.org/jira/browse/CASSANDRA-8791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386715#comment-14386715 ] Brandon Williams commented on CASSANDRA-8791: - Honestly, parsing nodetool output is a bad idea in general. We make no guarantees we won't make cosmetic changes to that, even in a minor. Instead you need to use JMX programmatically yourself because that won't change. 'nodetool status' reports nodes in undocumented '?' state - Key: CASSANDRA-8791 URL: https://issues.apache.org/jira/browse/CASSANDRA-8791 Project: Cassandra Issue Type: Bug Reporter: Stuart Bishop Priority: Minor Fix For: 2.1.4 'nodetool status' self documents the node status with the statement 'Status=Up/Down' which corresponds to the 'U' and 'D' statuses in the first column of the node listing. There is also a very important '?' status that is not documented, which seems to appear when a node is bootstrapping. Automation and deployment tools need to cope with nodes in this transition state, and the first step is to inform them about it in the documentation. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-8481) ghost node in gossip
[ https://issues.apache.org/jira/browse/CASSANDRA-8481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams resolved CASSANDRA-8481. - Resolution: Not a Problem If it doesn't show in status or system.peers then it's fine. We keep dead states for 3 days in case there's a partition. ghost node in gossip Key: CASSANDRA-8481 URL: https://issues.apache.org/jira/browse/CASSANDRA-8481 Project: Cassandra Issue Type: Bug Reporter: Alexey Larkov Assignee: Brandon Williams Priority: Minor Fix For: 2.0.14 After inaccurate removing nodes from cluster nodetool gossipinfo and jmx org.apache.cassandra.net.FailureDetector.AllEndpointsStates shows the node status is LEFT. Name Value TypeDisplay NameUpdate Interval Description /192.168.58.75 generation:3 heartbeat:0 REMOVAL_COORDINATOR:REMOVER,f9a28f8c-3244-42d1-986e-592aafe1406c STATUS:LEFT,-3361705224534889554,141446785 jmx org.apache.cassandra.net.FailureDetector.DownEndpointCount is 1 node 58.75 is absent in nodetool status and system.peers table. Before node got LEFT status it was in REMOVING state. I've done unsafeassassinateendpoint and it's status became LEFT, but DownEndpointCount is still 1. And org.apache.cassandra.net.FailureDetector.SimpleStates is still DOWN. How to remove this node from gossip? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9070) Race in cancelling compactions
Marcus Eriksson created CASSANDRA-9070: -- Summary: Race in cancelling compactions Key: CASSANDRA-9070 URL: https://issues.apache.org/jira/browse/CASSANDRA-9070 Project: Cassandra Issue Type: Bug Reporter: Marcus Eriksson Fix For: 2.0.14 seems we might have a race situation when cancelling compactions currently we do the following to ensure that we don't start any new compactions when we try to do markAllCompacting() # pause compactions - this makes sure we don't create any new compaction tasks from the compaction strategies # cancel any ongoing compactions - compactions register themselves with the CompactionMetrics and then, when cancelling we get all compactions here, and tell them to stop Problem is that there is a window between when the CompactionTask is created and when it is registered in CompactionMetrics meaning with a bit of bad luck, we could have a situation like this: # we finish a compaction and create a new CompactionTask from the compaction strategy # we pause the compaction strategies to not create any new CompactionTasks # we cancel all ongoing compactions # The CompactionTask created in #1 above registers itself in CompactionMetrics and misses that it should be cancelled -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-8238) NPE in SizeTieredCompactionStrategy.filterColdSSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-8238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386279#comment-14386279 ] Marcus Eriksson edited comment on CASSANDRA-8238 at 3/30/15 10:26 AM: -- dunno, my thinking was that if anyone has enabled this in 2.0, they probably know what they are doing and removing this would change behavior a bit too much in 2.0 edit: oops, thought this was another ticket, yes it should probably be backported was (Author: krummas): dunno, my thinking was that if anyone has enabled this in 2.0, they probably know what they are doing and removing this would change behavior a bit too much in 2.0 NPE in SizeTieredCompactionStrategy.filterColdSSTables -- Key: CASSANDRA-8238 URL: https://issues.apache.org/jira/browse/CASSANDRA-8238 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Marcus Eriksson Fix For: 2.1.4 Attachments: 0001-assert-that-readMeter-is-not-null.patch, 0001-dont-always-set-client-mode-for-sstable-loader.patch {noformat} ERROR [CompactionExecutor:15] 2014-10-31 15:28:32,318 CassandraDaemon.java:153 - Exception in thread Thread[CompactionExecutor:15,1,main] java.lang.NullPointerException: null at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.filterColdSSTables(SizeTieredCompactionStrategy.java:181) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundSSTables(SizeTieredCompactionStrategy.java:83) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundTask(SizeTieredCompactionStrategy.java:267) ~[apache-cassandra-2.1.1.jar:2.1.1] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:226) ~[apache-cassandra-2.1.1.jar:2.1.1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_72] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_72] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_72] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_72] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_72] {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7807) Push notification when tracing completes for an operation
[ https://issues.apache.org/jira/browse/CASSANDRA-7807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386543#comment-14386543 ] Robert Stupp commented on CASSANDRA-7807: - Sorry - meant {{QueryState.traceNextQuery}} Push notification when tracing completes for an operation - Key: CASSANDRA-7807 URL: https://issues.apache.org/jira/browse/CASSANDRA-7807 Project: Cassandra Issue Type: Sub-task Components: Core Reporter: Tyler Hobbs Assignee: Robert Stupp Priority: Minor Labels: client-impacting, protocolv4 Fix For: 3.0 Attachments: 7807-v2.txt, 7807-v3.txt, 7807-v4.txt, 7807.txt Tracing is an asynchronous operation, and drivers currently poll to determine when the trace is complete (in a loop with sleeps). Instead, the server could push a notification to the driver when the trace completes. I'm guessing that most of the work for this will be around pushing notifications to a single connection instead of all connections that have registered listeners for a particular event type. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9027) Error processing org.apache.cassandra.metrics:type=HintedHandOffManager,name=Hints_created-IPv6 address
[ https://issues.apache.org/jira/browse/CASSANDRA-9027?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-9027: Assignee: Erik Forsberg Error processing org.apache.cassandra.metrics:type=HintedHandOffManager,name=Hints_created-IPv6 address - Key: CASSANDRA-9027 URL: https://issues.apache.org/jira/browse/CASSANDRA-9027 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Erik Forsberg Fix For: 3.0, 2.1.4, 2.0.14 Attachments: cassandra-2.0-9027.txt, cassandra-2.0-9027.txt Getting some of these on 2.0.13: {noformat} WARN [MutationStage:92] 2015-03-24 08:57:20,204 JmxReporter.java (line 397) Error processing org.apache.cassandra.metrics:type=HintedHandOffManager,name=Hints_created-2001:4c28:1:413:0:1:4:1 javax.management.MalformedObjectNameException: Invalid character ':' in value part of property at javax.management.ObjectName.construct(ObjectName.java:618) at javax.management.ObjectName.init(ObjectName.java:1382) at com.yammer.metrics.reporting.JmxReporter.onMetricAdded(JmxReporter.java:395) at com.yammer.metrics.core.MetricsRegistry.notifyMetricAdded(MetricsRegistry.java:516) at com.yammer.metrics.core.MetricsRegistry.getOrAdd(MetricsRegistry.java:491) at com.yammer.metrics.core.MetricsRegistry.newCounter(MetricsRegistry.java:115) at com.yammer.metrics.Metrics.newCounter(Metrics.java:108) at org.apache.cassandra.metrics.HintedHandoffMetrics$2.load(HintedHandoffMetrics.java:58) at org.apache.cassandra.metrics.HintedHandoffMetrics$2.load(HintedHandoffMetrics.java:55) at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3522) at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2315) at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2278) at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2193) at com.google.common.cache.LocalCache.get(LocalCache.java:3932) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3936) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4806) at com.google.common.cache.LocalCache$LocalLoadingCache.getUnchecked(LocalCache.java:4812) at org.apache.cassandra.metrics.HintedHandoffMetrics.incrCreatedHints(HintedHandoffMetrics.java:64) at org.apache.cassandra.db.HintedHandOffManager.hintFor(HintedHandOffManager.java:124) at org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:957) at org.apache.cassandra.service.StorageProxy$6.runMayThrow(StorageProxy.java:927) at org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:2069) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Seems to be about the same as CASSANDRA-5298. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8880) Add metrics to monitor the amount of tombstones created
[ https://issues.apache.org/jira/browse/CASSANDRA-8880?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386623#comment-14386623 ] Lyuben Todorov commented on CASSANDRA-8880: --- Sounds reasonable, so we count the created tombstones, and range tombstones. I think we might need separate metrics to avoid confusion between the two meaning, or is it a fair assumption that tombstones and range tombstones should be counted together? Add metrics to monitor the amount of tombstones created --- Key: CASSANDRA-8880 URL: https://issues.apache.org/jira/browse/CASSANDRA-8880 Project: Cassandra Issue Type: Improvement Reporter: Michaël Figuière Priority: Minor Labels: metrics Fix For: 2.1.4 Attachments: cassandra-2.1-8880.patch AFAIK there's currently no way to monitor the amount of tombstones created on a Cassandra node. CASSANDRA-6057 has made it possible for users to figure out how many tombstones are scanned at read time, but in write mostly workloads, it may not be possible to realize if some inappropriate queries are generating too many tombstones. Therefore the following additional metrics should be added: * {{writtenCells}}: amount of cells that have been written * {{writtenTombstoneCells}}: amount of tombstone cells that have been written Alternatively these could be exposed as a single gauge such as {{writtenTombstoneCellsRatio}}. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8880) Add metrics to monitor the amount of tombstones created
[ https://issues.apache.org/jira/browse/CASSANDRA-8880?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386622#comment-14386622 ] Lyuben Todorov commented on CASSANDRA-8880: --- Sounds reasonable, so we count the created tombstones, and range tombstones. I think we might need separate metrics to avoid confusion between the two meaning, or is it a fair assumption that tombstones and range tombstones should be counted together? Add metrics to monitor the amount of tombstones created --- Key: CASSANDRA-8880 URL: https://issues.apache.org/jira/browse/CASSANDRA-8880 Project: Cassandra Issue Type: Improvement Reporter: Michaël Figuière Priority: Minor Labels: metrics Fix For: 2.1.4 Attachments: cassandra-2.1-8880.patch AFAIK there's currently no way to monitor the amount of tombstones created on a Cassandra node. CASSANDRA-6057 has made it possible for users to figure out how many tombstones are scanned at read time, but in write mostly workloads, it may not be possible to realize if some inappropriate queries are generating too many tombstones. Therefore the following additional metrics should be added: * {{writtenCells}}: amount of cells that have been written * {{writtenTombstoneCells}}: amount of tombstone cells that have been written Alternatively these could be exposed as a single gauge such as {{writtenTombstoneCellsRatio}}. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8880) Add metrics to monitor the amount of tombstones created
[ https://issues.apache.org/jira/browse/CASSANDRA-8880?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386624#comment-14386624 ] Lyuben Todorov commented on CASSANDRA-8880: --- Sounds reasonable, so we count the created tombstones, and range tombstones. I think we might need separate metrics to avoid confusion between the two meaning, or is it a fair assumption that tombstones and range tombstones should be counted together? Add metrics to monitor the amount of tombstones created --- Key: CASSANDRA-8880 URL: https://issues.apache.org/jira/browse/CASSANDRA-8880 Project: Cassandra Issue Type: Improvement Reporter: Michaël Figuière Priority: Minor Labels: metrics Fix For: 2.1.4 Attachments: cassandra-2.1-8880.patch AFAIK there's currently no way to monitor the amount of tombstones created on a Cassandra node. CASSANDRA-6057 has made it possible for users to figure out how many tombstones are scanned at read time, but in write mostly workloads, it may not be possible to realize if some inappropriate queries are generating too many tombstones. Therefore the following additional metrics should be added: * {{writtenCells}}: amount of cells that have been written * {{writtenTombstoneCells}}: amount of tombstone cells that have been written Alternatively these could be exposed as a single gauge such as {{writtenTombstoneCellsRatio}}. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8938) Full Row Scan does not count towards Reads
[ https://issues.apache.org/jira/browse/CASSANDRA-8938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386768#comment-14386768 ] Jonathan Ellis commented on CASSANDRA-8938: --- Yes, we've always considered read [single partiton] and scan to be separate things. Full Row Scan does not count towards Reads -- Key: CASSANDRA-8938 URL: https://issues.apache.org/jira/browse/CASSANDRA-8938 Project: Cassandra Issue Type: Bug Components: API, Core, Tools Environment: Unix, Cassandra 2.0.3 Reporter: Amit Singh Chowdhery Assignee: Marcus Eriksson Priority: Minor Labels: none When a CQL SELECT statement is executed with WHERE clause, Read Count is incremented in cfstats of the column family. But, when a full row scan is done using SELECT statement without WHERE clause, Read Count is not incremented. Similarly, when using Size Tiered Compaction, if we do a full row scan using Hector RangeslicesQuery, Read Count is not incremented in cfstats, Cassandra still considers all sstables as cold and does not trigger compaction for them. If we fire MultigetSliceQuery, Read Count is incremented and sstables becomes hot, triggering compaction of these sstables. Expected Behavior: 1. Read Count must be incremented by number of rows read during a full row scan done using CQL SELECT statement or Hector RangeslicesQuery. 2. Size Tiered compaction must consider all sstables as Hot after a full row scan. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8938) Full Row Scan does not count towards Reads
[ https://issues.apache.org/jira/browse/CASSANDRA-8938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386793#comment-14386793 ] Anuj commented on CASSANDRA-8938: - Even though single partition read and range scan are technically different, from Application point of view they are just reads. I feel that scans should also make sstables HOT and make them eligible for STCS. Regarding nodetool cfstats, if Read count and Read Latency are not including scans , Don't you think we should have stats for scan count and latency ? Full Row Scan does not count towards Reads -- Key: CASSANDRA-8938 URL: https://issues.apache.org/jira/browse/CASSANDRA-8938 Project: Cassandra Issue Type: Bug Components: API, Core, Tools Environment: Unix, Cassandra 2.0.3 Reporter: Amit Singh Chowdhery Assignee: Marcus Eriksson Priority: Minor Labels: none When a CQL SELECT statement is executed with WHERE clause, Read Count is incremented in cfstats of the column family. But, when a full row scan is done using SELECT statement without WHERE clause, Read Count is not incremented. Similarly, when using Size Tiered Compaction, if we do a full row scan using Hector RangeslicesQuery, Read Count is not incremented in cfstats, Cassandra still considers all sstables as cold and does not trigger compaction for them. If we fire MultigetSliceQuery, Read Count is incremented and sstables becomes hot, triggering compaction of these sstables. Expected Behavior: 1. Read Count must be incremented by number of rows read during a full row scan done using CQL SELECT statement or Hector RangeslicesQuery. 2. Size Tiered compaction must consider all sstables as Hot after a full row scan. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386805#comment-14386805 ] Philip Thompson commented on CASSANDRA-9045: This patch doesn't apply to 2.0 Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-9072) MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7
[ https://issues.apache.org/jira/browse/CASSANDRA-9072?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp reassigned CASSANDRA-9072: --- Assignee: Robert Stupp (was: Ariel Weisberg) MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7 Key: CASSANDRA-9072 URL: https://issues.apache.org/jira/browse/CASSANDRA-9072 Project: Cassandra Issue Type: Bug Reporter: Ariel Weisberg Assignee: Robert Stupp Fails for me on Linux and OS X {noformat}java.lang.RuntimeException: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:91) Caused by: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at java.lang.reflect.AccessibleObject.slowCheckMemberAccess(AccessibleObject.java:296) at java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:288) at java.lang.reflect.Field.setInt(Field.java:946) at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:86) {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9072) MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7
[ https://issues.apache.org/jira/browse/CASSANDRA-9072?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386827#comment-14386827 ] Robert Stupp commented on CASSANDRA-9072: - Will provide a patch soon. MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7 Key: CASSANDRA-9072 URL: https://issues.apache.org/jira/browse/CASSANDRA-9072 Project: Cassandra Issue Type: Bug Reporter: Ariel Weisberg Assignee: Robert Stupp Fails for me on Linux and OS X {noformat}java.lang.RuntimeException: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:91) Caused by: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at java.lang.reflect.AccessibleObject.slowCheckMemberAccess(AccessibleObject.java:296) at java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:288) at java.lang.reflect.Field.setInt(Field.java:946) at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:86) {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-8978) CQLSSTableWriter causes ArrayIndexOutOfBoundsException
[ https://issues.apache.org/jira/browse/CASSANDRA-8978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386848#comment-14386848 ] Benjamin Lerer edited comment on CASSANDRA-8978 at 3/30/15 3:18 PM: [~carlyeks] I do not fully understand the problem that you describe. Could you explain a bit more where is the race condition and how it trigger the ArrayIndexOutOfBoundException? was (Author: blerer): @Carl I do not fully understand the problem that you describe. Could you explain a bit more where is the race condition and how it trigger the ArrayIndexOutOfBoundException? CQLSSTableWriter causes ArrayIndexOutOfBoundsException -- Key: CASSANDRA-8978 URL: https://issues.apache.org/jira/browse/CASSANDRA-8978 Project: Cassandra Issue Type: Bug Components: Core Environment: 3.8.0-42-generic #62~precise1-Ubuntu SMP Wed Jun 4 22:04:18 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux java version 1.8.0_20 Java(TM) SE Runtime Environment (build 1.8.0_20-b26) Java HotSpot(TM) 64-Bit Server VM (build 25.20-b23, mixed mode) Reporter: Thomas Borg Salling Assignee: Carl Yeksigian Fix For: 2.1.4 Attachments: 8978-2.1.txt On long-running jobs with CQLSSTableWriter preparing sstables for later bulk load via sstableloader, occassionally I get the sporadic error shown below. I can run the exact same job again - and it will succeed or fail with the same error at another location in the input stream. The error is appears to occur randomly - with the same input it may occur never, early or late in the run with no apparent logic or system. I use five instances of CQLSSTableWriter in the application (to write redundantly to five different tables). But these instances do not exist at the same time; and thus never used concurrently. {code} 09:26:33.582 [main] INFO d.dma.ais.store.FileSSTableConverter - Finished processing directory, 369582175 packets was converted from /nas1/ Exception in thread main java.lang.reflect.InvocationTargetException at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:483) at dk.dma.commons.app.CliCommandList$1.execute(CliCommandList.java:50) at dk.dma.commons.app.CliCommandList.invoke(CliCommandList.java:80) at dk.dma.ais.store.Main.main(Main.java:34) Caused by: java.lang.ArrayIndexOutOfBoundsException: 297868 at org.apache.cassandra.db.ArrayBackedSortedColumns.append(ArrayBackedSortedColumns.java:196) at org.apache.cassandra.db.ArrayBackedSortedColumns.appendOrReconcile(ArrayBackedSortedColumns.java:191) at org.apache.cassandra.db.ArrayBackedSortedColumns.sortCells(ArrayBackedSortedColumns.java:176) at org.apache.cassandra.db.ArrayBackedSortedColumns.maybeSortCells(ArrayBackedSortedColumns.java:125) at org.apache.cassandra.db.ArrayBackedSortedColumns.access$1100(ArrayBackedSortedColumns.java:44) at org.apache.cassandra.db.ArrayBackedSortedColumns$CellCollection.iterator(ArrayBackedSortedColumns.java:622) at org.apache.cassandra.db.ColumnFamily.iterator(ColumnFamily.java:476) at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:129) at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:215){code} So far I overcome this problem by simply retrying with another run of the application in attempt to generate the sstables. But this is a rather time consuming and shaky approach - and I feel a bit uneasy relying on the produced sstables, though their contents appear to be correct when I sample them with cqlsh 'select' after load into Cassandra. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8670) Large columns + NIO memory pooling causes excessive direct memory usage
[ https://issues.apache.org/jira/browse/CASSANDRA-8670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386852#comment-14386852 ] Benedict commented on CASSANDRA-8670: - I've just pushed another update, which undoes part of the SequentialWriter changes, since all writes (esp. for compression) should go through the SW itself. It also makes one further minor change to stop using Channels.newChannel() everywhere, since that introduces an extra layer of byte shuffling unnecessarily, when DataOutputPlus implements a compatible method that can be called directly. Large columns + NIO memory pooling causes excessive direct memory usage --- Key: CASSANDRA-8670 URL: https://issues.apache.org/jira/browse/CASSANDRA-8670 Project: Cassandra Issue Type: Bug Components: Core Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.0 Attachments: largecolumn_test.py If you provide a large byte array to NIO and ask it to populate the byte array from a socket it will allocate a thread local byte buffer that is the size of the requested read no matter how large it is. Old IO wraps new IO for sockets (but not files) so old IO is effected as well. Even If you are using Buffered{Input | Output}Stream you can end up passing a large byte array to NIO. The byte array read method will pass the array to NIO directly if it is larger than the internal buffer. Passing large cells between nodes as part of intra-cluster messaging can cause the NIO pooled buffers to quickly reach a high watermark and stay there. This ends up costing 2x the largest cell size because there is a buffer for input and output since they are different threads. This is further multiplied by the number of nodes in the cluster - 1 since each has a dedicated thread pair with separate thread locals. Anecdotally it appears that the cost is doubled beyond that although it isn't clear why. Possibly the control connections or possibly there is some way in which multiple Need a workload in CI that tests the advertised limits of cells on a cluster. It would be reasonable to ratchet down the max direct memory for the test to trigger failures if a memory pooling issue is introduced. I don't think we need to test concurrently pulling in a lot of them, but it should at least work serially. The obvious fix to address this issue would be to read in smaller chunks when dealing with large values. I think small should still be relatively large (4 megabytes) so that code that is reading from a disk can amortize the cost of a seek. It can be hard to tell what the underlying thing being read from is going to be in some of the contexts where we might choose to implement switching to reading chunks. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9071) CQLSSTableWriter gives java.lang.AssertionError: Empty partition
[ https://issues.apache.org/jira/browse/CASSANDRA-9071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ajit Joglekar updated CASSANDRA-9071: - Description: I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit was: I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit CQLSSTableWriter gives java.lang.AssertionError: Empty partition Key: CASSANDRA-9071 URL: https://issues.apache.org/jira/browse/CASSANDRA-9071 Project: Cassandra Issue Type: Bug Components: Tools Environment: java 7 / 8 cassandra 2.1.3 snapshot build locally with last commit https://github.com/apache/cassandra/commit/6ee4b0989d9a3ae3e704918622024fa57fdf63e7 macos Yosemite 10.10.2 Reporter: Ajit Joglekar I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8978) CQLSSTableWriter causes ArrayIndexOutOfBoundsException
[ https://issues.apache.org/jira/browse/CASSANDRA-8978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386848#comment-14386848 ] Benjamin Lerer commented on CASSANDRA-8978: --- @Carl I do not fully understand the problem that you describe. Could you explain a bit more where is the race condition and how it trigger the ArrayIndexOutOfBoundException? CQLSSTableWriter causes ArrayIndexOutOfBoundsException -- Key: CASSANDRA-8978 URL: https://issues.apache.org/jira/browse/CASSANDRA-8978 Project: Cassandra Issue Type: Bug Components: Core Environment: 3.8.0-42-generic #62~precise1-Ubuntu SMP Wed Jun 4 22:04:18 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux java version 1.8.0_20 Java(TM) SE Runtime Environment (build 1.8.0_20-b26) Java HotSpot(TM) 64-Bit Server VM (build 25.20-b23, mixed mode) Reporter: Thomas Borg Salling Assignee: Carl Yeksigian Fix For: 2.1.4 Attachments: 8978-2.1.txt On long-running jobs with CQLSSTableWriter preparing sstables for later bulk load via sstableloader, occassionally I get the sporadic error shown below. I can run the exact same job again - and it will succeed or fail with the same error at another location in the input stream. The error is appears to occur randomly - with the same input it may occur never, early or late in the run with no apparent logic or system. I use five instances of CQLSSTableWriter in the application (to write redundantly to five different tables). But these instances do not exist at the same time; and thus never used concurrently. {code} 09:26:33.582 [main] INFO d.dma.ais.store.FileSSTableConverter - Finished processing directory, 369582175 packets was converted from /nas1/ Exception in thread main java.lang.reflect.InvocationTargetException at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:483) at dk.dma.commons.app.CliCommandList$1.execute(CliCommandList.java:50) at dk.dma.commons.app.CliCommandList.invoke(CliCommandList.java:80) at dk.dma.ais.store.Main.main(Main.java:34) Caused by: java.lang.ArrayIndexOutOfBoundsException: 297868 at org.apache.cassandra.db.ArrayBackedSortedColumns.append(ArrayBackedSortedColumns.java:196) at org.apache.cassandra.db.ArrayBackedSortedColumns.appendOrReconcile(ArrayBackedSortedColumns.java:191) at org.apache.cassandra.db.ArrayBackedSortedColumns.sortCells(ArrayBackedSortedColumns.java:176) at org.apache.cassandra.db.ArrayBackedSortedColumns.maybeSortCells(ArrayBackedSortedColumns.java:125) at org.apache.cassandra.db.ArrayBackedSortedColumns.access$1100(ArrayBackedSortedColumns.java:44) at org.apache.cassandra.db.ArrayBackedSortedColumns$CellCollection.iterator(ArrayBackedSortedColumns.java:622) at org.apache.cassandra.db.ColumnFamily.iterator(ColumnFamily.java:476) at org.apache.cassandra.db.ColumnIndex$Builder.build(ColumnIndex.java:129) at org.apache.cassandra.io.sstable.SSTableWriter.rawAppend(SSTableWriter.java:233) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:218) at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:215){code} So far I overcome this problem by simply retrying with another run of the application in attempt to generate the sstables. But this is a rather time consuming and shaky approach - and I feel a bit uneasy relying on the produced sstables, though their contents appear to be correct when I sample them with cqlsh 'select' after load into Cassandra. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-8894) Our default buffer size for (uncompressed) buffered reads should be smaller, and based on the expected record size
[ https://issues.apache.org/jira/browse/CASSANDRA-8894?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis reassigned CASSANDRA-8894: - Assignee: Stefania (was: Benedict) Our default buffer size for (uncompressed) buffered reads should be smaller, and based on the expected record size -- Key: CASSANDRA-8894 URL: https://issues.apache.org/jira/browse/CASSANDRA-8894 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Stefania Fix For: 3.0 A large contributor to slower buffered reads than mmapped is likely that we read a full 64Kb at once, when average record sizes may be as low as 140 bytes on our stress tests. The TLB has only 128 entries on a modern core, and each read will touch 32 of these, meaning we are unlikely to almost ever be hitting the TLB, and will be incurring at least 30 unnecessary misses each time (as well as the other costs of larger than necessary accesses). When working with an SSD there is little to no benefit reading more than 4Kb at once, and in either case reading more data than we need is wasteful. So, I propose selecting a buffer size that is the next larger power of 2 than our average record size (with a minimum of 4Kb), so that we expect to read in one operation. I also propose that we create a pool of these buffers up-front, and that we ensure they are all exactly aligned to a virtual page, so that the source and target operations each touch exactly one virtual page per 4Kb of expected record size. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8374) Better support of null for UDF
[ https://issues.apache.org/jira/browse/CASSANDRA-8374?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8374: -- Issue Type: Improvement (was: Bug) Better support of null for UDF -- Key: CASSANDRA-8374 URL: https://issues.apache.org/jira/browse/CASSANDRA-8374 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Robert Stupp Labels: client-impacting, cql3.3, docs-impacting, udf Fix For: 3.0 Attachments: 8374-3.txt, 8374-3.txt, 8473-1.txt, 8473-2.txt Currently, every function needs to deal with it's argument potentially being {{null}}. There is very many case where that's just annoying, users should be able to define a function like: {noformat} CREATE FUNCTION addTwo(val int) RETURNS int LANGUAGE JAVA AS 'return val + 2;' {noformat} without having this crashing as soon as a column it's applied to doesn't a value for some rows (I'll note that this definition apparently cannot be compiled currently, which should be looked into). In fact, I think that by default methods shouldn't have to care about {{null}} values: if the value is {{null}}, we should not call the method at all and return {{null}}. There is still methods that may explicitely want to handle {{null}} (to return a default value for instance), so maybe we can add an {{ALLOW NULLS}} to the creation syntax. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8374) Better support of null for UDF
[ https://issues.apache.org/jira/browse/CASSANDRA-8374?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8374: -- Reviewer: Sylvain Lebresne Better support of null for UDF -- Key: CASSANDRA-8374 URL: https://issues.apache.org/jira/browse/CASSANDRA-8374 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Assignee: Robert Stupp Labels: client-impacting, cql3.3, docs-impacting, udf Fix For: 3.0 Attachments: 8374-3.txt, 8374-3.txt, 8473-1.txt, 8473-2.txt Currently, every function needs to deal with it's argument potentially being {{null}}. There is very many case where that's just annoying, users should be able to define a function like: {noformat} CREATE FUNCTION addTwo(val int) RETURNS int LANGUAGE JAVA AS 'return val + 2;' {noformat} without having this crashing as soon as a column it's applied to doesn't a value for some rows (I'll note that this definition apparently cannot be compiled currently, which should be looked into). In fact, I think that by default methods shouldn't have to care about {{null}} values: if the value is {{null}}, we should not call the method at all and return {{null}}. There is still methods that may explicitely want to handle {{null}} (to return a default value for instance), so maybe we can add an {{ALLOW NULLS}} to the creation syntax. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7950) Output of nodetool compactionstats and compactionhistory does not work well with long keyspace and column family names.
[ https://issues.apache.org/jira/browse/CASSANDRA-7950?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7950: -- Reviewer: Michael Shuler WDYT [~mshuler]? Output of nodetool compactionstats and compactionhistory does not work well with long keyspace and column family names. - Key: CASSANDRA-7950 URL: https://issues.apache.org/jira/browse/CASSANDRA-7950 Project: Cassandra Issue Type: Bug Environment: CentOS 5, 64bit, Oracle JDK 7, DSE Reporter: Eugene Priority: Minor Labels: lhf Fix For: 2.0.14 Attachments: 7950.patch, nodetool-examples.txt When running these commands: nodetool compactionstats nodetool compactionhistory The output can be difficult to grok due to long keyspace names, column family names, and long values. I have attached an example. It's difficult for both humans and grep/sed/awk/perl to read. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-8890) Enhance cassandra-env.sh to handle Java version output in case of OpenJDK icedtea
[ https://issues.apache.org/jira/browse/CASSANDRA-8890?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams reassigned CASSANDRA-8890: --- Assignee: Brandon Williams Enhance cassandra-env.sh to handle Java version output in case of OpenJDK icedtea -- Key: CASSANDRA-8890 URL: https://issues.apache.org/jira/browse/CASSANDRA-8890 Project: Cassandra Issue Type: Improvement Components: Config Environment: Red Hat Enterprise Linux Server release 6.4 (Santiago) Reporter: Sumod Pawgi Assignee: Brandon Williams Priority: Minor Labels: conf, icedtea Fix For: 3.0 Attachments: 8890-v2.txt, trunk-8890.patch, trunk-8890.txt Where observed - Cassandra node has OpenJDK - java version 1.7.0_09-icedtea In some situations, external agents trying to monitor a C* cluster would need to run cassandra -v command to determine the Cassandra version and would expect a numerical output e.g. java version 1.7.0_75 as in case of Oracle JDK. But if the cluster has OpenJDK IcedTea installed, then this condition is not satisfied and the agents will not work correctly as the output from cassandra -v is /opt/apache/cassandra/bin/../conf/cassandra-env.sh: line 102: [: 09-icedtea: integer expression expected Cause - The line which is causing this behavior is - jvmver=`echo $java_ver_output | grep '[openjdk|java] version' | awk -F'' 'NR==1 {print $2}'` Suggested enhancement - If we change the line to - jvmver=`echo $java_ver_output | grep '[openjdk|java] version' | awk -F'' 'NR==1 {print $2}' | awk 'BEGIN {FS=-};{print $1}'`, it will give $jvmver as - 1.7.0_09 for the above case. Can we add this enhancement in the cassandra-env.sh? I would like to add it myself and submit for review, but I am not familiar with C* check in process. There might be better ways to do this, but I thought of this to be simplest and as the edition is at the end of the line, it will be easy to reverse if needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9072) MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7
[ https://issues.apache.org/jira/browse/CASSANDRA-9072?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-9072: Attachment: 9072.txt MessagePayloadTest.resetCqlQueryHandlerField fails on Java 7 Key: CASSANDRA-9072 URL: https://issues.apache.org/jira/browse/CASSANDRA-9072 Project: Cassandra Issue Type: Bug Reporter: Ariel Weisberg Assignee: Robert Stupp Attachments: 9072.txt Fails for me on Linux and OS X {noformat}java.lang.RuntimeException: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:91) Caused by: java.lang.IllegalAccessException: Class org.apache.cassandra.transport.MessagePayloadTest can not access a member of class java.lang.reflect.Field with modifiers private at java.lang.reflect.AccessibleObject.slowCheckMemberAccess(AccessibleObject.java:296) at java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:288) at java.lang.reflect.Field.setInt(Field.java:946) at org.apache.cassandra.transport.MessagePayloadTest.resetCqlQueryHandlerField(MessagePayloadTest.java:86) {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8360) In DTCS, always compact SSTables in the same time window, even if they are fewer than min_threshold
[ https://issues.apache.org/jira/browse/CASSANDRA-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8360: -- Reviewer: Marcus Eriksson In DTCS, always compact SSTables in the same time window, even if they are fewer than min_threshold --- Key: CASSANDRA-8360 URL: https://issues.apache.org/jira/browse/CASSANDRA-8360 Project: Cassandra Issue Type: Improvement Reporter: Björn Hegerfors Assignee: Björn Hegerfors Priority: Minor Attachments: cassandra-2.0-CASSANDRA-8360.txt DTCS uses min_threshold to decide how many time windows of the same size that need to accumulate before merging into a larger window. The age of an SSTable is determined as its min timestamp, and it always falls into exactly one of the time windows. If multiple SSTables fall into the same window, DTCS considers compacting them, but if they are fewer than min_threshold, it decides not to do it. When do more than 1 but fewer than min_threshold SSTables end up in the same time window (except for the current window), you might ask? In the current state, DTCS can spill some extra SSTables into bigger windows when the previous window wasn't fully compacted, which happens all the time when the latest window stops being the current one. Also, repairs and hints can put new SSTables in old windows. I think, and [~jjordan] agreed in a comment on CASSANDRA-6602, that DTCS should ignore min_threshold and compact tables in the same windows regardless of how few they are. I guess max_threshold should still be respected. [~jjordan] suggested that this should apply to all windows but the current window, where all the new SSTables end up. That could make sense. I'm not clear on whether compacting many SSTables at once is more cost efficient or not, when it comes to the very newest and smallest SSTables. Maybe compacting as soon as 2 SSTables are seen is fine if the initial window size is small enough? I guess the opposite could be the case too; that the very newest SSTables should be compacted very many at a time? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8740) java.lang.AssertionError when reading saved cache
[ https://issues.apache.org/jira/browse/CASSANDRA-8740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8740: -- Reviewer: Robert Stupp [~snazy] to review java.lang.AssertionError when reading saved cache - Key: CASSANDRA-8740 URL: https://issues.apache.org/jira/browse/CASSANDRA-8740 Project: Cassandra Issue Type: Bug Components: Core Environment: OEL 6.5, DSE 4.6.0, Cassandra 2.0.11.83 Reporter: Nikolai Grigoriev Assignee: Dave Brosius Fix For: 2.0.14 Attachments: 8740.txt I have started seeing it recently. Not sure from which version but now it happens relatively often one some of my nodes. {code} INFO [main] 2015-02-04 18:18:09,253 ColumnFamilyStore.java (line 249) Initializing duo_xxx INFO [main] 2015-02-04 18:18:09,254 AutoSavingCache.java (line 114) reading saved cache /var/lib/cassandra/saved_caches/duo_xxx-RowCach e-b.db ERROR [main] 2015-02-04 18:18:09,256 CassandraDaemon.java (line 513) Exception encountered during startup java.lang.AssertionError at org.apache.cassandra.cache.SerializingCacheProvider$RowCacheSerializer.serialize(SerializingCacheProvider.java:41) at org.apache.cassandra.cache.SerializingCacheProvider$RowCacheSerializer.serialize(SerializingCacheProvider.java:37) at org.apache.cassandra.cache.SerializingCache.serialize(SerializingCache.java:118) at org.apache.cassandra.cache.SerializingCache.put(SerializingCache.java:177) at org.apache.cassandra.cache.InstrumentingCache.put(InstrumentingCache.java:44) at org.apache.cassandra.cache.AutoSavingCache.loadSaved(AutoSavingCache.java:130) at org.apache.cassandra.db.ColumnFamilyStore.initRowCache(ColumnFamilyStore.java:592) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:119) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:92) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:305) at com.datastax.bdp.server.DseDaemon.setup(DseDaemon.java:419) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496) at com.datastax.bdp.server.DseDaemon.main(DseDaemon.java:659) INFO [Thread-2] 2015-02-04 18:18:09,259 DseDaemon.java (line 505) DSE shutting down... ERROR [Thread-2] 2015-02-04 18:18:09,279 CassandraDaemon.java (line 199) Exception in thread Thread[Thread-2,5,main] java.lang.AssertionError at org.apache.cassandra.gms.Gossiper.addLocalApplicationState(Gossiper.java:1274) at com.datastax.bdp.gms.DseState.setActiveStatus(DseState.java:171) at com.datastax.bdp.server.DseDaemon.stop(DseDaemon.java:506) at com.datastax.bdp.server.DseDaemon$1.run(DseDaemon.java:408) INFO [main] 2015-02-04 18:18:49,144 CassandraDaemon.java (line 135) Logging initialized INFO [main] 2015-02-04 18:18:49,169 DseDaemon.java (line 382) DSE version: 4.6.0 {code} Cassandra version: 2.0.11.83 (DSE 4.6.0) Looks like similar issues were reported and fixed in the past - like CASSANDRA-6325. Maybe I am missing something, but I think that Cassandra should not crash and stop at startup if it cannot read a saved cache. This does not make the node inoperable and does not necessarily indicate a severe data corruption. I have applied a small change to my cluster config, restarted it and 30% of my nodes did not start because of that. Of course the solution is simple, but it requires to go to every node that failed to start, wipe the cache and start. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8056) nodetool snapshot keyspace -cf table -t sametagname does not work on multiple tabes of the same keyspace
[ https://issues.apache.org/jira/browse/CASSANDRA-8056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8056: -- Reviewer: Yuki Morishita Reproduced In: 2.1.2, 2.0.6 (was: 2.0.6, 2.1.2) [~yukim] to review nodetool snapshot keyspace -cf table -t sametagname does not work on multiple tabes of the same keyspace -- Key: CASSANDRA-8056 URL: https://issues.apache.org/jira/browse/CASSANDRA-8056 Project: Cassandra Issue Type: Improvement Components: Core Environment: Cassandra 2.0.6 debian wheezy and squeeze Reporter: Esha Pathak Priority: Trivial Labels: lhf Fix For: 2.0.14 Attachments: CASSANDRA-8056.txt scenario keyspace thing has tables : thing:user , thing:object, thing:user_details steps to reproduce : 1. nodetool snapshot thing --column-family user --tag tagname Requested creating snapshot for: thing and table: user Snapshot directory: tagname 2.nodetool snapshot thing --column-family object --tag tagname Requested creating snapshot for: thing and table: object Exception in thread main java.io.IOException: Snapshot tagname already exists. at org.apache.cassandra.service.StorageService.takeColumnFamilySnapshot(StorageService.java:2274) at sun.reflect.GeneratedMethodAccessor129.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75) at sun.reflect.GeneratedMethodAccessor9.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1487) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:97) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1328) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1420) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:848) at sun.reflect.GeneratedMethodAccessor39.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:556) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:811) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:670) 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:745) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14386955#comment-14386955 ] Philip Thompson commented on CASSANDRA-9045: I am still unable to reproduce with the following worklow: Create a multiDC cluster, where in_memory_compaction_limit_in_mb = 1 set up a keyspace using NTS Create the same table as the reporter. Write 5M rows to a single partition. Flush Select a row from the partition Delete the row Repair on all nodes Select the row Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8992) CommitLogTest hangs intermittently
[ https://issues.apache.org/jira/browse/CASSANDRA-8992?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8992: -- Reviewer: Benedict [~benedict] to review CommitLogTest hangs intermittently -- Key: CASSANDRA-8992 URL: https://issues.apache.org/jira/browse/CASSANDRA-8992 Project: Cassandra Issue Type: Bug Components: Tests Environment: trunk HEAD (commit 1279009) Reporter: Michael Shuler Assignee: Branimir Lambov Fix For: 3.0 Attachments: system.log CommitLogTest hangs about 20% of the time in trunk. I haven't seen this happen in 2.1, yet, but will have to loop over it to be sure. {noformat} 21:26:15 [junit] Testsuite: org.apache.cassandra.db.CommitLogTest 21:46:15 Build timed out (after 20 minutes). Marking the build as aborted. {noformat} I was able to repro locally, looping over the test and have attached the system.log from that repro. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8348) allow takeColumnFamilySnapshot to take a list of ColumnFamilies
[ https://issues.apache.org/jira/browse/CASSANDRA-8348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387011#comment-14387011 ] Jonathan Ellis commented on CASSANDRA-8348: --- [~nickmbailey] or [~hoangelos] can you review? allow takeColumnFamilySnapshot to take a list of ColumnFamilies --- Key: CASSANDRA-8348 URL: https://issues.apache.org/jira/browse/CASSANDRA-8348 Project: Cassandra Issue Type: Improvement Reporter: Peter Halliday Priority: Minor Fix For: 2.1.4 Attachments: Patch-8348.patch Within StorageServiceMBean.java the function takeSnapshot allows for a list of keyspaces to snapshot. However, the function takeColumnFamilySnapshot only allows for a single ColumnFamily to snapshot. This should allow for multiple ColumnFamilies within the same Keyspace. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9071) CQLSSTableWriter gives java.lang.AssertionError: Empty partition
[ https://issues.apache.org/jira/browse/CASSANDRA-9071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9071: --- Assignee: Benedict CQLSSTableWriter gives java.lang.AssertionError: Empty partition Key: CASSANDRA-9071 URL: https://issues.apache.org/jira/browse/CASSANDRA-9071 Project: Cassandra Issue Type: Bug Components: Tools Environment: java 7 / 8 cassandra 2.1.3 snapshot build locally with last commit https://github.com/apache/cassandra/commit/6ee4b0989d9a3ae3e704918622024fa57fdf63e7 macos Yosemite 10.10.2 Reporter: Ajit Joglekar Assignee: Benedict Fix For: 2.1.4 I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9071) CQLSSTableWriter gives java.lang.AssertionError: Empty partition
[ https://issues.apache.org/jira/browse/CASSANDRA-9071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9071: --- Fix Version/s: 2.1.4 CQLSSTableWriter gives java.lang.AssertionError: Empty partition Key: CASSANDRA-9071 URL: https://issues.apache.org/jira/browse/CASSANDRA-9071 Project: Cassandra Issue Type: Bug Components: Tools Environment: java 7 / 8 cassandra 2.1.3 snapshot build locally with last commit https://github.com/apache/cassandra/commit/6ee4b0989d9a3ae3e704918622024fa57fdf63e7 macos Yosemite 10.10.2 Reporter: Ajit Joglekar Assignee: Benedict Fix For: 2.1.4 I am always getting the following error: Exception in thread main java.lang.AssertionError: Empty partition at org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter$DiskWriter.run(SSTableSimpleUnsortedWriter.java:228) It happens at a certain point that seems to be repeatable. Only issue is I am converting 400 million records into multiple SSTables and creating small test is a challenge Last comment from Benedict looks relevant here https://issues.apache.org/jira/browse/CASSANDRA-8619 Is there a work around, quick fix, fix that I can try out locally? Thanks, -Ajit -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8773) cassandra-stress should validate its results in user mode
[ https://issues.apache.org/jira/browse/CASSANDRA-8773?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-8773: -- Reviewer: Robert Stupp Can you review, [~snazy]? cassandra-stress should validate its results in user mode --- Key: CASSANDRA-8773 URL: https://issues.apache.org/jira/browse/CASSANDRA-8773 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Benedict Assignee: Benedict Labels: stress Fix For: 2.1.4 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8894) Our default buffer size for (uncompressed) buffered reads should be smaller, and based on the expected record size
[ https://issues.apache.org/jira/browse/CASSANDRA-8894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387002#comment-14387002 ] Benedict commented on CASSANDRA-8894: - IMO CASSANDRA-8897 should be addressed first Our default buffer size for (uncompressed) buffered reads should be smaller, and based on the expected record size -- Key: CASSANDRA-8894 URL: https://issues.apache.org/jira/browse/CASSANDRA-8894 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Stefania Fix For: 3.0 A large contributor to slower buffered reads than mmapped is likely that we read a full 64Kb at once, when average record sizes may be as low as 140 bytes on our stress tests. The TLB has only 128 entries on a modern core, and each read will touch 32 of these, meaning we are unlikely to almost ever be hitting the TLB, and will be incurring at least 30 unnecessary misses each time (as well as the other costs of larger than necessary accesses). When working with an SSD there is little to no benefit reading more than 4Kb at once, and in either case reading more data than we need is wasteful. So, I propose selecting a buffer size that is the next larger power of 2 than our average record size (with a minimum of 4Kb), so that we expect to read in one operation. I also propose that we create a pool of these buffers up-front, and that we ensure they are all exactly aligned to a virtual page, so that the source and target operations each touch exactly one virtual page per 4Kb of expected record size. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8773) cassandra-stress should validate its results in user mode
[ https://issues.apache.org/jira/browse/CASSANDRA-8773?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387003#comment-14387003 ] Robert Stupp commented on CASSANDRA-8773: - Yes cassandra-stress should validate its results in user mode --- Key: CASSANDRA-8773 URL: https://issues.apache.org/jira/browse/CASSANDRA-8773 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Benedict Assignee: Benedict Labels: stress Fix For: 2.1.4 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9036) disk full when running cleanup (on a far from full disk)
[ https://issues.apache.org/jira/browse/CASSANDRA-9036?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-9036: -- Reviewer: Yuki Morishita (Marcus has a lot of reviews on his plate atm so giving this one to [~yukim].) disk full when running cleanup (on a far from full disk) -- Key: CASSANDRA-9036 URL: https://issues.apache.org/jira/browse/CASSANDRA-9036 Project: Cassandra Issue Type: Bug Reporter: Erik Forsberg Assignee: Robert Stupp Attachments: 9036-2.0.txt, 9036-2.1.txt, 9036-3.0.txt I'm trying to run cleanup, but get this: {noformat} INFO [CompactionExecutor:18] 2015-03-25 10:29:16,355 CompactionManager.java (line 564) Cleaning up SSTableReader(path='/cassandra/production/Data_daily/production-Data_daily-jb-4345750-Data.db') ERROR [CompactionExecutor:18] 2015-03-25 10:29:16,664 CassandraDaemon.java (line 199) Exception in thread Thread[CompactionExecutor:18,1,main] java.io.IOException: disk full at org.apache.cassandra.db.compaction.CompactionManager.doCleanupCompaction(CompactionManager.java:567) at org.apache.cassandra.db.compaction.CompactionManager.access$400(CompactionManager.java:63) at org.apache.cassandra.db.compaction.CompactionManager$5.perform(CompactionManager.java:281) at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:225) at java.util.concurrent.FutureTask.run(FutureTask.java:262) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} Now that's odd, since: * Disk has some 680G left * The sstable it's trying to cleanup is far less than 680G: {noformat} # ls -lh *4345750* -rw-r--r-- 1 cassandra cassandra 64M Mar 21 04:42 production-Data_daily-jb-4345750-CompressionInfo.db -rw-r--r-- 1 cassandra cassandra 219G Mar 21 04:42 production-Data_daily-jb-4345750-Data.db -rw-r--r-- 1 cassandra cassandra 503M Mar 21 04:42 production-Data_daily-jb-4345750-Filter.db -rw-r--r-- 1 cassandra cassandra 42G Mar 21 04:42 production-Data_daily-jb-4345750-Index.db -rw-r--r-- 1 cassandra cassandra 5.9K Mar 21 04:42 production-Data_daily-jb-4345750-Statistics.db -rw-r--r-- 1 cassandra cassandra 81M Mar 21 04:42 production-Data_daily-jb-4345750-Summary.db -rw-r--r-- 1 cassandra cassandra 79 Mar 21 04:42 production-Data_daily-jb-4345750-TOC.txt {noformat} Sure, it's large, but it's not 680G. No other compactions are running on that server. I'm getting this on 12 / 56 servers right now. Could it be some bug in the calculation of the expected size of the new sstable, perhaps? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7402) Add metrics to track memory used by client requests
[ https://issues.apache.org/jira/browse/CASSANDRA-7402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7402: -- Reviewer: Aleksey Yeschenko Priority: Minor (was: Major) Add metrics to track memory used by client requests --- Key: CASSANDRA-7402 URL: https://issues.apache.org/jira/browse/CASSANDRA-7402 Project: Cassandra Issue Type: Improvement Reporter: T Jake Luciani Assignee: T Jake Luciani Priority: Minor Labels: ops, performance, stability Fix For: 2.1.4 Attachments: 7402.txt When running a production cluster one common operational issue is quantifying GC pauses caused by ongoing requests. Since different queries return varying amount of data you can easily get your self into a situation where you Stop the world from a couple of bad actors in the system. Or more likely the aggregate garbage generated on a single node across all in flight requests causes a GC. It would be very useful for operators to see how much garbage the system is using to handle in flight mutations and queries. It would also be nice to have either a log of queries which generate the most garbage so operators can track this. Also a histogram. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8897) Remove FileCacheService, instead pooling the buffers
[ https://issues.apache.org/jira/browse/CASSANDRA-8897?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-8897: Assignee: Stefania (was: Benedict) Remove FileCacheService, instead pooling the buffers Key: CASSANDRA-8897 URL: https://issues.apache.org/jira/browse/CASSANDRA-8897 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Stefania Fix For: 3.0 After CASSANDRA-8893, a RAR will be a very lightweight object and will not need caching, so we can eliminate this cache entirely. Instead we should have a pool of buffers that are page-aligned. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8890) Enhance cassandra-env.sh to handle Java version output in case of OpenJDK icedtea
[ https://issues.apache.org/jira/browse/CASSANDRA-8890?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-8890: Attachment: 8890-v2.txt It seems that you're adding the complexity of awk to do cut's job. Patch to instead just use cut. Enhance cassandra-env.sh to handle Java version output in case of OpenJDK icedtea -- Key: CASSANDRA-8890 URL: https://issues.apache.org/jira/browse/CASSANDRA-8890 Project: Cassandra Issue Type: Improvement Components: Config Environment: Red Hat Enterprise Linux Server release 6.4 (Santiago) Reporter: Sumod Pawgi Priority: Minor Labels: conf, icedtea Fix For: 3.0 Attachments: 8890-v2.txt, trunk-8890.patch, trunk-8890.txt Where observed - Cassandra node has OpenJDK - java version 1.7.0_09-icedtea In some situations, external agents trying to monitor a C* cluster would need to run cassandra -v command to determine the Cassandra version and would expect a numerical output e.g. java version 1.7.0_75 as in case of Oracle JDK. But if the cluster has OpenJDK IcedTea installed, then this condition is not satisfied and the agents will not work correctly as the output from cassandra -v is /opt/apache/cassandra/bin/../conf/cassandra-env.sh: line 102: [: 09-icedtea: integer expression expected Cause - The line which is causing this behavior is - jvmver=`echo $java_ver_output | grep '[openjdk|java] version' | awk -F'' 'NR==1 {print $2}'` Suggested enhancement - If we change the line to - jvmver=`echo $java_ver_output | grep '[openjdk|java] version' | awk -F'' 'NR==1 {print $2}' | awk 'BEGIN {FS=-};{print $1}'`, it will give $jvmver as - 1.7.0_09 for the above case. Can we add this enhancement in the cassandra-env.sh? I would like to add it myself and submit for review, but I am not familiar with C* check in process. There might be better ways to do this, but I thought of this to be simplest and as the edition is at the end of the line, it will be easy to reverse if needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8890) Enhance cassandra-env.sh to handle Java version output in case of OpenJDK icedtea
[ https://issues.apache.org/jira/browse/CASSANDRA-8890?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-8890: Reviewer: Michael Shuler Enhance cassandra-env.sh to handle Java version output in case of OpenJDK icedtea -- Key: CASSANDRA-8890 URL: https://issues.apache.org/jira/browse/CASSANDRA-8890 Project: Cassandra Issue Type: Improvement Components: Config Environment: Red Hat Enterprise Linux Server release 6.4 (Santiago) Reporter: Sumod Pawgi Priority: Minor Labels: conf, icedtea Fix For: 3.0 Attachments: 8890-v2.txt, trunk-8890.patch, trunk-8890.txt Where observed - Cassandra node has OpenJDK - java version 1.7.0_09-icedtea In some situations, external agents trying to monitor a C* cluster would need to run cassandra -v command to determine the Cassandra version and would expect a numerical output e.g. java version 1.7.0_75 as in case of Oracle JDK. But if the cluster has OpenJDK IcedTea installed, then this condition is not satisfied and the agents will not work correctly as the output from cassandra -v is /opt/apache/cassandra/bin/../conf/cassandra-env.sh: line 102: [: 09-icedtea: integer expression expected Cause - The line which is causing this behavior is - jvmver=`echo $java_ver_output | grep '[openjdk|java] version' | awk -F'' 'NR==1 {print $2}'` Suggested enhancement - If we change the line to - jvmver=`echo $java_ver_output | grep '[openjdk|java] version' | awk -F'' 'NR==1 {print $2}' | awk 'BEGIN {FS=-};{print $1}'`, it will give $jvmver as - 1.7.0_09 for the above case. Can we add this enhancement in the cassandra-env.sh? I would like to add it myself and submit for review, but I am not familiar with C* check in process. There might be better ways to do this, but I thought of this to be simplest and as the edition is at the end of the line, it will be easy to reverse if needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9073) o.a.c.schema.LegacySchemaTablesTest.dropCf may be flapping
Ariel Weisberg created CASSANDRA-9073: - Summary: o.a.c.schema.LegacySchemaTablesTest.dropCf may be flapping Key: CASSANDRA-9073 URL: https://issues.apache.org/jira/browse/CASSANDRA-9073 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Ariel Weisberg Saw this fail on my Linux desktop. It doesn't fail every time. This is running trunk @ ff5ed7a03f7b9968c0156b05226af67882e5670e -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387265#comment-14387265 ] Philip Thompson commented on CASSANDRA-9045: No, I absolutely didn't mean columns. I'm using CQL terminology here, not thrift. Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9074) Hadoop Cassandra CqlInputFormat pagination - not reading all input rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9074?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] fuggy_yama updated CASSANDRA-9074: -- Description: I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? What if the amount of rows I need to read per cassandra node is very large - in other words how to avoid OOM when setting InputCQLPageRowSize very large to handle all data? was: I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? Hadoop Cassandra CqlInputFormat pagination - not reading all input rows --- Key: CASSANDRA-9074 URL: https://issues.apache.org/jira/browse/CASSANDRA-9074 Project: Cassandra Issue Type: Bug Components: Hadoop Environment: Cassandra 2.0.11, Hadoop 1.0.4, Datastax java cassandra-driver-core 2.1.4 Reporter: fuggy_yama Priority: Minor I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? What if the amount of rows I need to read per cassandra node is very large - in other words how to avoid OOM when setting InputCQLPageRowSize very large to handle all data? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9048) Delimited File Bulk Loader
[ https://issues.apache.org/jira/browse/CASSANDRA-9048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387311#comment-14387311 ] Jonathan Ellis commented on CASSANDRA-9048: --- No. Delimited File Bulk Loader -- Key: CASSANDRA-9048 URL: https://issues.apache.org/jira/browse/CASSANDRA-9048 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brian Hess Attachments: CASSANDRA-9048.patch There is a strong need for bulk loading data from delimited files into Cassandra. Starting with delimited files means that the data is not currently in the SSTable format, and therefore cannot immediately leverage Cassandra's bulk loading tool, sstableloader, directly. A tool supporting delimited files much closer matches the format of the data more often than the SSTable format itself, and a tool that loads from delimited files is very useful. In order for this bulk loader to be more generally useful to customers, it should handle a number of options at a minimum: - support specifying the input file or to read the data from stdin (so other command-line programs can pipe into the loader) - supply the CQL schema for the input data - support all data types other than collections (collections is a stretch goal/need) - an option to specify the delimiter - an option to specify comma as the decimal delimiter (for international use casese) - an option to specify how NULL values are specified in the file (e.g., the empty string or the string NULL) - an option to specify how BOOLEAN values are specified in the file (e.g., TRUE/FALSE or 0/1) - an option to specify the Date and Time format - an option to skip some number of rows at the beginning of the file - an option to only read in some number of rows from the file - an option to indicate how many parse errors to tolerate - an option to specify a file that will contain all the lines that did not parse correctly (up to the maximum number of parse errors) - an option to specify the CQL port to connect to (with 9042 as the default). Additional options would be useful, but this set of options/features is a start. A word on COPY. COPY comes via CQLSH which requires the client to be the same version as the server (e.g., 2.0 CQLSH does not work with 2.1 Cassandra, etc). This tool should be able to connect to any version of Cassandra (within reason). For example, it should be able to handle 2.0.x and 2.1.x. Moreover, CQLSH's COPY command does not support a number of the options above. Lastly, the performance of COPY in 2.0.x is not high enough to be considered a bulk ingest tool. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8821) Errors in JVM_OPTS and cassandra_parms environment vars
[ https://issues.apache.org/jira/browse/CASSANDRA-8821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Shuler updated CASSANDRA-8821: -- Fix Version/s: 2.1.4 Errors in JVM_OPTS and cassandra_parms environment vars --- Key: CASSANDRA-8821 URL: https://issues.apache.org/jira/browse/CASSANDRA-8821 Project: Cassandra Issue Type: Bug Environment: Ubuntu 14.04 LTS amd64 Reporter: Terry Moschou Assignee: Michael Shuler Priority: Minor Fix For: 2.1.4, 2.0.14 Attachments: 8821_2.0.txt, 8821_2.1.txt Repos: deb http://www.apache.org/dist/cassandra/debian 21x main deb-src http://www.apache.org/dist/cassandra/debian 21x main The cassandra init script /etc/init.d/cassandra is sourcing the environment file /etc/cassandra/cassandra-env.sh twice. Once directly from the init script, and again inside /usr/sbin/cassandra The result is arguments in JVM_OPTS are duplicated. Further the JVM opt -XX:CMSWaitDuration=1 is defined twice if jvm = 1.7.60. Also, for the environment variable CASSANDRA_CONF used in this context -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler is undefined when /etc/cassandra/cassandra-env.sh is sourced from the init script. Lastly the variable cassandra_storagedir is undefined in /usr/sbin/cassandra when used in this context -Dcassandra.storagedir=$cassandra_storagedir -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387265#comment-14387265 ] Philip Thompson edited comment on CASSANDRA-9045 at 3/30/15 7:24 PM: - No, I absolutely didn't mean columns. I'm using CQL terminology here, not thrift. And yes, I saw what you said about the digest mismatch, and messaged [~thobbs]. I'm not sure what the implications are for that. was (Author: philipthompson): No, I absolutely didn't mean columns. I'm using CQL terminology here, not thrift. Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9074) Hadoop Cassandra CqlInputFormat pagination - not reading all input rows
fuggy_yama created CASSANDRA-9074: - Summary: Hadoop Cassandra CqlInputFormat pagination - not reading all input rows Key: CASSANDRA-9074 URL: https://issues.apache.org/jira/browse/CASSANDRA-9074 Project: Cassandra Issue Type: Bug Components: Hadoop Environment: Cassandra 2.0.11, Hadoop 1.0.4, Datastax java cassandra-driver-core 2.1.4 Reporter: fuggy_yama Priority: Minor I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9048) Delimited File Bulk Loader
[ https://issues.apache.org/jira/browse/CASSANDRA-9048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387295#comment-14387295 ] Sebastian Estevez commented on CASSANDRA-9048: -- Will the new COPY command be back ported or made compatible with the 2.0 branch? Delimited File Bulk Loader -- Key: CASSANDRA-9048 URL: https://issues.apache.org/jira/browse/CASSANDRA-9048 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brian Hess Attachments: CASSANDRA-9048.patch There is a strong need for bulk loading data from delimited files into Cassandra. Starting with delimited files means that the data is not currently in the SSTable format, and therefore cannot immediately leverage Cassandra's bulk loading tool, sstableloader, directly. A tool supporting delimited files much closer matches the format of the data more often than the SSTable format itself, and a tool that loads from delimited files is very useful. In order for this bulk loader to be more generally useful to customers, it should handle a number of options at a minimum: - support specifying the input file or to read the data from stdin (so other command-line programs can pipe into the loader) - supply the CQL schema for the input data - support all data types other than collections (collections is a stretch goal/need) - an option to specify the delimiter - an option to specify comma as the decimal delimiter (for international use casese) - an option to specify how NULL values are specified in the file (e.g., the empty string or the string NULL) - an option to specify how BOOLEAN values are specified in the file (e.g., TRUE/FALSE or 0/1) - an option to specify the Date and Time format - an option to skip some number of rows at the beginning of the file - an option to only read in some number of rows from the file - an option to indicate how many parse errors to tolerate - an option to specify a file that will contain all the lines that did not parse correctly (up to the maximum number of parse errors) - an option to specify the CQL port to connect to (with 9042 as the default). Additional options would be useful, but this set of options/features is a start. A word on COPY. COPY comes via CQLSH which requires the client to be the same version as the server (e.g., 2.0 CQLSH does not work with 2.1 Cassandra, etc). This tool should be able to connect to any version of Cassandra (within reason). For example, it should be able to handle 2.0.x and 2.1.x. Moreover, CQLSH's COPY command does not support a number of the options above. Lastly, the performance of COPY in 2.0.x is not high enough to be considered a bulk ingest tool. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9045) Deleted columns are resurrected after repair in wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387262#comment-14387262 ] Roman Tkachenko commented on CASSANDRA-9045: When you say rows do you actually mean columns? :) Did you guys see my comment about this affecting only certain columns and the digest mismatch thing? Can it be related? Deleted columns are resurrected after repair in wide rows - Key: CASSANDRA-9045 URL: https://issues.apache.org/jira/browse/CASSANDRA-9045 Project: Cassandra Issue Type: Bug Components: Core Reporter: Roman Tkachenko Assignee: Marcus Eriksson Priority: Critical Fix For: 2.0.14 Attachments: cqlsh.txt Hey guys, After almost a week of researching the issue and trying out multiple things with (almost) no luck I was suggested (on the user@cass list) to file a report here. h5. Setup Cassandra 2.0.13 (we had the issue with 2.0.10 as well and upgraded to see if it goes away) Multi datacenter 12+6 nodes cluster. h5. Schema {code} cqlsh describe keyspace blackbook; CREATE KEYSPACE blackbook WITH replication = { 'class': 'NetworkTopologyStrategy', 'IAD': '3', 'ORD': '3' }; USE blackbook; CREATE TABLE bounces ( domainid text, address text, message text, timestamp bigint, PRIMARY KEY (domainid, address) ) WITH bloom_filter_fp_chance=0.10 AND caching='KEYS_ONLY' AND comment='' AND dclocal_read_repair_chance=0.10 AND gc_grace_seconds=864000 AND index_interval=128 AND read_repair_chance=0.00 AND populate_io_cache_on_flush='false' AND default_time_to_live=0 AND speculative_retry='99.0PERCENTILE' AND memtable_flush_period_in_ms=0 AND compaction={'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} h5. Use case Each row (defined by a domainid) can have many many columns (bounce entries) so rows can get pretty wide. In practice, most of the rows are not that big but some of them contain hundreds of thousands and even millions of columns. Columns are not TTL'ed but can be deleted using the following CQL3 statement: {code} delete from bounces where domainid = 'domain.com' and address = 'al...@example.com'; {code} All queries are performed using LOCAL_QUORUM CL. h5. Problem We weren't very diligent about running repairs on the cluster initially, but shorty after we started doing it we noticed that some of previously deleted columns (bounce entries) are there again, as if tombstones have disappeared. I have run this test multiple times via cqlsh, on the row of the customer who originally reported the issue: * delete an entry * verify it's not returned even with CL=ALL * run repair on nodes that own this row's key * the columns reappear and are returned even with CL=ALL I tried the same test on another row with much less data and everything was correctly deleted and didn't reappear after repair. h5. Other steps I've taken so far Made sure NTP is running on all servers and clocks are synchronized. Increased gc_grace_seconds to 100 days, ran full repair (on the affected keyspace) on all nodes, then changed it back to the default 10 days again. Didn't help. Performed one more test. Updated one of the resurrected columns, then deleted it and ran repair again. This time the updated version of the column reappeared. Finally, I noticed these log entries for the row in question: {code} INFO [ValidationExecutor:77] 2015-03-25 20:27:43,936 CompactionController.java (line 192) Compacting large row blackbook/bounces:4ed558feba8a483733001d6a (279067683 bytes) incrementally {code} Figuring it may be related I bumped in_memory_compaction_limit_in_mb to 512MB so the row fits into it, deleted the entry and ran repair once again. The log entry for this row was gone and the columns didn't reappear. We have a lot of rows much larger than 512MB so can't increase this parameters forever, if that is the issue. Please let me know if you need more information on the case or if I can run more experiments. Thanks! Roman -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-8484) Value of variable JVM_OPTS is duplicated in command line arguments
[ https://issues.apache.org/jira/browse/CASSANDRA-8484?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Shuler resolved CASSANDRA-8484. --- Resolution: Duplicate Value of variable JVM_OPTS is duplicated in command line arguments -- Key: CASSANDRA-8484 URL: https://issues.apache.org/jira/browse/CASSANDRA-8484 Project: Cassandra Issue Type: Bug Components: Config Environment: debian/ubuntu Reporter: Andrey Trubachev Assignee: Michael Shuler Priority: Minor Fix For: 2.1.4, 2.0.14 For example: {noformat} $ ps aux | grep cassandra cassand+ 322 100 27.3 14942216 8995592 ?SLl 16:28 117:30 java -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.8.jar -XX:+CMSClassUnloadingEnabled -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms8043M -Xmx8043M -Xmn1600M -XX:+HeapDumpOnOutOfMemoryError -Xss256k -XX:StringTableSize=103 -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=1 -XX:CMSInitiatingOccupancyFraction=75 -XX:+UseCMSInitiatingOccupancyOnly -XX:+UseTLAB -XX:CompileCommandFile=/hotspot_compiler -XX:CMSWaitDuration=1 -XX:+UseCondCardMark -Djava.net.preferIPv6Addresses=true -Dcom.sun.management.jmxremote.port=7199 -Dcom.sun.management.jmxremote.rmi.port=7199 -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcassandra.metricsReporterConfigFile=/etc/vs/cassandra/graphite.yaml -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.8.jar -XX:+CMSClassUnloadingEnabled -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms8043M -Xmx8043M -Xmn1600M -XX:+HeapDumpOnOutOfMemoryError -Xss256k -XX:StringTableSize=103 -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=1 -XX:CMSInitiatingOccupancyFraction=75 -XX:+UseCMSInitiatingOccupancyOnly -XX:+UseTLAB -XX:CompileCommandFile=/etc/cassandra/hotspot_compiler -XX:CMSWaitDuration=1 -XX:+UseCondCardMark -Djava.net.preferIPv6Addresses=true -Dcom.sun.management.jmxremote.port=7199 -Dcom.sun.management.jmxremote.rmi.port=7199 -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcassandra.metricsReporterConfigFile=/etc/vs/cassandra/graphite.yaml -Dlogback.configurationFile=logback.xml -Dcassandra.logdir=/var/log/cassandra -Dcassandra.storagedir= -Dcassandra-pidfile=/var/run/cassandra/cassandra.pid -cp /etc/cassandra:/usr/share/cassandra/lib/airline-0.6.jar:/usr/share/cassandra/lib/antlr-runtime-3.5.2.jar:/usr/share/cassandra/lib/commons-cli-1.1.jar:/usr/share/cassandra/lib/commons-codec-1.2.jar:/usr/share/cassandra/lib/commons-lang3-3.1.jar:/usr/share/cassandra/lib/commons-math3-3.2.jar:/usr/share/cassandra/lib/compress-lzf-0.8.4.jar:/usr/share/cassandra/lib/concurrentlinkedhashmap-lru-1.4.jar:/usr/share/cassandra/lib/disruptor-3.0.1.jar:/usr/share/cassandra/lib/guava-16.0.jar:/usr/share/cassandra/lib/high-scale-lib-1.0.6.jar:/usr/share/cassandra/lib/jackson-core-asl-1.9.2.jar:/usr/share/cassandra/lib/jackson-mapper-asl-1.9.2.jar:/usr/share/cassandra/lib/jamm-0.2.8.jar:/usr/share/cassandra/lib/javax.inject.jar:/usr/share/cassandra/lib/jbcrypt-0.3m.jar:/usr/share/cassandra/lib/jline-1.0.jar:/usr/share/cassandra/lib/jna-4.0.0.jar:/usr/share/cassandra/lib/json-simple-1.1.jar:/usr/share/cassandra/lib/libthrift-0.9.1.jar:/usr/share/cassandra/lib/logback-classic-1.1.2.jar:/usr/share/cassandra/lib/logback-core-1.1.2.jar:/usr/share/cassandra/lib/lz4-1.2.0.jar:/usr/share/cassandra/lib/metrics-core-2.2.0.jar:/usr/share/cassandra/lib/metrics-graphite-2.2.0.jar:/usr/share/cassandra/lib/netty-all-4.0.23.Final.jar:/usr/share/cassandra/lib/reporter-config-2.2.0-SNAPSHOT.jar:/usr/share/cassandra/lib/slf4j-api-1.7.2.jar:/usr/share/cassandra/lib/snakeyaml-1.11.jar:/usr/share/cassandra/lib/snappy-java-1.0.5.2.jar:/usr/share/cassandra/lib/stream-2.5.2.jar:/usr/share/cassandra/lib/stringtemplate-4.0.2.jar:/usr/share/cassandra/lib/super-csv-2.1.0.jar:/usr/share/cassandra/lib/thrift-server-0.3.7.jar:/usr/share/cassandra/apache-cassandra-2.1.2.jar:/usr/share/cassandra/apache-cassandra.jar:/usr/share/cassandra/apache-cassandra-thrift-2.1.2.jar:/usr/share/cassandra/cassandra-driver-core-2.0.5.jar:/usr/share/cassandra/netty-3.9.0.Final.jar:/usr/share/cassandra/stress.jar: -XX:HeapDumpPath=/var/lib/cassandra/java_1418650085.hprof -XX:ErrorFile=/var/lib/cassandra/hs_err_1418650085.log org.apache.cassandra.service.CassandraDaemon {noformat} Variable JVM_OPTS isn't cleaned up in cassandra-env.sh. And cassandra-env.sh is called twice: first time in /etc/init.d/cassandra and second time in /usr/sbin/cassandra. {noformat} $ fgrep
[jira] [Commented] (CASSANDRA-8917) Upgrading from 2.0.9 to 2.1.3 with 3 nodes, CL = quorum causes exceptions
[ https://issues.apache.org/jira/browse/CASSANDRA-8917?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387300#comment-14387300 ] Gary Ogden commented on CASSANDRA-8917: --- I ran nodetool status on each node and had the exact same result: {quote} Datacenter: PRODDC1 === Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns Host ID Rack UN 10.6.71.204 122.39 GB 256 31.2% 93772457-9f70-42ea-89f2-a63d40d76703 RAC2 UN 10.6.71.205 123.49 GB 256 36.3% db0e2389-bbe5-43e4-b0e9-c99aff0449b8 RAC2 UN 10.6.71.198 122.45 GB 256 32.6% c0123329-3262-45a6-a6df-c3fe1b1b2978 RAC2 [gary@secasprddb01-2 ~]$ nodetool status company Datacenter: PRODDC1 === Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns (effective) Host ID Rack UN 10.6.71.204 122.39 GB 256 100.0% 93772457-9f70-42ea-89f2-a63d40d76703 RAC2 UN 10.6.71.205 123.49 GB 256 100.0% db0e2389-bbe5-43e4-b0e9-c99aff0449b8 RAC2 UN 10.6.71.198 122.45 GB 256 100.0% c0123329-3262-45a6-a6df-c3fe1b1b2978 RAC2 {quote} And when I run the select * from system.peers against each node, it only ever shows the other 2 nodes. There's no extra old nodes in the list. Upgrading from 2.0.9 to 2.1.3 with 3 nodes, CL = quorum causes exceptions - Key: CASSANDRA-8917 URL: https://issues.apache.org/jira/browse/CASSANDRA-8917 Project: Cassandra Issue Type: Bug Environment: C* 2.0.9, Centos 6.5, Java 1.7.0_72, spring data cassandra 1.1.1, cassandra java driver 2.0.9 Reporter: Gary Ogden Fix For: 2.1.4 Attachments: b_output.log, jersey_error.log, node1-cassandra.yaml, node1-system.log, node2-cassandra.yaml, node2-system.log, node3-cassandra.yaml, node3-system.log We have java apps running on glassfish that read/write to our 3 node cluster running on 2.0.9. we have the CL set to quorum for all reads and writes. When we started to upgrade the first node and did the sstable upgrade on that node, we started getting this error on reads and writes: com.datastax.driver.core.exceptions.UnavailableException: Not enough replica available for query at consistency QUORUM (2 required but only 1 alive) How is that possible when we have 3 nodes total, and there was 2 that were up and it's saying we can't get the required CL? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9074) Hadoop Cassandra CqlInputFormat pagination - not reading all input rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9074?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9074: --- Assignee: Alex Liu Hadoop Cassandra CqlInputFormat pagination - not reading all input rows --- Key: CASSANDRA-9074 URL: https://issues.apache.org/jira/browse/CASSANDRA-9074 Project: Cassandra Issue Type: Bug Components: Hadoop Environment: Cassandra 2.0.11, Hadoop 1.0.4, Datastax java cassandra-driver-core 2.1.4 Reporter: fuggy_yama Assignee: Alex Liu Priority: Minor Fix For: 2.0.14 I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? What if the amount of rows I need to read per cassandra node is very large - in other words how to avoid OOM when setting InputCQLPageRowSize very large to handle all data? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9074) Hadoop Cassandra CqlInputFormat pagination - not reading all input rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387392#comment-14387392 ] Alex Liu commented on CASSANDRA-9074: - Please try the latest 2.1.x or 2.0.x branch, it should have been fixed. Hadoop Cassandra CqlInputFormat pagination - not reading all input rows --- Key: CASSANDRA-9074 URL: https://issues.apache.org/jira/browse/CASSANDRA-9074 Project: Cassandra Issue Type: Bug Components: Hadoop Environment: Cassandra 2.0.11, Hadoop 1.0.4, Datastax java cassandra-driver-core 2.1.4 Reporter: fuggy_yama Assignee: Alex Liu Priority: Minor Fix For: 2.0.14 I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? What if the amount of rows I need to read per cassandra node is very large - in other words how to avoid OOM when setting InputCQLPageRowSize very large to handle all data? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[5/6] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5abab57b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5abab57b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5abab57b Branch: refs/heads/cassandra-2.1 Commit: 5abab57b93817dfac939eb13eac812091d3655c6 Parents: 6ee4b09 0ff9c3c Author: Yuki Morishita yu...@apache.org Authored: Mon Mar 30 17:02:11 2015 -0500 Committer: Yuki Morishita yu...@apache.org Committed: Mon Mar 30 17:02:11 2015 -0500 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/5abab57b/CHANGES.txt -- diff --cc CHANGES.txt index bd5e277,539c4c8..00b632d --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -87,6 -24,10 +87,7 @@@ Merged from 2.0 * Fix CQLSSTableWriter throwing exception and spawning threads (CASSANDRA-8808) * Fix MT mismatch between empty and GC-able data (CASSANDRA-8979) + * Fix incorrect validation when snapshotting single table (CASSANDRA-8056) - - -2.0.13: * Add offline tool to relevel sstables (CASSANDRA-8301) * Preserve stream ID for more protocol errors (CASSANDRA-8848) * Fix combining token() function with multi-column relations on http://git-wip-us.apache.org/repos/asf/cassandra/blob/5abab57b/src/java/org/apache/cassandra/service/StorageService.java --
[1/6] cassandra git commit: Allow reusing snapshot tags across different column families.
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 102f59c64 - 0ff9c3cb2 refs/heads/cassandra-2.1 6ee4b0989 - 5abab57b9 refs/heads/trunk ff5ed7a03 - a7e6e0f55 Allow reusing snapshot tags across different column families. patch by Yifu Wang; reviewed by yukim for CASSANDRA-8056 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0ff9c3cb Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0ff9c3cb Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0ff9c3cb Branch: refs/heads/cassandra-2.0 Commit: 0ff9c3cb222be634e46c378bb357ba0658e0e55b Parents: 102f59c Author: Yifu Wang yifuwang2...@gmail.com Authored: Sat Nov 29 00:45:27 2014 -0800 Committer: Yuki Morishita yu...@apache.org Committed: Mon Mar 30 16:59:31 2015 -0500 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ff9c3cb/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 9494f61..539c4c8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -24,6 +24,7 @@ * Fix CQLSSTableWriter throwing exception and spawning threads (CASSANDRA-8808) * Fix MT mismatch between empty and GC-able data (CASSANDRA-8979) + * Fix incorrect validation when snapshotting single table (CASSANDRA-8056) 2.0.13: http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ff9c3cb/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index d997459..cfd8fe1 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2257,10 +2257,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE throw new IOException(You must supply a snapshot name.); Keyspace keyspace = getValidKeyspace(keyspaceName); -if (keyspace.snapshotExists(tag)) +ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(columnFamilyName); +if (columnFamilyStore.snapshotExists(tag)) throw new IOException(Snapshot + tag + already exists.); -keyspace.snapshot(tag, columnFamilyName); +columnFamilyStore.snapshot(tag); } private Keyspace getValidKeyspace(String keyspaceName) throws IOException
[2/6] cassandra git commit: Allow reusing snapshot tags across different column families.
Allow reusing snapshot tags across different column families. patch by Yifu Wang; reviewed by yukim for CASSANDRA-8056 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0ff9c3cb Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0ff9c3cb Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0ff9c3cb Branch: refs/heads/cassandra-2.1 Commit: 0ff9c3cb222be634e46c378bb357ba0658e0e55b Parents: 102f59c Author: Yifu Wang yifuwang2...@gmail.com Authored: Sat Nov 29 00:45:27 2014 -0800 Committer: Yuki Morishita yu...@apache.org Committed: Mon Mar 30 16:59:31 2015 -0500 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ff9c3cb/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 9494f61..539c4c8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -24,6 +24,7 @@ * Fix CQLSSTableWriter throwing exception and spawning threads (CASSANDRA-8808) * Fix MT mismatch between empty and GC-able data (CASSANDRA-8979) + * Fix incorrect validation when snapshotting single table (CASSANDRA-8056) 2.0.13: http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ff9c3cb/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index d997459..cfd8fe1 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2257,10 +2257,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE throw new IOException(You must supply a snapshot name.); Keyspace keyspace = getValidKeyspace(keyspaceName); -if (keyspace.snapshotExists(tag)) +ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(columnFamilyName); +if (columnFamilyStore.snapshotExists(tag)) throw new IOException(Snapshot + tag + already exists.); -keyspace.snapshot(tag, columnFamilyName); +columnFamilyStore.snapshot(tag); } private Keyspace getValidKeyspace(String keyspaceName) throws IOException
[4/6] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5abab57b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5abab57b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5abab57b Branch: refs/heads/trunk Commit: 5abab57b93817dfac939eb13eac812091d3655c6 Parents: 6ee4b09 0ff9c3c Author: Yuki Morishita yu...@apache.org Authored: Mon Mar 30 17:02:11 2015 -0500 Committer: Yuki Morishita yu...@apache.org Committed: Mon Mar 30 17:02:11 2015 -0500 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/5abab57b/CHANGES.txt -- diff --cc CHANGES.txt index bd5e277,539c4c8..00b632d --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -87,6 -24,10 +87,7 @@@ Merged from 2.0 * Fix CQLSSTableWriter throwing exception and spawning threads (CASSANDRA-8808) * Fix MT mismatch between empty and GC-able data (CASSANDRA-8979) + * Fix incorrect validation when snapshotting single table (CASSANDRA-8056) - - -2.0.13: * Add offline tool to relevel sstables (CASSANDRA-8301) * Preserve stream ID for more protocol errors (CASSANDRA-8848) * Fix combining token() function with multi-column relations on http://git-wip-us.apache.org/repos/asf/cassandra/blob/5abab57b/src/java/org/apache/cassandra/service/StorageService.java --
[6/6] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a7e6e0f5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a7e6e0f5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a7e6e0f5 Branch: refs/heads/trunk Commit: a7e6e0f55c1cf6f2a5668cdf00e08b8e963e9269 Parents: ff5ed7a 5abab57 Author: Yuki Morishita yu...@apache.org Authored: Mon Mar 30 17:02:16 2015 -0500 Committer: Yuki Morishita yu...@apache.org Committed: Mon Mar 30 17:02:16 2015 -0500 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7e6e0f5/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/a7e6e0f5/src/java/org/apache/cassandra/service/StorageService.java --
[3/6] cassandra git commit: Allow reusing snapshot tags across different column families.
Allow reusing snapshot tags across different column families. patch by Yifu Wang; reviewed by yukim for CASSANDRA-8056 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0ff9c3cb Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0ff9c3cb Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0ff9c3cb Branch: refs/heads/trunk Commit: 0ff9c3cb222be634e46c378bb357ba0658e0e55b Parents: 102f59c Author: Yifu Wang yifuwang2...@gmail.com Authored: Sat Nov 29 00:45:27 2014 -0800 Committer: Yuki Morishita yu...@apache.org Committed: Mon Mar 30 16:59:31 2015 -0500 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/StorageService.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ff9c3cb/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 9494f61..539c4c8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -24,6 +24,7 @@ * Fix CQLSSTableWriter throwing exception and spawning threads (CASSANDRA-8808) * Fix MT mismatch between empty and GC-able data (CASSANDRA-8979) + * Fix incorrect validation when snapshotting single table (CASSANDRA-8056) 2.0.13: http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ff9c3cb/src/java/org/apache/cassandra/service/StorageService.java -- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index d997459..cfd8fe1 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2257,10 +2257,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE throw new IOException(You must supply a snapshot name.); Keyspace keyspace = getValidKeyspace(keyspaceName); -if (keyspace.snapshotExists(tag)) +ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(columnFamilyName); +if (columnFamilyStore.snapshotExists(tag)) throw new IOException(Snapshot + tag + already exists.); -keyspace.snapshot(tag, columnFamilyName); +columnFamilyStore.snapshot(tag); } private Keyspace getValidKeyspace(String keyspaceName) throws IOException
[jira] [Resolved] (CASSANDRA-9075) Learning to submit Cassandra Jiras through an example
[ https://issues.apache.org/jira/browse/CASSANDRA-9075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Laura Adney resolved CASSANDRA-9075. Resolution: Fixed Was not meant to be saved. Learning to submit Cassandra Jiras through an example - Key: CASSANDRA-9075 URL: https://issues.apache.org/jira/browse/CASSANDRA-9075 Project: Cassandra Issue Type: Bug Reporter: Laura Adney -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9074) Hadoop Cassandra CqlInputFormat pagination - not reading all input rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14387547#comment-14387547 ] Philip Thompson commented on CASSANDRA-9074: [~alexliu68], do you know of the ticket, so I can close as duplicate? Hadoop Cassandra CqlInputFormat pagination - not reading all input rows --- Key: CASSANDRA-9074 URL: https://issues.apache.org/jira/browse/CASSANDRA-9074 Project: Cassandra Issue Type: Bug Components: Hadoop Environment: Cassandra 2.0.11, Hadoop 1.0.4, Datastax java cassandra-driver-core 2.1.4 Reporter: fuggy_yama Assignee: Alex Liu Priority: Minor Fix For: 2.0.14 I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? What if the amount of rows I need to read per cassandra node is very large - in other words how to avoid OOM when setting InputCQLPageRowSize very large to handle all data? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9074) Hadoop Cassandra CqlInputFormat pagination - not reading all input rows
[ https://issues.apache.org/jira/browse/CASSANDRA-9074?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9074: --- Fix Version/s: 2.0.14 Hadoop Cassandra CqlInputFormat pagination - not reading all input rows --- Key: CASSANDRA-9074 URL: https://issues.apache.org/jira/browse/CASSANDRA-9074 Project: Cassandra Issue Type: Bug Components: Hadoop Environment: Cassandra 2.0.11, Hadoop 1.0.4, Datastax java cassandra-driver-core 2.1.4 Reporter: fuggy_yama Priority: Minor Fix For: 2.0.14 I have a 7-node Cassandra (v2.0.11) cluster and a table with 10k rows. I run a hadoop job (datanodes reside on cassandra nodes of course) that reads data from that table and I see that only 7k rows is read to map phase. I checked CqlInputFormat source code and noticed that a CQL query is build to select node-local date and also LIMIT clause is added (1k default). So that 7k read rows can be explained: 7 nodes * 1k limit = 7k rows read total The limit can be changed using CqlConfigHelper: CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), 1000); Please help me with questions below: Is this a desired behavior? Why CqlInputFormat does not page through the rest of rows? Is it a bug or should I just increase the InputCQLPageRowSize value? What if I want to read all data in table and do not know the row count? What if the amount of rows I need to read per cassandra node is very large - in other words how to avoid OOM when setting InputCQLPageRowSize very large to handle all data? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9073) o.a.c.schema.LegacySchemaTablesTest.dropCf may be flapping
[ https://issues.apache.org/jira/browse/CASSANDRA-9073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-9073: -- Description: Saw this fail on my Linux desktop. It doesn't fail every time. This is running trunk @ ff5ed7a03f7b9968c0156b05226af67882e5670e This doesn't fail in cassci was:Saw this fail on my Linux desktop. It doesn't fail every time. This is running trunk @ ff5ed7a03f7b9968c0156b05226af67882e5670e o.a.c.schema.LegacySchemaTablesTest.dropCf may be flapping -- Key: CASSANDRA-9073 URL: https://issues.apache.org/jira/browse/CASSANDRA-9073 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Ariel Weisberg Saw this fail on my Linux desktop. It doesn't fail every time. This is running trunk @ ff5ed7a03f7b9968c0156b05226af67882e5670e This doesn't fail in cassci -- This message was sent by Atlassian JIRA (v6.3.4#6332)