[jira] [Commented] (CASSANDRA-4175) Reduce memory, disk space, and cpu usage with a column name/id map
[ https://issues.apache.org/jira/browse/CASSANDRA-4175?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13698701#comment-13698701 ] Terje Marthinussen commented on CASSANDRA-4175: --- Hi, Sorry for the late update. Yes, we have a cluster with some 20-30 billion columns (maybe even closer to 40 billion by now) which implements a column name map and has been in production for about 2 years. I was actually looking at committing this 2 years ago together with fairly large number of other changes which was implemented in the column/supercolumn serializer code but I never got around to implement a good way to push the sstable version numbers into the serializer to make things backwards compatible before focus moved resources elsewhere. As mentioned above by others, while not benchmarked and proven, I had a very good feeling the total change helped quite a bit on GC issues, memtables and a bit on performance in general, but in terms of disk space, the benefit was somewhat limited after sstable compression was implemented as the repeating column names are compressed pretty well. This is already 2 years ago (the cluster still runs by the way), but if memory serves me right: 30-40% reduction in disk space without compression 10% reduction on top of compression (I did a test after it was implemented). In my case, the implementation is actually hardcoded due to time constraints. A static map which is global for the entire cassandra installation. If committing this into cassandra, I believe my plan was split in 3. Possible as 3 different implementation stages: 1. A simple config option (as a config file or as a columnfamily) where users themselves can assign repeating column names. Sure, it is not as fancy as many other options, but maybe we could open up to cover some strange corner case usages here with things like substrings as well. Think options to cover complex versions of patterns like date/times such as 20130701202020 where a large chunk of the column name repeats, but not all of it. In the current implementation, if there is a mapping entry, it converts the string to a variable length integer which becomes the new column name. If there is no mapping entry, it stores the raw data. In our case, we have 40 repeating column names so I never need more than 1 byte, but the implementation would handle more if I had. I modified the sstable to add a bitmap at the start of each column to be able to turn on/off mapping entries, timestamps not used, TTL's and other things. There is a bunch of 64 bit numbers in the column format which only have default value in 99.999% of all cases and very often your column value is just an 8 byte int, a boolean or a short text entry. I think in 99% of the columns in this cassandra store, the column timestamp takes up more space than the column value. This would have been my first implementation. Mostly because I have a working implementation of it already and the mapping table would be very easy to move to a config file read at start of a column family similar to what we have for CF config but also here, it is a bit work to push such config data down to the serializer as the code was organized 2 years ago. Notice again, you do not need atomic handling of the updates to the map in any way in this implementation. You can add map entries at any time. The result after deserializing is always the same as column names can have a mix of raw and map id values thanks to the column feature bitmap that was introduced. 2. Auto learning feature with mapping table per sstable. This would be stage 2 of the implementation. When starting to create a new SSTable, build a sampling of the most frequently occuring column names and gradually start mapping them to ID's. Add the mapping table to the end of the SSTable or in a separate .map file (similar to index files) at the completion of sstable generation. The initial id mapping could be further improved by maintaining a global map of column names. This global map would not be used for serialization/deserialization. It would be used to pre-populate the value for a sstable and would only be statistics to optimize things further by reducing the number of mapping variances between sstables and reducing the number of raw values getting stored a bit more. The id map would still be local to each sstable in terms of storage, but having such statistics would allow you to dramatically reduce the size of a potentially shared id cache across sstables where a lot of mapping entries would be identical. Some may feel that we would run out of memory quickly or use a lot of extra disk with maps per sstable, but I guess that we only really need to deal with the top few thousand entries in each sstable and this would not be a problem to keep in a idmap cache in terms of size. This is really just the top X
[jira] [Comment Edited] (CASSANDRA-4175) Reduce memory, disk space, and cpu usage with a column name/id map
[ https://issues.apache.org/jira/browse/CASSANDRA-4175?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13698701#comment-13698701 ] Terje Marthinussen edited comment on CASSANDRA-4175 at 7/3/13 7:19 AM: --- Hi, Sorry for the late update. Yes, we have a cluster with some 20-30 billion columns (maybe even closer to 40 billion by now) which implements a column name map and has been in production for about 2 years. I was actually looking at committing this 2 years ago together with fairly large number of other changes which was implemented in the column/supercolumn serializer code but I never got around to implement a good way to push the sstable version numbers into the serializer to make things backwards compatible before focus moved resources elsewhere. As mentioned above by others, while not benchmarked and proven, I had a very good feeling the total change helped quite a bit on GC issues, memtables and a bit on performance in general, but in terms of disk space, the benefit was somewhat limited after sstable compression was implemented as the repeating column names are compressed pretty well. This is already 2 years ago (the cluster still runs by the way), but if memory serves me right: 30-40% reduction in disk space without compression 10% reduction on top of compression (I did a test after it was implemented). In my case, the implementation is actually hardcoded due to time constraints. A static map which is global for the entire cassandra installation. If committing this into cassandra, I believe my plan was split in 3. Possible as 3 different implementation stages: 1. A simple config option (as a config file or as a columnfamily) where users themselves can assign repeating column names. Sure, it is not as fancy as many other options, but maybe we could open up to cover some strange corner case usages here with things like substrings as well. Think options to cover complex versions of patterns like date/times such as 20130701202020 where a large chunk of the column name repeats, but not all of it. In the current implementation, if there is a mapping entry, it converts the string to a variable length integer which becomes the new column name. If there is no mapping entry, it stores the raw data. In our case, we have 40 repeating column names so I never need more than a 1 byte varint. I also modified the column format to add a column feature bitmap at the start of each column. This allowed me to turn on/off name/id mapping as well as things like TTL's and a handful of other meta data. There is a bunch of 64 bit numbers in the column format which only have default value in 99.999% of all cases and very often your column value is just an 8 byte int, a boolean or a short text entry. That is, in most cases the column meta data is many times larger than the value stored. This would have been my first implementation. Mostly because I have a working implementation of it already and the mapping table would be very easy to move to a config file with just a list of column names read at cassandra startup, or stored in a similar way to column family and other internal config (just as another keyspace for config). Unfortunately, it is a little bit work also to push such config data down to the serializer. At least as the code was organized 2 years ago. Notice again, you do not need any sort of atomic handling of the updates to the map in any way in this implementation. You can add map entries at any time. The result after deserializing is always the same as column names can have a mix of raw and map id values thanks to the column feature bitmap that was introduced. Entries that was stored as raw strings will eventually be replaced by ID's to the map as compaction clean things up. 2. Auto learning feature with mapping table per sstable. This would be stage 2 of the implementation. When starting to create a new SSTable, build a sampling of the most frequently occuring column names and gradually start mapping them to ID's. Add the mapping table to the end of the SSTable or in a separate .map file (similar to index files) at the completion of sstable generation. The initial id mapping could be further improved by maintaining a global map of column names. This global map would not be used for serialization/deserialization. It would be used to pre-populate the value for a sstable and would only be statistics to optimize things further by reducing the number of mapping variances between sstables and reducing the number of raw values getting stored a bit more. The id map would still be local to each sstable in terms of storage, but having such statistics would allow you to dramatically reduce the size of a potentially shared id cache across sstables where a lot of mapping entries would be identical. Some may feel that we would run out of
[jira] [Commented] (CASSANDRA-4175) Reduce memory, disk space, and cpu usage with a column name/id map
[ https://issues.apache.org/jira/browse/CASSANDRA-4175?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13698710#comment-13698710 ] Terje Marthinussen commented on CASSANDRA-4175: --- I should maybe add, 1 and 2 above does not exclude but rather complement each other. #1 is a manual map and could allow things like a prefix map such as '$201212' which will map all such prefixes to an id #2 is a auto map. It may require 1 if we want to consider to allow user to give hints to substring maps such as '$(201\d\d\d)' to map all year+month like string starting on 201 to a mapping entry. This will just be a hint. The sampling of number of entries should decide what gets mapped to avoid running out of memory. I am a bit unsure if these advanced features like substrings would never be used and should maybe only be implemented as some sort of substring detection separately. As this can be a bit processing intensive, substring statistics (top substrings) could be detected and maintained node wide in compaction and given as hints to the serializer later. Reduce memory, disk space, and cpu usage with a column name/id map -- Key: CASSANDRA-4175 URL: https://issues.apache.org/jira/browse/CASSANDRA-4175 Project: Cassandra Issue Type: Improvement Reporter: Jonathan Ellis Fix For: 2.1 We spend a lot of memory on column names, both transiently (during reads) and more permanently (in the row cache). Compression mitigates this on disk but not on the heap. The overhead is significant for typical small column values, e.g., ints. Even though we intern once we get to the memtable, this affects writes too via very high allocation rates in the young generation, hence more GC activity. Now that CQL3 provides us some guarantees that column names must be defined before they are inserted, we could create a map of (say) 32-bit int column id, to names, and use that internally right up until we return a resultset to the client. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-3024) sstable and message varint encoding
[ https://issues.apache.org/jira/browse/CASSANDRA-3024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13103640#comment-13103640 ] Terje Marthinussen commented on CASSANDRA-3024: --- For my test data (real life dataset using supercolumns) I got 10% additional size reduction over compression using vlq coded numbers on supercolumns/columns as well as column timestamps which are relative to the supercolumns rather than absolute timestamps. Already have some code for row relative timestamps, but not tested that yet sstable and message varint encoding --- Key: CASSANDRA-3024 URL: https://issues.apache.org/jira/browse/CASSANDRA-3024 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jonathan Ellis Priority: Minor Fix For: 1.1 We could save some sstable space by encoding longs and ints as vlong and vint, respectively. (Probably most short lengths would be better as vint as well.) -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-3024) sstable and message varint encoding
[ https://issues.apache.org/jira/browse/CASSANDRA-3024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13103647#comment-13103647 ] Terje Marthinussen commented on CASSANDRA-3024: --- As a side note, is there any problems vlq encoding some unneeded values. If they are that rare, they probably have no real impact on performance? I did not run any real benchmarks yet, but json2sstable actually ran a tiny bit faster (1-2%) with vlq encoding and relative timestamps. sstable and message varint encoding --- Key: CASSANDRA-3024 URL: https://issues.apache.org/jira/browse/CASSANDRA-3024 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jonathan Ellis Priority: Minor Fix For: 1.1 We could save some sstable space by encoding longs and ints as vlong and vint, respectively. (Probably most short lengths would be better as vint as well.) -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-3024) sstable and message varint encoding
[ https://issues.apache.org/jira/browse/CASSANDRA-3024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13103272#comment-13103272 ] Terje Marthinussen commented on CASSANDRA-3024: --- I have ported stuff related to handling this for columns/supercolumns (requires changes to size calculations as well) including more dense timestamp handling. I was looking quickly at the sstables as well as other places where we write short/ints/longs and I started realizing that modifying all of this gets a bit of a work. What about changing RandomAccessReader to something like (Descriptor desc,...) rather than (File file, ) and adding backwards compatibility there for fixed length/ variable length readInt/Long there? On the write side, modify SequentialWriter with overrides for variable length write methods. Any objections against doing this at such a low level? sstable and message varint encoding --- Key: CASSANDRA-3024 URL: https://issues.apache.org/jira/browse/CASSANDRA-3024 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jonathan Ellis Priority: Minor Fix For: 1.1 We could save some sstable space by encoding longs and ints as vlong and vint, respectively. (Probably most short lengths would be better as vint as well.) -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2995) Making Storage Engine Pluggable
[ https://issues.apache.org/jira/browse/CASSANDRA-2995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13089352#comment-13089352 ] Terje Marthinussen commented on CASSANDRA-2995: --- I have to agree with Stu here. Making the storage layer more pluggable and inspiring people to make alternative storage engines must be a good thing and the data model, or the functionality provided by it, is not magical. Also, I also increasingly feel we have too many pieces of code somehow touching to a larger or lesser degree the internals of code manipulating the internals of sstables today (compactions, streaming, cache updates, etc) and this increases risk of sstable corruptions, which there has been way too many of since the 0.7 branch was made. With ssd's and memory (for caching) dropping faster in price than most of us can track, I will (looking forwards) gladly take some performance penalties due to slight increase in random ops, in return for better isolation and reduced risk of data corruptions. Making Storage Engine Pluggable --- Key: CASSANDRA-2995 URL: https://issues.apache.org/jira/browse/CASSANDRA-2995 Project: Cassandra Issue Type: New Feature Components: Core Affects Versions: 0.8.2 Reporter: Muga Nishizawa Will you design and implement Cassandra's storage engine API like MyCassandra? MyCassandra provides extensible architecture for pluging other storage engines to Cassandra like MySQL. https://github.com/sunsuk7tp/MyCassandra/ It could be advantageous for Cassandra to make the storage engine pluggable. This could allow Cassandra to - deal with potential use cases where maybe the current sstables are not the best fit - allow several types of internal storage formats (at the same time) optimized for different data types - allow easier experiments and research on new storage formats (encourage research institutions to do strange things with Cassandra) - there could also be potential advantages from better isolation of the data engine in terms of less risk for data corruptions if other parts of Cassandra change -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13071504#comment-13071504 ] Terje Marthinussen commented on CASSANDRA-47: - Instead of on/off we could use size. In the cassandra we run, we have compression implemented on a supercolumn level. It turned out to be very good for performance for us not to compress data in memtables (which we would normally do with compression on supercolumns) or during flushing from memtables as both of these caused slowdown in the write path. Under heavy write activity, the resulting sstables from memtable flushes often gets pretty small (maybe avg. 20MB in our case) so compression does not really make much difference on disk consumption there, but the performance penalty does. All the compression/decompression on compacting the smallest tables also makes a noticable difference when trying to keep up on the compaction side. Instead we went for compression which only happens when a source sstable during compaction is larger than 4GB. I would recommend to consider similar functionality here. I started off with ning for our compression, but I now run the built in java deflate to get even better compression. Since we only compress the largest sstables, and do no other compression in the write path or on compaction of small sstables,the very slow compression of deflate does not bother us that much. The read side is of course still slower with inflate, but it is still more than fast enough to not be a problem. OS caching will also be better thanks to the better compression so we can regain some of the performance lost vs. ning/snappy there. We could also consider being very tunable with deflate for very large sstables, ning/snappy for smaller and no compression for the smallest, but I am not sure it is worth it. By the way, how much difference did you see on ning vs. snappy? When I tested it was not all that much difference and I felt ning was easier to bundle so to me it seemed like a better alternative. SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Labels: compression Fix For: 1.0 Attachments: CASSANDRA-47-v2.patch, CASSANDRA-47-v3-rebased.patch, CASSANDRA-47-v3.patch, CASSANDRA-47-v4.patch, CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13063788#comment-13063788 ] Terje Marthinussen commented on CASSANDRA-47: - Tried to import some data with json2sstable to compare sizes, but seems like json2sstable is not entirely working yet? SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Labels: compression Fix For: 1.0 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13063990#comment-13063990 ] Terje Marthinussen commented on CASSANDRA-47: - It starts making the file, but then it crashes. java.lang.IndexOutOfBoundsException at java.io.RandomAccessFile.readBytes(Native Method) at java.io.RandomAccessFile.read(RandomAccessFile.java:338) at org.apache.cassandra.io.compress.AbstractCompressedFile.readCompressedChunk(AbstractCompressedFile.java:169) at org.apache.cassandra.io.compress.AbstractCompressedFile.reBuffer(AbstractCompressedFile.java:85) at org.apache.cassandra.io.compress.CompressedDataFile$Output.writeAtMost(CompressedDataFile.java:420) at org.apache.cassandra.io.compress.CompressedDataFile$Output.write(CompressedDataFile.java:404) at org.apache.cassandra.db.ColumnIndexer.writeBloomFilter(ColumnIndexer.java:144) at org.apache.cassandra.db.ColumnIndexer.serializeInternal(ColumnIndexer.java:114) at org.apache.cassandra.db.ColumnIndexer.serialize(ColumnIndexer.java:50) at org.apache.cassandra.db.ColumnFamilySerializer.serializeWithIndexes(ColumnFamilySerializer.java:107) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:156) at org.apache.cassandra.tools.SSTableImport.importUnsorted(SSTableImport.java:290) at org.apache.cassandra.tools.SSTableImport.importJson(SSTableImport.java:252) at org.apache.cassandra.tools.SSTableImport.main(SSTableImport.java:476) ERROR: null I added a bit debug printing to the read/writeChunkLength logger.info(TNM: readChunkLength + len + in + this.path + at: + super.getFilePointer()); INFO 08:13:40,652 TNM: writeChunkLength 28511 in /var/folders/8d/8drNFwr6FEe7uEdizf5pAE+++TI/-Tmp-/Test-h-22-Data.db2879008826574155982compressing at: 1245194 INFO 08:13:40,655 TNM: writeChunkLength 65538 in /var/folders/8d/8drNFwr6FEe7uEdizf5pAE+++TI/-Tmp-/Test-h-22-Data.db2879008826574155982compressing at: 1310730 INFO 08:13:40,656 TNM: readChunkLength -1313507164 in /var/folders/8d/8drNFwr6FEe7uEdizf5pAE+++TI/-Tmp-/Test-h-22-Data.db2879008826574155982compressing at: 1376270 Importing 787 keys... Currently imported 1 keys. [Some lines deleted] INFO 15:10:18,219 TNM: writeChunkLength 14870 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1114122 INFO 15:10:18,219 TNM: readChunkLength 21106 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1179662 INFO 15:10:18,229 TNM: writeChunkLength 21104 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1179658 INFO 15:10:18,258 TNM: readChunkLength 26901 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1245198 INFO 15:10:18,260 TNM: writeChunkLength 26901 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1245194 INFO 15:10:18,260 TNM: readChunkLength 21104 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1179662 INFO 15:10:18,261 TNM: writeChunkLength 21106 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1179658 INFO 15:10:18,261 TNM: readChunkLength 26901 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1245198 INFO 15:20:18,348 TNM: writeChunkLength 26901 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1245194 INFO 15:20:18,370 TNM: readChunkLength 65545 in /Users/terje/work/cassandra-0.8/data/JP/Test-h-22-Data.db at: 1310734 And then the above exception Note, this is on OSX and there seems to be another report of problems with OSX above? SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Labels: compression Fix For: 1.0 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13064184#comment-13064184 ] Terje Marthinussen commented on CASSANDRA-47: - Sparse files is indeed a filesystem feature. It is standard in extfs, xfs and probably most of the common filesystems on linux (not considering FAT :)) It is not standard in NTFS, but applications can enable sparse files on a file by file basis there. But my bet, without checking, is that this is not really supported by java... For OSX, it is not supported on HFS, but is supposedly supported on UFS (but I have not tested on OSX, so only read this). Anyway, I agree (from experience) stay away from using sparseness in files if possible. It's a portability headache and it tends to cause major issues when it fails. SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Pavel Yaskevich Labels: compression Fix For: 1.0 Attachments: CASSANDRA-47.patch, snappy-java-1.0.3-rc4.jar We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059908#comment-13059908 ] Terje Marthinussen commented on CASSANDRA-2521: --- I found an error in how the patch had gotten merged in our code. After fixing that, I see no Compacted files around that I cannot explain. This found the the error: {noformat} public void releaseReference() { if (holdReferences.decrementAndGet() == 0 isCompacted.get()) { // Force finalizing mmapping if necessary ifile.cleanup(); dfile.cleanup(); deletingTask.schedule(); } assert holdReferences.get() = 0 : Reference counter + holdReferences.get() + for + dfile.path; } {noformat} that assert turned quite useful so a recommended addon. Been doing a lot of updating, repairing, querying since then on a few system here ranging from 3 nodes with 100-200 million docs to 12 nodes with more than a billion. Nothing abnormal so far. Things that made us run out of disk before or sent nodes into full GC land, now works. I would still like to see that full GC is not called if disk runs past 90% usage as and mmap is not used. No point halting the world if not needed? Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt, 2521-v4.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059445#comment-13059445 ] Terje Marthinussen commented on CASSANDRA-2816: --- Things definitely seems to be improved overall, but weird things still happens. So... 12 node cluster, this is maybe ugly, I know, but start repair on all of them. Most nodes are fine, but one goes crazy. Disk use is now 3-4 times what it was before the repair started, and it does not seem to be done yet. I have really no idea if this is the case, but I am getting the hunch that this node has ended up streaming out some of the data it is getting in. Would this be possible? Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059485#comment-13059485 ] Terje Marthinussen commented on CASSANDRA-2816: --- Cool! Then you confirmed what I have sort of believed for a while, but my understanding of code has been a bit in conflict with: http://wiki.apache.org/cassandra/Operations which says: It is safe to run repair against multiple machines at the same time, but to minimize the impact on your application workload it is recommended to wait for it to complete on one node before invoking it against the next. I have always read that as if you have the HW, go for it! May I change to: It is safe to run repair against multiple machines at the same time. However, to minimize the amount of data transferred during a repair, careful synchronization is required between the nodes taking part of the repair. This is difficult to do if nodes with the same data replicas runs repair at the same time and doing so can in extreme cases generate excessive transfers of data. Improvements is being worked on, but for now, avoid scheduling repair on several nodes with replicas of the same data at the same time. Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059637#comment-13059637 ] Terje Marthinussen commented on CASSANDRA-2816: --- Regardless of change of documentation however, I don't think it should be possible to actually trigger a scenario like this in the first place. The system should protect the user from that. I also noticed that in this case, we have RF3. The node which is going somewhat crazy is number 6, however during the repair, it does log that it talks compares and streams data with node 4, 5, 7 and 8. Seems like a couple of nodes too many? Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059636#comment-13059636 ] Terje Marthinussen commented on CASSANDRA-2816: --- Regardless of change of documentation however, I don't think it should be possible to actually trigger a scenario like this in the first place. The system should protect the user from that. I also noticed that in this case, we have RF3. The node which is going somewhat crazy is number 6, however during the repair, it does log that it talks compares and streams data with node 4, 5, 7 and 8. Seems like a couple of nodes too many? Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059655#comment-13059655 ] Terje Marthinussen commented on CASSANDRA-2816: --- bq.I'm not sure that in a p2p design we can posit an omniscient the system. Is that a philosophical statement? :) As Cassandra, at least for now, is a p2p network with fairly clearly defined boundaries, I will continue calling it a system for now :) However, looking at it from the p2p viewpoint, the user potentially have no clue about where replicas are stored and given this, it may be impossible for the user to issue repair manually on more than one node at a time without getting in trouble. Given a large enough p2p setup, it would also be non-trivial to actually schedule a complete repair without ending up with 2 or more repairs running on the same replica set. Since Cassandra do no checkpoint the synchronization so it is forced to rescan everything on every repair, repairs easily take so long that you are forced to run it on several nodes at a time if you are going to manage to finish repairing all nodes in 10 days... Anyway, this is way outside the scope of this jira :) Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059658#comment-13059658 ] Terje Marthinussen commented on CASSANDRA-2816: --- bq. I also noticed that in this case, we have RF3. The node which is going somewhat crazy is number 6, however during the repair, it does log that it talks compares and streams data with node 4, 5, 7 and 8. This is maybe correct. Node 7 will replicate to node 6 and 8 so 6 and 8 would share data. So, to make things safe, even with this patch, every 4th node can run repair at the same time if RF=3?, but you still need to run repair on each of those 4 nodes to make sure it is all repaired? Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Issue Comment Edited] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13059658#comment-13059658 ] Terje Marthinussen edited comment on CASSANDRA-2816 at 7/5/11 2:31 AM: --- bq. I also noticed that in this case, we have RF3. The node which is going somewhat crazy is number 6, however during the repair, it does log that it talks compares and streams data with node 4, 5, 7 and 8. This is maybe correct. Node 7 will replicate to node 6 and 8 so 6 and 8 would share data. So, to make things safe, even with this patch, every 4th node can run repair at the same time if RF=3?, but you still need to run repair on each of those 4 nodes to make sure it is all repaired? As for the comment I made earlier. To me, it looks like if the repair start triggering transfers on a large scale, the file the node get streamed in will not be streamed out, but this may get compacted before the repair finished and the compacted file I suspect gets streamed out and the repair just never finishe was (Author: terjem): bq. I also noticed that in this case, we have RF3. The node which is going somewhat crazy is number 6, however during the repair, it does log that it talks compares and streams data with node 4, 5, 7 and 8. This is maybe correct. Node 7 will replicate to node 6 and 8 so 6 and 8 would share data. So, to make things safe, even with this patch, every 4th node can run repair at the same time if RF=3?, but you still need to run repair on each of those 4 nodes to make sure it is all repaired? Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13057295#comment-13057295 ] Terje Marthinussen commented on CASSANDRA-2521: --- I have not found any further major issues here, but I think there is still situations where files are deleted late but they do seem to go away. Not sure if we are missing something in terms of reference counting and GC delete it eventually or it is just a delayed free or delete for some reason, but it does not happen too often. Will see try to add some debug logging and see what I find. I was looking at the code though and I am wondering about one segment. I have not had time to actually test this, but in submitUserDefined() there is a finally statement removing References for sstables but I could not immediately see where there are References acquired for all the sstables that needs to be freed there? I am sure it's just me missing something, but anyway... Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt, 2521-v4.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13057339#comment-13057339 ] Terje Marthinussen commented on CASSANDRA-2816: --- This is what heap looks like when GC start slowing things down so much that even gossip gets delayed long enough for nodes to be down for some seconds. num #instances #bytes class name -- 1: 9453188 453753024 java.nio.HeapByteBuffer 2: 10081546 392167064 [B 3: 7616875 24374 org.apache.cassandra.db.Column 4: 9739914 233757936 java.util.concurrent.ConcurrentSkipListMap$Node 5: 4131938 99166512 java.util.concurrent.ConcurrentSkipListMap$Index 6: 1549230 49575360 org.apache.cassandra.db.DeletedColumn I guess this really ends up maybe being the mix of everything going on in total and all the reading and writing that may occur when repair runs (valiadation compactions, streaming, normal compactions and regular traffic all at the same time and maybe many CFs at the same time). However, I have suspected for some time that our young size was a bit on the small side and after increasing it and giving the heap a few more GB to work with, it seems like things are behaving quite a bit better. I mentioned issues with this patch when testing for CASSANDRA-2521. That was a problem caused by me. Was playing around with git for the first time and I manage to apply 2816 to a different branch than the one I used for testing :( My appologies. Initial testing with that corrected looks a lot better for my small scale test case, but I noticed one time where I deleted an sstable and restarted. It did not get repaired (repair scanned but did nothing). Not entirely sure what to make out of that, I then tested to delete another sstable and repair started running. I will test more over the next days. Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13057614#comment-13057614 ] Terje Marthinussen commented on CASSANDRA-2521: --- In releaseReference(), if holdReferences for some reason gets less than 0, maybe the code should do an assert or throw an exception (or anything else that gives a stack trace)? Should help debugging for some error scenarios with reference mismatches. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt, 2521-v4.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13056645#comment-13056645 ] Terje Marthinussen commented on CASSANDRA-2521: --- Latest iteration of this patch look great in my case, but please note again, I have not tested mmap. Maybe you can make the full GC trigger conditional if mmap is used? Only thing missing beyond that is to get this into 0.8. Disk use is very significantly improved in some cases and I think this could easily be the biggest improvement you can do in 0.8.2 in terms of operation friendliness. Regarding the above mentioned tmp tables, there is definitely issues there, but don't think it is related to this patch so feel free to merge. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt, 2521-v4.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13055455#comment-13055455 ] Terje Marthinussen commented on CASSANDRA-2816: --- I don't know what causes GC when doing repairs either, but fire off repair on a few nodes with 100 million docs/node and there is a reasonable chance that a node here and there will log messages about reducing cache sizes due to memory pressure (I am not really sure it is a good idea to do this at all, reducing caches during stress rarely improves anything) or full GC. The thought about the master controlled compaction would not really affect network splits etc. Reconciliation after a network split is really as complex with or without a master. We need to get back to a state where all the nodes have the same data anyway which is a complex task anyway. This is more a consideration of the fact that we do not necessarily need to live in quorum based world during compaction and we are free to use alternative approaches in the compaction without changing read/write path or affecting availability. Master selection is not really a problem here. Start compaction, talk to other nodes with the same token ranges, select a leader. Does not even have to be the same master every time and could consider if we could make compaction part of a background read repair to reduce the amount of times we need to read/write data. For instance, if we can verify that the oldest/biggest sstables is 100% in sync with data on other replicas when it is compacted (why not do it during compaction when we go through the data anyway rather than later?),can we use that info to optimize the scans done during repairs by only using data in sstables with data received after some checkpoint in time as the starting point for the consistency check? Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13056239#comment-13056239 ] Terje Marthinussen commented on CASSANDRA-2521: --- First impression is good. I merged this with 2816 last night and tested overnight with a dataset that normally uses 32-40% disk space when compacted. Calling repair on a handful of nodes, I would normally have a good chance of seeing a node or two with 90%+ full disk and regular full GC's. With these patches, disk use seem to peak in the 55-65% disk area which is a very significant improvement. However, unfortunately, I started repairs on 5 of 12 nodes and one of them has gone crazy however and filled the disk 100%. :( I did happen to start repair twice on this node by accident. I somewhat do not think that is the real problem however, but I am not sure. Will make a ticket anyway on adding functionality to prevent repairs from getting started twice on the same CF and I will test more to see if this happens again. However I noticed this in the log: INFO [Thread-185] 2011-06-28 05:01:15,390 StorageService.java (line 2083) requesting GC to free disk space I guess we can get rid of that? Should be nothing good from calling full GC there anymore? If anything, maybe we should consider a variation here where cassandra instead aborts all repairs and compactions and clean up everything before trying again with smaller amounts of data in each compaction/repair attempt? Another ticket on that maybe? Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt, 2521-v4.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-2834) Avoid repair getting started twice at the same time for the same CF
Avoid repair getting started twice at the same time for the same CF --- Key: CASSANDRA-2834 URL: https://issues.apache.org/jira/browse/CASSANDRA-2834 Project: Cassandra Issue Type: Improvement Reporter: Terje Marthinussen It may seem like it is possible to start repair twice at the same time on the same CF? Not 100% verified, but if this is indeed the case, we may want to consider avoiding that including making nodetool repair abort and return and error if repair is attempted on the same CF as one which already have repair running. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13056263#comment-13056263 ] Terje Marthinussen commented on CASSANDRA-2521: --- Looking a bit more on files and logs from the node with full disk. I suspect things mainly went wrong due to 2 repairs (if they actually both ran) 1. Seems there are some Compacted files around which does not get cleaned up, but I guess these may be results of references acquired which are not freed as the streaming fills up the disk and fails. 2. there are no less but 53 -tmp- files. A lot of concurrent streams here! I still think it may be a good idea if we could detect that we were heading for full disk and abort everything before things crashed. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt, 2521-v4.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13055026#comment-13055026 ] Terje Marthinussen commented on CASSANDRA-2521: --- Did another repair overnight, one minor compaction included some 20 small sstables, all of them remains as well as a few from other compactions and the files from the repairs described before. Definitely something fishy here. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13055106#comment-13055106 ] Terje Marthinussen commented on CASSANDRA-2521: --- As for the last version of this patch, a quick look tonight shows access problems with markCurrentViewReferenced() which is called from outside cfs. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Issue Comment Edited] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054841#comment-13054841 ] Terje Marthinussen edited comment on CASSANDRA-2816 at 6/25/11 9:04 AM: This sounds very interesting. We have also spotted very noticable issues with full GCs when the merkle trees are passed around. Hopefully this could fix that too. I will see if I can get this patch tested somewhere if it is ready for that. On a side topic, given the importance of getting tombstones properly synchronized within GCGraceSeconds, would it be an potential interesting idea to separate tombstones in different sstables to reduce the need to scan the whole dataset very frequently in the first place? Another thought may be to make compaction deterministic or synchronized by a master across nodes so for older data, all we needed was to compare pre-stored md5s of how whole sstables? That is, while keeping the masterless design for updates, we could consider a master based design for how older data is being organized by the compactor. so it would be much easier to verify that old data is the same without any large regular scans and that data is really the same after big compactions etc. was (Author: terjem): Sounds good to me. This sounds very interesting. We have also spotted very noticable issues with full GCs when the merkle trees are passed around. Hopefully this could fix that too. I will see if I can get this patch tested somewhere if it is ready for that. On a side topic, given the importance of getting tombstones properly synchronized within GCGraceSeconds, would it be an potential interesting idea to separate tombstones in different sstables to reduce the need to scan the whole dataset very frequently in the first place? Another thought may be to make compaction deterministic or synchronized by a master across nodes so for older data, all we needed was to compare pre-stored md5s of how whole sstables? That is, while keeping the masterless design for updates, we could consider a master based design for how older data is being organized by the compactor. so it would be much easier to verify that old data is the same without any large regular scans and that data is really the same after big compactions etc. Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will
[jira] [Commented] (CASSANDRA-2816) Repair doesn't synchronize merkle tree creation properly
[ https://issues.apache.org/jira/browse/CASSANDRA-2816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054841#comment-13054841 ] Terje Marthinussen commented on CASSANDRA-2816: --- Sounds good to me. This sounds very interesting. We have also spotted very noticable issues with full GCs when the merkle trees are passed around. Hopefully this could fix that too. I will see if I can get this patch tested somewhere if it is ready for that. On a side topic, given the importance of getting tombstones properly synchronized within GCGraceSeconds, would it be an potential interesting idea to separate tombstones in different sstables to reduce the need to scan the whole dataset very frequently in the first place? Another thought may be to make compaction deterministic or synchronized by a master across nodes so for older data, all we needed was to compare pre-stored md5s of how whole sstables? That is, while keeping the masterless design for updates, we could consider a master based design for how older data is being organized by the compactor. so it would be much easier to verify that old data is the same without any large regular scans and that data is really the same after big compactions etc. Repair doesn't synchronize merkle tree creation properly Key: CASSANDRA-2816 URL: https://issues.apache.org/jira/browse/CASSANDRA-2816 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Labels: repair Fix For: 0.8.2 Attachments: 0001-Schedule-merkle-tree-request-one-by-one.patch Being a little slow, I just realized after having opened CASSANDRA-2811 and CASSANDRA-2815 that there is a more general problem with repair. When a repair is started, it will send a number of merkle tree to its neighbor as well as himself and assume for correction that the building of those trees will be started on every node roughly at the same time (if not, we end up comparing data snapshot at different time and will thus mistakenly repair a lot of useless data). This is bogus for many reasons: * Because validation compaction runs on the same executor that other compaction, the start of the validation on the different node is subject to other compactions. 0.8 mitigates this in a way by being multi-threaded (and thus there is less change to be blocked a long time by a long running compaction), but the compaction executor being bounded, its still a problem) * if you run a nodetool repair without arguments, it will repair every CFs. As a consequence it will generate lots of merkle tree requests and all of those requests will be issued at the same time. Because even in 0.8 the compaction executor is bounded, some of those validations will end up being queued behind the first ones. Even assuming that the different validation are submitted in the same order on each node (which isn't guaranteed either), there is no guarantee that on all nodes, the first validation will take the same time, hence desynchronizing the queued ones. Overall, it is important for the precision of repair that for a given CF and range (which is the unit at which trees are computed), we make sure that all node will start the validation at the same time (or, since we can't do magic, as close as possible). One (reasonably simple) proposition to fix this would be to have repair schedule validation compactions across nodes one by one (i.e, one CF/range at a time), waiting for all nodes to return their tree before submitting the next request. Then on each node, we should make sure that the node will start the validation compaction as soon as requested. For that, we probably want to have a specific executor for validation compaction and: * either we fail the whole repair whenever one node is not able to execute the validation compaction right away (because no thread are available right away). * we simply tell the user that if he start too many repairs in parallel, he may start seeing some of those repairing more data than it should. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054844#comment-13054844 ] Terje Marthinussen commented on CASSANDRA-2521: --- I have not tested Jonathan's updated version, but the version before seems to work good so far. Repair still keep data around while running, but that may be solved with CASSANDRA-2816. Other than that, sstables in general seems to disappear within a minute of being freed from compaction. Except for when running repair, I have to actively hunt for -Compacted files to find them now. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054904#comment-13054904 ] Terje Marthinussen commented on CASSANDRA-2521: --- Yes, no problem that files stay for a while during repair, but I believe currently, files are not deleted until the entire repair is finished even if they have completed streaming, and I suppose that is not needed? Regardless, I believe CASSANDRA-2816 may fix this? Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054906#comment-13054906 ] Terje Marthinussen commented on CASSANDRA-2521: --- When that is said, I have to actually verify that nothing is deleted until the repair is completed. I am pretty sure this happened with GC doing the cleanup, but I have actually not verified that with this patch. I just observe that I do not see anything get deleted until it finishes, but just because I don't see that with some occasional ls's does not mean it does not happen. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054941#comment-13054941 ] Terje Marthinussen commented on CASSANDRA-2521: --- Hm... there may actually be something here. I am getting some *Compacted which does not seem to go away, not even after 7 hours after finished. 3 Compactions 2 of them have left 1 sstable, one has left 2. I also happened to run 3 repairs, but one compaction starts just after one of those repairs finished (the other 2 starts and finished during the repair. This is running 2 as minimum compaction threshold. The compaction which left 2 files, left them after a compaction of 3 files. The 2 other *Compacted was made after compactions of 2 files The system is under reasonably heavy input of data and sstables are added, compacted and removed all the time. Just these which do not get removed and the oldest is 8 hours now. Note, I do not use mmap. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13054949#comment-13054949 ] Terje Marthinussen commented on CASSANDRA-2521: --- Hm, forget that. All 3 compactions happens during the repairs. This test is also using CASSANDRA-2433 and there are no errors reported during the repairs. Only one of the leftover sstables is listed in the log entries for stream context metadata Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch, 0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch, 2521-v3.txt http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13050309#comment-13050309 ] Terje Marthinussen commented on CASSANDRA-2521: --- Fabulous! If you backport that to 0.8 (or if it is back-portable), I will volunteer to give it a decent doze of real life testing asap. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2779) files not cleaned up by GC?
[ https://issues.apache.org/jira/browse/CASSANDRA-2779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13050314#comment-13050314 ] Terje Marthinussen commented on CASSANDRA-2779: --- If we get CASSANDRA-2521 in for 0.8 then first thing would be to check if this magically fixes itself. If not, I think I can volunteer to tune things further. Starting to get familiar with the code and I already have a setup to test and reproduce. files not cleaned up by GC? --- Key: CASSANDRA-2779 URL: https://issues.apache.org/jira/browse/CASSANDRA-2779 Project: Cassandra Issue Type: Bug Reporter: Terje Marthinussen This is 0.8.0 + a few 0.8.1 patches on repair. We tested repair on 2 nodes in the cluster last night. Interestingly enough, I don't believe the node described here is in any way neighbour of the nodes we tested repair on so I am not sure why it is streaming data both in and out, but in any case, it has joined the streaming party. We now see: ERROR [CompactionExecutor:5] 2011-06-16 09:12:23,928 CompactionManager.java (line 510) insufficient space to compact even the two smallest files, aborting INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space And we see a lot of them: INFO [CompactionExecutor:5] 2011-06-16 09:11:59,164 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:46,489 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:17:53,299 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:17,782 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:42,078 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:06,984 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:32,079 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:57,265 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:22,706 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:47,331 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:13,062 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:38,288 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:03,500 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:29,407 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:55,577 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:20,951 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:46,448 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:24:12,030 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:00,633 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:26,119 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:49,002 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:20,196 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:45,322 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:11:07,619 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:01:45,562 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:02:10,236 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:05:31,297 StorageService.java (line 2071) requesting GC to free disk space Available disk is 105GB and it is trying to compact a set of the largest sstables. There is probably easily enough disk to do so, but the
[jira] [Commented] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
[ https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13050773#comment-13050773 ] Terje Marthinussen commented on CASSANDRA-2521: --- We stopped using mmap a while back. Benchmarks was highly inconclusive about any performance benefits and the operational benefits of seeing how much memory the darned thing actually uses and general improved stabilty has so far easily outweighed any teorethical lower singel digit performance benefit we could find in static benchmarks and on more realistic use we so far don't really see any performance issues. Personally I have no problems with a patch with that limitation. Move away from Phantom References for Compaction/Memtable - Key: CASSANDRA-2521 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Goffinet Assignee: Sylvain Lebresne Fix For: 1.0 Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch http://wiki.apache.org/cassandra/MemtableSSTable Let's move to using reference counting instead of relying on GC to be called in StorageService. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-2779) files not cleaned up by GC?
files not cleaned up by GC? --- Key: CASSANDRA-2779 URL: https://issues.apache.org/jira/browse/CASSANDRA-2779 Project: Cassandra Issue Type: Bug Reporter: Terje Marthinussen Priority: Critical Fix For: 0.8.0 This is 0.8.0 + a few 0.8.1 patches on repair. We tested repair on 2 nodes in the cluster last night. Interestingly enough, I don't believe the node described here is in any way neighbour of the nodes we tested repair on so I am not sure why it is streaming data both in and out, but in any case, it has joined the streaming party. We now see: ERROR [CompactionExecutor:5] 2011-06-16 09:12:23,928 CompactionManager.java (line 510) insufficient space to compact even the two smallest files, aborting INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space And we see a lot of them: INFO [CompactionExecutor:5] 2011-06-16 09:11:59,164 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:46,489 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:17:53,299 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:17,782 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:42,078 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:06,984 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:32,079 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:57,265 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:22,706 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:47,331 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:13,062 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:38,288 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:03,500 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:29,407 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:55,577 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:20,951 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:46,448 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:24:12,030 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:00,633 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:26,119 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:49,002 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:20,196 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:45,322 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:11:07,619 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:01:45,562 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:02:10,236 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:05:31,297 StorageService.java (line 2071) requesting GC to free disk space Available disk is 105GB and it is trying to compact a set of the largest sstables. There is probably easily enough disk to do so, but the estimation is not sufficient (lots of dupes here after streaming I think, probably heavily affected by CASSANDRA-2698). It is trying to compact 2 sstables of 58 and 41GB. If I look at the data dir, I see 46 *Compacted files which makes up an additional 137GB of space. The oldest of these Compacted files dates back to Jun 16th 01:26, so 10 hours old. It does however succeed at cleaning up some files. There are definitely files which do get deleted. Just that there is a lot which is not. Either the GC cleanup tactic is seriously
[jira] [Commented] (CASSANDRA-2779) files not cleaned up by GC?
[ https://issues.apache.org/jira/browse/CASSANDRA-2779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13050191#comment-13050191 ] Terje Marthinussen commented on CASSANDRA-2779: --- Interestingly enough... Just as I posted this it started removing the files and the case study disappeared and started working fine. A few minutes before doing so, if completed manual repair, so yes, I guess repair is a likely candidate for keeping file references. If I remember right, it builds a list of files to stream at the start. They may not be dereferenced fully until the streaming finishes? files not cleaned up by GC? --- Key: CASSANDRA-2779 URL: https://issues.apache.org/jira/browse/CASSANDRA-2779 Project: Cassandra Issue Type: Bug Reporter: Terje Marthinussen Priority: Critical Fix For: 0.8.0 This is 0.8.0 + a few 0.8.1 patches on repair. We tested repair on 2 nodes in the cluster last night. Interestingly enough, I don't believe the node described here is in any way neighbour of the nodes we tested repair on so I am not sure why it is streaming data both in and out, but in any case, it has joined the streaming party. We now see: ERROR [CompactionExecutor:5] 2011-06-16 09:12:23,928 CompactionManager.java (line 510) insufficient space to compact even the two smallest files, aborting INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space And we see a lot of them: INFO [CompactionExecutor:5] 2011-06-16 09:11:59,164 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:46,489 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:17:53,299 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:17,782 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:42,078 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:06,984 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:32,079 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:57,265 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:22,706 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:47,331 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:13,062 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:38,288 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:03,500 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:29,407 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:55,577 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:20,951 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:46,448 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:24:12,030 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:00,633 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:26,119 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:49,002 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:20,196 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:45,322 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:11:07,619 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:01:45,562 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:02:10,236 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:05:31,297
[jira] [Updated] (CASSANDRA-2779) files not cleaned up by GC?
[ https://issues.apache.org/jira/browse/CASSANDRA-2779?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Terje Marthinussen updated CASSANDRA-2779: -- Priority: Major (was: Critical) Dropping to major again. Things do get cleaned up, eventually... files not cleaned up by GC? --- Key: CASSANDRA-2779 URL: https://issues.apache.org/jira/browse/CASSANDRA-2779 Project: Cassandra Issue Type: Bug Reporter: Terje Marthinussen Fix For: 0.8.0 This is 0.8.0 + a few 0.8.1 patches on repair. We tested repair on 2 nodes in the cluster last night. Interestingly enough, I don't believe the node described here is in any way neighbour of the nodes we tested repair on so I am not sure why it is streaming data both in and out, but in any case, it has joined the streaming party. We now see: ERROR [CompactionExecutor:5] 2011-06-16 09:12:23,928 CompactionManager.java (line 510) insufficient space to compact even the two smallest files, aborting INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space And we see a lot of them: INFO [CompactionExecutor:5] 2011-06-16 09:11:59,164 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:46,489 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:17:53,299 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:17,782 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:42,078 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:06,984 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:32,079 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:57,265 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:22,706 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:47,331 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:13,062 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:38,288 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:03,500 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:29,407 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:55,577 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:20,951 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:46,448 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:24:12,030 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:00,633 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:26,119 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:49,002 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:20,196 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:45,322 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:11:07,619 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:01:45,562 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:02:10,236 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:05:31,297 StorageService.java (line 2071) requesting GC to free disk space Available disk is 105GB and it is trying to compact a set of the largest sstables. There is probably easily enough disk to do so, but the estimation is not sufficient (lots of dupes here after streaming I think, probably heavily affected by CASSANDRA-2698). It is trying to compact 2 sstables of 58 and 41GB. If I
[jira] [Commented] (CASSANDRA-2779) files not cleaned up by GC?
[ https://issues.apache.org/jira/browse/CASSANDRA-2779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13050213#comment-13050213 ] Terje Marthinussen commented on CASSANDRA-2779: --- The problem is the disk use. I have no problem with the disk use if all recommendations for Cassandra is changed from typically 2x disk space required to hold the dataset to 4x disk space vs. what is needed to hold the dataset. Either we have to fix this issue to keep disk space down or we have to change the recommendations. files not cleaned up by GC? --- Key: CASSANDRA-2779 URL: https://issues.apache.org/jira/browse/CASSANDRA-2779 Project: Cassandra Issue Type: Bug Reporter: Terje Marthinussen This is 0.8.0 + a few 0.8.1 patches on repair. We tested repair on 2 nodes in the cluster last night. Interestingly enough, I don't believe the node described here is in any way neighbour of the nodes we tested repair on so I am not sure why it is streaming data both in and out, but in any case, it has joined the streaming party. We now see: ERROR [CompactionExecutor:5] 2011-06-16 09:12:23,928 CompactionManager.java (line 510) insufficient space to compact even the two smallest files, aborting INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space And we see a lot of them: INFO [CompactionExecutor:5] 2011-06-16 09:11:59,164 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:46,489 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:17:53,299 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:17,782 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:42,078 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:06,984 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:32,079 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:57,265 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:22,706 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:47,331 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:13,062 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:38,288 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:03,500 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:29,407 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:55,577 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:20,951 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:46,448 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:24:12,030 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:00,633 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:26,119 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:49,002 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:20,196 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:45,322 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:11:07,619 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:01:45,562 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:02:10,236 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:05:31,297 StorageService.java (line 2071) requesting GC to free disk space Available disk is 105GB and it is trying to compact a set of the
[jira] [Reopened] (CASSANDRA-2779) files not cleaned up by GC?
[ https://issues.apache.org/jira/browse/CASSANDRA-2779?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Terje Marthinussen reopened CASSANDRA-2779: --- Reopening until a decision is made to fix either documentation or disk usage, Things like: http://wiki.apache.org/cassandra/CassandraHardware As covered in MemtableSSTable, compactions can require up to 100% of your in-use space temporarily in the worst case should not exist, and in addition to an up to 100% increase of data from streaming, you need 100% for compaction and headroom to avoid full GC from running every few minutes like occured in this Jira (basically an unusable service) files not cleaned up by GC? --- Key: CASSANDRA-2779 URL: https://issues.apache.org/jira/browse/CASSANDRA-2779 Project: Cassandra Issue Type: Bug Reporter: Terje Marthinussen This is 0.8.0 + a few 0.8.1 patches on repair. We tested repair on 2 nodes in the cluster last night. Interestingly enough, I don't believe the node described here is in any way neighbour of the nodes we tested repair on so I am not sure why it is streaming data both in and out, but in any case, it has joined the streaming party. We now see: ERROR [CompactionExecutor:5] 2011-06-16 09:12:23,928 CompactionManager.java (line 510) insufficient space to compact even the two smallest files, aborting INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space And we see a lot of them: INFO [CompactionExecutor:5] 2011-06-16 09:11:59,164 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:23,929 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:5] 2011-06-16 09:12:46,489 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:17:53,299 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:17,782 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:18:42,078 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:06,984 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:32,079 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:19:57,265 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:22,706 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:20:47,331 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:13,062 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:21:38,288 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:03,500 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:29,407 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:22:55,577 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:20,951 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:23:46,448 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:3] 2011-06-16 09:24:12,030 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:00,633 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:26,119 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 09:48:49,002 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:20,196 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:10:45,322 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:6] 2011-06-16 10:11:07,619 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:01:45,562 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:02:10,236 StorageService.java (line 2071) requesting GC to free disk space INFO [CompactionExecutor:7] 2011-06-16 11:05:31,297 StorageService.java
[jira] [Created] (CASSANDRA-2766) ConcurrentModificationException during node recovery
ConcurrentModificationException during node recovery Key: CASSANDRA-2766 URL: https://issues.apache.org/jira/browse/CASSANDRA-2766 Project: Cassandra Issue Type: Bug Components: Core Reporter: Terje Marthinussen Testing some node recovery operations. In this case: 1. Data is being added/updated as it would in production 2. repair is running on other nodes in the cluster 3. we wiped data on this node and started up again, but before repair was actually started on this node (but it had gotten data through the regular data feed) we got this error. I see no indication in the logs that outgoing streams has been started, but the node have finished one incoming stream before this (I guess from some other node doing repair). INFO [CompactionExecutor:11] 2011-06-14 14:15:09,078 SSTableReader.java (line 155) Opening /data/cassandra/node1/data/JP/test-g-8 INFO [CompactionExecutor:13] 2011-06-14 14:15:09,079 SSTableReader.java (line 155) Opening /data/cassandra/node1/data/JP/test-g-10 INFO [HintedHandoff:1] 2011-06-14 14:15:26,623 HintedHandOffManager.java (line 302) Started hinted handoff for endpoint /1.10.42.216 INFO [HintedHandoff:1] 2011-06-14 14:15:26,623 HintedHandOffManager.java (line 358) Finished hinted handoff of 0 rows to endpoint /1.10.42.216 INFO [CompactionExecutor:9] 2011-06-14 14:15:29,417 SSTableReader.java (line 155) Opening /data/cassandra/node1/data/JP/Datetest-g-2 ERROR [Thread-84] 2011-06-14 14:15:36,755 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-84,5,main] java.util.ConcurrentModificationException at java.util.AbstractList$Itr.checkForComodification(AbstractList.java:372) at java.util.AbstractList$Itr.next(AbstractList.java:343) at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:132) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) ERROR [Thread-79] 2011-06-14 14:15:36,755 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-79,5,main] java.util.ConcurrentModificationException at java.util.AbstractList$Itr.checkForComodification(AbstractList.java:372) at java.util.AbstractList$Itr.next(AbstractList.java:343) at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:132) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) ERROR [Thread-83] 2011-06-14 14:15:36,755 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-83,5,main] java.util.ConcurrentModificationException at java.util.AbstractList$Itr.checkForComodification(AbstractList.java:372) at java.util.AbstractList$Itr.next(AbstractList.java:343) at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:132) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) ERROR [Thread-85] 2011-06-14 14:15:36,755 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-85,5,main] java.util.ConcurrentModificationException at java.util.AbstractList$Itr.checkForComodification(AbstractList.java:372) at java.util.AbstractList$Itr.next(AbstractList.java:343) at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:132) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2752) repair fails with java.io.EOFException
[ https://issues.apache.org/jira/browse/CASSANDRA-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13048655#comment-13048655 ] Terje Marthinussen commented on CASSANDRA-2752: --- SSTableWriter.java: ColumnFamily.serializer().deserializeFromSSTableNoColumns(ColumnFamily.create(cfs.metadata), dfile); // don't move that statement around, it expects the dfile to be before the columns updateCache(key, dataSize, null); rowSizes.add(dataSize); columnCounts.add(dfile.readInt()); I believe the problem is in updateCache. If rowcache is enabled (and it is in this case) and the row needs to be updated in cache, this will read (deserialize) the row. However, after all the columns is read, the offset in the file is not reset back to the location where the column count is stored and things go bad. I haven't actually tried to change the code to test, but I tried to disable the row cache, and so far, repair seems to work fine when it is disabled. repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread
[jira] [Created] (CASSANDRA-2758) nodetool repair never finishes. Loops forever through merkle trees
nodetool repair never finishes. Loops forever through merkle trees -- Key: CASSANDRA-2758 URL: https://issues.apache.org/jira/browse/CASSANDRA-2758 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen I am not sure all steps here is needed, but as part of testing something else, I set up node1: initial_token: 1 node2: initial_token: 5 Then: {noformat} create keyspace myks with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' with strategy_options = [{ replication_factor:2 }]; use myks; create column family test with comparator = AsciiType and column_metadata=[ {column_name: 'up_', validation_class: LongType, index_type: 0}, {column_name: 'del_', validation_class: LongType, index_type: 0} ] and keys_cached = 10 and rows_cached = 1 and min_compaction_threshold = 2; quit; {noformat} Doing nodetool repair after this gets both nodes busy looping forever. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-2758) nodetool repair never finishes. Loops forever through merkle trees?
[ https://issues.apache.org/jira/browse/CASSANDRA-2758?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Terje Marthinussen updated CASSANDRA-2758: -- Description: I am not sure all steps here is needed, but as part of testing something else, I set up node1: initial_token: 1 node2: initial_token: 5 Then: {noformat} create keyspace myks with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' with strategy_options = [{ replication_factor:2 }]; use myks; create column family test with comparator = AsciiType and column_metadata=[ {column_name: 'up_', validation_class: LongType, index_type: 0}, {column_name: 'del_', validation_class: LongType, index_type: 0} ] and keys_cached = 10 and rows_cached = 1 and min_compaction_threshold = 2; quit; {noformat} Doing nodetool repair after this gets both nodes busy looping forever. A quick look at one node in eclipse makes me guess its having fun spinning through merkle trees, but I have to admit I have not look at it for a long time. was: I am not sure all steps here is needed, but as part of testing something else, I set up node1: initial_token: 1 node2: initial_token: 5 Then: {noformat} create keyspace myks with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' with strategy_options = [{ replication_factor:2 }]; use myks; create column family test with comparator = AsciiType and column_metadata=[ {column_name: 'up_', validation_class: LongType, index_type: 0}, {column_name: 'del_', validation_class: LongType, index_type: 0} ] and keys_cached = 10 and rows_cached = 1 and min_compaction_threshold = 2; quit; {noformat} Doing nodetool repair after this gets both nodes busy looping forever. Summary: nodetool repair never finishes. Loops forever through merkle trees? (was: nodetool repair never finishes. Loops forever through merkle trees) nodetool repair never finishes. Loops forever through merkle trees? --- Key: CASSANDRA-2758 URL: https://issues.apache.org/jira/browse/CASSANDRA-2758 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen I am not sure all steps here is needed, but as part of testing something else, I set up node1: initial_token: 1 node2: initial_token: 5 Then: {noformat} create keyspace myks with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' with strategy_options = [{ replication_factor:2 }]; use myks; create column family test with comparator = AsciiType and column_metadata=[ {column_name: 'up_', validation_class: LongType, index_type: 0}, {column_name: 'del_', validation_class: LongType, index_type: 0} ] and keys_cached = 10 and rows_cached = 1 and min_compaction_threshold = 2; quit; {noformat} Doing nodetool repair after this gets both nodes busy looping forever. A quick look at one node in eclipse makes me guess its having fun spinning through merkle trees, but I have to admit I have not look at it for a long time. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2752) repair fails with java.io.EOFException
[ https://issues.apache.org/jira/browse/CASSANDRA-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13047211#comment-13047211 ] Terje Marthinussen commented on CASSANDRA-2752: --- Chris: Same as the compactionexecutor or the infinite loop? (made CASSANDRA-2758 for that just now). Seems like the sstables here has been truncated for some reason. Rowindexer iterates through a bunch of rows just fine. Then it reaches the problem row. For this row, it can get the key, it jumps pass the bloomfilter etc and when it is about to read the column count, it fails and is trying to read at an offset which equals the length of the file.. So far, all the stacktraces I have seen are all are on the column count read. A wild guess may be that it has failed to write the actual content (columns) of the last row that was stream? Unfortunately it does not to happen all the time, but it does only happen on the CF with secondary indexes. repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-132,5,main] java.lang.RuntimeException: java.util.concurrent.ExecutionException:
[jira] [Commented] (CASSANDRA-2752) repair fails with java.io.EOFException
[ https://issues.apache.org/jira/browse/CASSANDRA-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13047266#comment-13047266 ] Terje Marthinussen commented on CASSANDRA-2752: --- Also struck me on the way home from work today that the CF with secondary indexes also happen to be the only CF in this system which, I think, on a regular basis may actually update all columns for a key. That is, sstables will on a regular basis have keys where no columns is valid anymore. Not sure if that could for instance trigger something odd in the streaming? repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-132,5,main] java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.EOFException at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:152) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at
[jira] [Created] (CASSANDRA-2752) repair fails with java.io.EOFException
repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-132,5,main] java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.EOFException at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:152) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) Caused by: java.util.concurrent.ExecutionException: java.io.EOFException at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:136) ... 3 more Caused by: java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at
[jira] [Commented] (CASSANDRA-2752) repair fails with java.io.EOFException
[ https://issues.apache.org/jira/browse/CASSANDRA-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13046509#comment-13046509 ] Terje Marthinussen commented on CASSANDRA-2752: --- Need to test a bit more, but quite likely this is related to repair on CFs with secondary indexes. repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-132,5,main] java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.EOFException at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:152) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) Caused by: java.util.concurrent.ExecutionException: java.io.EOFException at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at
[jira] [Commented] (CASSANDRA-2752) repair fails with java.io.EOFException
[ https://issues.apache.org/jira/browse/CASSANDRA-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13046586#comment-13046586 ] Terje Marthinussen commented on CASSANDRA-2752: --- Seems confirmed, only able to reproduce this on a CF with secondary indexes. repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-132,5,main] java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.EOFException at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:152) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:155) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:93) Caused by: java.util.concurrent.ExecutionException: java.io.EOFException at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222) at java.util.concurrent.FutureTask.get(FutureTask.java:83) at
[jira] [Commented] (CASSANDRA-2752) repair fails with java.io.EOFException
[ https://issues.apache.org/jira/browse/CASSANDRA-2752?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13046628#comment-13046628 ] Terje Marthinussen commented on CASSANDRA-2752: --- Fresh 0.8 data. Just as a side test since the compactionexecutor is involved, we issued a full compaction of that CF and it completed without any error so the source SSTables seems good. I was trying to reproduce this locally on my desktop before leaving office to get it in a debugger. Quickly generated 10k random inserts into a CF with secondary index, but then I experienced that repair got stuck eating 100% on both nodes instead... I did not have time to figure out if it was due to some config issue or related to the same issue though. repair fails with java.io.EOFException -- Key: CASSANDRA-2752 URL: https://issues.apache.org/jira/browse/CASSANDRA-2752 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8.0 Reporter: Terje Marthinussen Issuing repair on node 1 (1.10.42.81) in a cluster quickly fails with INFO [AntiEntropyStage:1] 2011-06-09 19:02:47,999 AntiEntropyService.java (line 234) Queueing comparison #Differencer #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306fb46e, /1 .10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 468) Endpoints somewhere/1.10.42.81 and /1.10.42.82 have 2 range(s) out of sync for (JP,XXX) on (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,026 AntiEntropyService.java (line 485) Performing streaming repair of 2 ranges for #TreeRequest manual-repair-0c17c5f9-583f-4a31-a6d4-a9e7306 fb46e, /1.10.42.82, (JP,XXX), (Token(bytes[6e]),Token(bytes[313039])] INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,030 StreamOut.java (line 173) Stream context metadata [/data/cassandra/node0/data/JP/XXX-g-3-Data.db sections=1 progress=0/36592 - 0%], 1 sstables. INFO [AntiEntropyStage:1] 2011-06-09 19:02:48,031 StreamOutSession.java (line 174) Streaming to /1.10.42.82 ERROR [CompactionExecutor:9] 2011-06-09 19:02:48,970 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:9,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) On .82 ERROR [CompactionExecutor:12] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[CompactionExecutor:12,1,main] java.io.EOFException at java.io.RandomAccessFile.readInt(RandomAccessFile.java:725) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.doIndexing(SSTableWriter.java:457) at org.apache.cassandra.io.sstable.SSTableWriter$RowIndexer.index(SSTableWriter.java:364) at org.apache.cassandra.io.sstable.SSTableWriter$Builder.build(SSTableWriter.java:315) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1099) at org.apache.cassandra.db.CompactionManager$9.call(CompactionManager.java:1090) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [Thread-132] 2011-06-09 19:02:48,051 AbstractCassandraDaemon.java (line 113) Fatal exception in thread Thread[Thread-132,5,main] java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.io.EOFException at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:152) at
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13033477#comment-13033477 ] Terje Marthinussen commented on CASSANDRA-47: - And yes, that traversal two times of the row to calculate serialized size before writing the row is reasonably expensive with compression. Any good reason we absolutely have to do this in two passes instead of one? Or just the way it is for historical purposes? Yes, we would need to write indexes etc. after the row or in a separate file if we don't traverse it twice, but this would seems a relatively small change on first sight? SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Priority: Minor Labels: compression Fix For: 1.0 We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13033467#comment-13033467 ] Terje Marthinussen commented on CASSANDRA-47: - Just curious if any active work is done or planned near future on compressing larger data blocks or is it all suspended waiting for a new sstable design? Having played with compression of just supercolumns for a while, I am a bit tempted to test out compression of larger blocks of data. At least row level compression seems reasonably easy to do. Some experiences so far which may be usefull: - Compression on sstables may actually be helpfull on memory pressure, but with my current implementation, non-batched update throughput may drop 50%.I am not 100% sure why actually. - Flushing of (compressed) memtables and compactions are clear potential bottlenecks The obvious trouble makers here is the fact that you ceep For really high pressure work, I think it would be usefull to only compress tables once they pass a certain size to reduce the amount of recompression occuring on memtable flushes and when compacting small sstables (which is generally not a big disk problem anyway) This is a bit awkward when doing things like I do in the super columns as I believe the supercolumn does not know anything about the data it is part of (except for recently, the deserializer has that info through inner. It would anyway probably be cleaner to let the datastructures/methods using the SC decide when to compress and noth - Working on a SC level, there seems to be some 10-15% extra compression on this specific data if column names that are highly repetetive in SC's can be extracted into some meta data structure so you only store references to these in the column names. That is, the final data is goes from about 40% compression to 50% compression. I don't think the effect of this will be equally big with larger blocks, but I suspect there should be some effect. - total size reduction of the sstables I have in this project is currently in the 60-65% range. It is mainly beneficial for those that have supercolumns with at least a handfull of columns (400-600 bytes of serialized column data per sc at least) - Reducing the meta data on columns by building a dictionary of timestamps as well as variable length name/value length data (instead of fixed short/int) cuts down another 10% in my test (I have just done a very quick simulation of this by a very quick 10 minute hack on the serializer) - We may want to look at how we can reuse whole compressed rows on compactions if for instance the other tables you compact with do not have the same data - We may want a new cache on the uncompressed disk chunks. In my case, I preserve the compressed part of the supercolumn and In my supercolumn compression case, I have a cache for the compressed data so I can write that back without recompression if not modified. This also makes calls to get the serialized size cheaper (don't need to compress both to find serialized size and to actually serialize) If people are interested in adding any of the above to current cassandra, I will try to get time to make some of this up to a quality where it could be used by the general public. If not, I will wait for new sstables to get a bit more ready and see if I can contribute there instead. SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Priority: Minor Labels: compression Fix For: 1.0 We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Issue Comment Edited] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13033467#comment-13033467 ] Terje Marthinussen edited comment on CASSANDRA-47 at 5/14/11 5:45 AM: -- Just curious if any active work is done or planned near future on compressing larger data blocks or is it all suspended waiting for a new sstable design? Having played with compression of just supercolumns for a while, I am a bit tempted to test out compression of larger blocks of data. At least row level compression seems reasonably easy to do. Some experiences so far which may be usefull: - Compression on sstables may actually be helpfull on memory pressure, but with my current implementation, non-batched update throughput may drop 50%.I am not 100% sure why actually. - Flushing of (compressed) memtables and compactions are clear potential bottlenecks The obvious trouble makers here is the fact that you ceep For really high pressure work, I think it would be usefull to only compress tables once they pass a certain size to reduce the amount of recompression occuring on memtable flushes and when compacting small sstables (which is generally not a big disk problem anyway) This is a bit awkward when doing things like I do in the super columns as I believe the supercolumn does not know anything about the data it is part of (except for recently, the deserializer has that info through inner. It would anyway probably be cleaner to let the datastructures/methods using the SC decide when to compress and noth - Working on a SC level, there seems to be some 10-15% extra compression on this specific data if column names that are highly repetetive in SC's can be extracted into some meta data structure so you only store references to these in the column names. That is, the final data is goes from about 40% compression to 50% compression. I don't think the effect of this will be equally big with larger blocks, but I suspect there should be some effect. - total size reduction of the sstables when using a dictionary for column names as well as timestamps and variable length lenght fields, is currently in the 60-65% range. It is however mainly beneficial for those that have supercolumns with at least a handfull of columns (400-600 bytes of serialized column data per sc at least) - Reducing the meta data on columns by building a dictionary of timestamps as well as variable length name/value length data (instead of fixed short/int) cuts down another 10% in my test (I have just done a very quick simulation of this by a very quick 10 minute hack on the serializer) - We may want to look at how we can reuse whole compressed rows on compactions if for instance the other tables you compact with do not have the same data - We may want a new cache on the uncompressed disk chunks. In my case, I preserve the compressed part of the supercolumn and In my supercolumn compression case, I have a cache for the compressed data so I can write that back without recompression if not modified. This also makes calls to get the serialized size cheaper (don't need to compress both to find serialized size and to actually serialize) If people are interested in adding any of the above to current cassandra, I will try to get time to make some of this up to a quality where it could be used by the general public. If not, I will wait for new sstables to get a bit more ready and see if I can contribute there instead. was (Author: terjem): Just curious if any active work is done or planned near future on compressing larger data blocks or is it all suspended waiting for a new sstable design? Having played with compression of just supercolumns for a while, I am a bit tempted to test out compression of larger blocks of data. At least row level compression seems reasonably easy to do. Some experiences so far which may be usefull: - Compression on sstables may actually be helpfull on memory pressure, but with my current implementation, non-batched update throughput may drop 50%.I am not 100% sure why actually. - Flushing of (compressed) memtables and compactions are clear potential bottlenecks The obvious trouble makers here is the fact that you ceep For really high pressure work, I think it would be usefull to only compress tables once they pass a certain size to reduce the amount of recompression occuring on memtable flushes and when compacting small sstables (which is generally not a big disk problem anyway) This is a bit awkward when doing things like I do in the super columns as I believe the supercolumn does not know anything about the data it is part of (except for recently, the deserializer has that info through inner. It would anyway probably be cleaner to let the datastructures/methods using the SC decide when to compress and noth - Working on a SC
[jira] [Created] (CASSANDRA-2626) stack overflow while compacting
stack overflow while compacting --- Key: CASSANDRA-2626 URL: https://issues.apache.org/jira/browse/CASSANDRA-2626 Project: Cassandra Issue Type: Bug Components: Documentation website Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen This is a trunk build from May 3. After adding CASSANDRA-2401, I have gotten the following on several nodes. I am not 100% sure right now if it is related to 2401 but it may seem likely. Unfortunately, as often is the case with stack overflows, I don't see the start of the stack ERROR [CompactionExecutor:17] 2011-05-09 07:56:32,479 AbstractCassandraDaemon.java (line 112) Fatal exception in thread Thread[CompactionExecutor:17,1,main] java.lang.StackOverflowError at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) at java.util.Collections$UnmodifiableCollection.size(Collections.java:998) -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2604) EOFException on commitlogs
[ https://issues.apache.org/jira/browse/CASSANDRA-2604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13029874#comment-13029874 ] Terje Marthinussen commented on CASSANDRA-2604: --- Not able to reproduce the problem with the patch in this ticket. Currently using CASSANDRA-2401 and so far so good. I think you can just close this ticket for now and I will reopen if I see the problem again. Thanks! EOFException on commitlogs -- Key: CASSANDRA-2604 URL: https://issues.apache.org/jira/browse/CASSANDRA-2604 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen Assignee: Sylvain Lebresne Fix For: 0.8.0 Attachments: 0001-avoid-modifying-original-mutation-during-apply.patch I have seen this occasionally since we started testing 0.8. It happens when reading commitlogs on startups. However, I have seen it a lot less on 0.8 beta2 (although this is from beta 2) ERROR [main] 2011-05-04 18:02:38,134 AbstractCassandraDaemon.java (line 330) Exception encountered during startup. java.io.EOFException at java.io.DataInputStream.readByte(DataInputStream.java:250) at org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:357) at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:368) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:252) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:43) at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:136) at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:126) at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:368) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:256) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:157) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:173) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:313) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:80) Note that the line numbers on columnserializer may be off due to some local changes, but those changes are in code not executed in this case and I am 100% sure they do not trigger this problem. I looked on this in the debugger in eclipse on a trunk from 0.8 2 weeks ago, and the interesting thing I saw was that according to the debugger, the offset of the inputstream to the deserializer was already at the end (very last byte) of the underlying bytebuffer but according to the stack, it was trying to read the length of the column name (first read done in the deserialized). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2604) EOFException on commitlogs
[ https://issues.apache.org/jira/browse/CASSANDRA-2604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13029345#comment-13029345 ] Terje Marthinussen commented on CASSANDRA-2604: --- Secondary indexes - Check (the error at least in the case I studied now is on a CF with secondary index) Not in batch mode - Check I am not 100% sure I understand in what cases a column may be removed. There should be no deletes here and I think no updates to columns that already exists (I will have to check that...), but I am messing a bit around with the system while a couple of feeders are continuously running and they will refeed if there are errors. It is very likely that there are are quite a few retries going on at times which I guess could trigger what you are referring too? I will apply the patch at once and test. Thanks! EOFException on commitlogs -- Key: CASSANDRA-2604 URL: https://issues.apache.org/jira/browse/CASSANDRA-2604 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen Assignee: Sylvain Lebresne Fix For: 0.8.0 Attachments: 0001-avoid-modifying-original-mutation-during-apply.patch I have seen this occasionally since we started testing 0.8. It happens when reading commitlogs on startups. However, I have seen it a lot less on 0.8 beta2 (although this is from beta 2) ERROR [main] 2011-05-04 18:02:38,134 AbstractCassandraDaemon.java (line 330) Exception encountered during startup. java.io.EOFException at java.io.DataInputStream.readByte(DataInputStream.java:250) at org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:357) at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:368) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:252) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:43) at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:136) at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:126) at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:368) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:256) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:157) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:173) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:313) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:80) Note that the line numbers on columnserializer may be off due to some local changes, but those changes are in code not executed in this case and I am 100% sure they do not trigger this problem. I looked on this in the debugger in eclipse on a trunk from 0.8 2 weeks ago, and the interesting thing I saw was that according to the debugger, the offset of the inputstream to the deserializer was already at the end (very last byte) of the underlying bytebuffer but according to the stack, it was trying to read the length of the column name (first read done in the deserialized). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2604) EOFException on commitlogs
[ https://issues.apache.org/jira/browse/CASSANDRA-2604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13029495#comment-13029495 ] Terje Marthinussen commented on CASSANDRA-2604: --- After stressing the system a bit with a bunch of restarts, I have not manage to reproduce the problem again, so looks good so far. I will leave it feeding overnight and try one more time in the morning and then I will try to reproduce with 2401 instead. On a side notice... I sometimes see quite a few commitlog segments. Right now, one node had 110 of them and it took 20 minutes to read all the logs. This seems a bit excessive? EOFException on commitlogs -- Key: CASSANDRA-2604 URL: https://issues.apache.org/jira/browse/CASSANDRA-2604 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen Assignee: Sylvain Lebresne Fix For: 0.8.0 Attachments: 0001-avoid-modifying-original-mutation-during-apply.patch I have seen this occasionally since we started testing 0.8. It happens when reading commitlogs on startups. However, I have seen it a lot less on 0.8 beta2 (although this is from beta 2) ERROR [main] 2011-05-04 18:02:38,134 AbstractCassandraDaemon.java (line 330) Exception encountered during startup. java.io.EOFException at java.io.DataInputStream.readByte(DataInputStream.java:250) at org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:357) at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:368) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:252) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:43) at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:136) at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:126) at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:368) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:256) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:157) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:173) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:313) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:80) Note that the line numbers on columnserializer may be off due to some local changes, but those changes are in code not executed in this case and I am 100% sure they do not trigger this problem. I looked on this in the debugger in eclipse on a trunk from 0.8 2 weeks ago, and the interesting thing I saw was that according to the debugger, the offset of the inputstream to the deserializer was already at the end (very last byte) of the underlying bytebuffer but according to the stack, it was trying to read the length of the column name (first read done in the deserialized). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2558) Add concurrent_compactions configuration
[ https://issues.apache.org/jira/browse/CASSANDRA-2558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13028692#comment-13028692 ] Terje Marthinussen commented on CASSANDRA-2558: --- You may be right. The tmp file could be memtable under flush. I will watch that a bit more. A few additional concerns to the entire concurrent compaction (although they probably belong in a different ticket): If I read the code right, I think each thread looks for empty disk space by itself without global coordination? If so, a more nasty side effect of concurrent compaction is that the disk space check fails. However, the threads should probably also continuously monitor available disk space and start shutting themselves down in a controlled and coordinated fashion so all the compaction threads don't write the disk full, but rather let some of them finish Add concurrent_compactions configuration -- Key: CASSANDRA-2558 URL: https://issues.apache.org/jira/browse/CASSANDRA-2558 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 0.8 beta 1 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Trivial Fix For: 0.8.0 beta 2 Attachments: 0001-Make-compaction-thread-number-configurable-v2.patch, 0001-Make-compaction-thread-number-configurable.patch Original Estimate: 2h Remaining Estimate: 2h We should expose a way to configure the max number of thread to use when multi_threaded compaction is turned on. So far, it uses nb_of_processors thread, which if you have many cores may be unreasonably high (as far as random IO is concerned and thus independently of compaction throttling)... at least unless you have SSD. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-2604) EOFException on commitlogs
EOFException on commitlogs -- Key: CASSANDRA-2604 URL: https://issues.apache.org/jira/browse/CASSANDRA-2604 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen I have seen this occasionally since we started testing 0.8. It happens when reading commitlogs on startups. However, I have seen it a lot less on 0.8 beta2 (although this is from beta 2) ERROR [main] 2011-05-04 18:02:38,134 AbstractCassandraDaemon.java (line 330) Exception encountered during startup. java.io.EOFException at java.io.DataInputStream.readByte(DataInputStream.java:250) at org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:357) at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:368) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:252) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:43) at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:136) at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:126) at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:368) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:256) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:157) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:173) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:313) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:80) Note that the line numbers on columnserializer may be off due to some local changes, but those changes are in code not executed in this case and I am 100% sure they do not trigger this problem. I looked on this in the debugger in eclipse on a trunk from 0.8 2 weeks ago, and the interesting thing I saw was that according to the debugger, the offset of the inputstream to the deserializer was already at the end (very last byte) of the underlying bytebuffer but according to the stack, it was trying to read the length of the column name (first read done in the deserialized). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2604) EOFException on commitlogs
[ https://issues.apache.org/jira/browse/CASSANDRA-2604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13029033#comment-13029033 ] Terje Marthinussen commented on CASSANDRA-2604: --- Time for some sleep, but I stepped through this code with a bad commit table (not the same which caused the above error though, but produces same stack). public void deserializeColumns(DataInput dis, ColumnFamily cf, boolean intern, boolean fromRemote) throws IOException { int size = dis.readInt(); ColumnFamilyStore interner = intern ? Table.open(CFMetaData.getCF(cf.id()).left).getColumnFamilyStore(cf.id()) : null; for (int i = 0; i size; ++i) { IColumn column = cf.getColumnSerializer().deserialize(dis, interner, fromRemote, (int) (System.currentTimeMillis() / 1000)); cf.addColumn(column); } } The size of the buffer underlying dis is 153 byte. size in the above code is 4. With i == 2 finished, dis has just reached position 153 (perfectly) in the underlying buffer, although I cannot guarantee that the value is completely read. Obviously, when trying to read at i==3, it crashes as it is already at the end of the buffer. I would expect the entry which it is trying to decode in this case to have 4, not 3 columns. I do see some of my own code is actually involved here, but unless the commitlog does its own serialization fully or partly and uses the standard deserializer my code is unlikely to cause any problems like this. EOFException on commitlogs -- Key: CASSANDRA-2604 URL: https://issues.apache.org/jira/browse/CASSANDRA-2604 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen Assignee: Sylvain Lebresne I have seen this occasionally since we started testing 0.8. It happens when reading commitlogs on startups. However, I have seen it a lot less on 0.8 beta2 (although this is from beta 2) ERROR [main] 2011-05-04 18:02:38,134 AbstractCassandraDaemon.java (line 330) Exception encountered during startup. java.io.EOFException at java.io.DataInputStream.readByte(DataInputStream.java:250) at org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:357) at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:368) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:252) at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:43) at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:136) at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:126) at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:368) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:256) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:157) at org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:173) at org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:313) at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:80) Note that the line numbers on columnserializer may be off due to some local changes, but those changes are in code not executed in this case and I am 100% sure they do not trigger this problem. I looked on this in the debugger in eclipse on a trunk from 0.8 2 weeks ago, and the interesting thing I saw was that according to the debugger, the offset of the inputstream to the deserializer was already at the end (very last byte) of the underlying bytebuffer but according to the stack, it was trying to read the length of the column name (first read done in the deserialized). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2540) Data reads by default
[ https://issues.apache.org/jira/browse/CASSANDRA-2540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13026880#comment-13026880 ] Terje Marthinussen commented on CASSANDRA-2540: --- Would it make sense to make dynamic snitch react to digest but not data error directly to improve recovery time and why do we need to wait for the 100ms to make dynamic snitch work? Couldn't we add a load balancer like function that would detect if a node has a significant number of outstanding requests on node 1 vs. 2 and 3, then send to node 2 instead? Overall, I am not asking for a round robin load balancer though... (not good for caching) The digest function may also in some cases maybe be made dynamic based on the size of the data being read. That is, just send the data for small data sizes and use digests for large responses? No, I don't know all the details on how this part of the code work, so my suggestions may be totally wrong :) I do wonder however if we get enough of these timeouts to actually be a problem. If there is a couple of delays in latencies for a few seconds 2-3 times a week/month, no problem. However, if we have so many of these errors that people see them many times a day, it would seem like this is a performance problem somewhere in cassandra which should be fixed rather than applying some patchwork which hide it... Data reads by default - Key: CASSANDRA-2540 URL: https://issues.apache.org/jira/browse/CASSANDRA-2540 Project: Cassandra Issue Type: Wish Reporter: Stu Hood Priority: Minor The intention of digest vs data reads is to save bandwidth in the read path at the cost of latency, but I expect that this has been a premature optimization. * Data requested by a read will often be within an order of magnitude of the digest size, and a failed digest means extra roundtrips, more bandwidth * The [digest reads but not your data read|https://issues.apache.org/jira/browse/CASSANDRA-2282?focusedCommentId=13004656page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13004656] problem means failing QUORUM reads because a single node is unavailable, and would require eagerly re-requesting at some fraction of your timeout * Saving bandwidth in cross datacenter usecases comes at huge cost to latency, but since both constraints change proportionally (enough), the tradeoff is not clear Some options: # Add an option to use digest reads # Remove digest reads entirely (and/or punt and make them a runtime optimization based on data size in the future) # Continue to use digest reads, but send them to {{N - R}} nodes for (somewhat) more predicatable behavior with QUORUM \\ The outcome of data-reads-by-default should be significantly improved latency, with a moderate increase in bandwidth usage for large reads. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-2574) java.lang.NoClassDefFoundError: org/apache/cassandra/db/commitlog/PeriodicCommitLogExecutorService$4
java.lang.NoClassDefFoundError: org/apache/cassandra/db/commitlog/PeriodicCommitLogExecutorService$4 - Key: CASSANDRA-2574 URL: https://issues.apache.org/jira/browse/CASSANDRA-2574 Project: Cassandra Issue Type: Bug Components: Core Reporter: Terje Marthinussen Priority: Minor We see this in the log occasionally, but the jar file does have the right class file. Seems to occur when cassandra is stopped and I suppose it may be some race between the vm class loader and the fact that it has got a kill signal and is shutting down? ERROR [Thread-2] 2011-04-26 19:32:03,841 AbstractCassandraDaemon.java (line 112) Fatal exception in thread Thread[Thread-2,5,main] java.lang.NoClassDefFoundError: org/apache/cassandra/db/commitlog/PeriodicCommitLogExecutorService$4 at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService.shutdown(PeriodicCommitLogExecutorService.java:121) at org.apache.cassandra.db.commitlog.CommitLog.shutdownBlocking(CommitLog.java:503) at org.apache.cassandra.service.StorageService$2.runMayThrow(StorageService.java:402) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) at java.lang.Thread.run(Thread.java:662) Caused by: java.lang.ClassNotFoundException: org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$4 at java.net.URLClassLoader$1.run(URLClassLoader.java:202) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:190) at java.lang.ClassLoader.loadClass(ClassLoader.java:307) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:301) at java.lang.ClassLoader.loadClass(ClassLoader.java:248) ... 5 more Not sure if it is a problem or not and does not happen all the time. In this case this is also followed by at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:551) at org.apache.cassandra.thrift.CassandraServer.internal_batch_mutate(CassandraServer.java:503) at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:511) at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3454) at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2889) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:187) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [pool-2-thread-91] 2011-04-26 19:32:04,267 Cassandra.java (line 3462) Internal error processing batch_mutate java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:73) at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767) at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658) at org.apache.cassandra.service.StorageProxy.insertLocal(StorageProxy.java:350) at org.apache.cassandra.service.StorageProxy.sendToHintedEndpoints(StorageProxy.java:230) at org.apache.cassandra.service.StorageProxy.access$000(StorageProxy.java:62) at org.apache.cassandra.service.StorageProxy$1.apply(StorageProxy.java:99) at org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:199) at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:154) at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:551) at org.apache.cassandra.thrift.CassandraServer.internal_batch_mutate(CassandraServer.java:503) at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:511) at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.process(Cassandra.java:3454) at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2889) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:187) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) ERROR [pool-2-thread-178] 2011-04-26 19:32:04,296 Cassandra.java (line 3462) Internal error processing batch_mutate java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down at
[jira] [Created] (CASSANDRA-2550) nodetool setcompactionthroughput requiring wrong number of arguments?
nodetool setcompactionthroughput requiring wrong number of arguments? - Key: CASSANDRA-2550 URL: https://issues.apache.org/jira/browse/CASSANDRA-2550 Project: Cassandra Issue Type: Bug Components: Tools Affects Versions: 0.8 beta 1 Reporter: Terje Marthinussen Priority: Minor --- case SETCOMPACTIONTHROUGHPUT : if (arguments.length != 2) { badUse(Missing value argument.); } probe.setCompactionThroughput(Integer.valueOf(arguments[1])); break; --- I would think arguments.length should be just 1? -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-47) SSTable compression
[ https://issues.apache.org/jira/browse/CASSANDRA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13014531#comment-13014531 ] Terje Marthinussen commented on CASSANDRA-47: - This is not so interesting for a proper solution maybe, but adding just for the reference. I needed to get space for more data, so I recently just crashed into a quick compression hack for supercolumns. I was considering to compress the index blocks as Jonathan suggested, but I could not make up my mind on how safe that would be in terms of other code accessing the data and had a bit short time, so I looked for something more isolated. Final decision was to simply compress the serialized columns in a supercolumn (+ add a bit caching to avoid recompressing all the time when serialized size is requested) The data I have, has supercolumns with typically 50-60 subcolumns. Mostly small strings or numbers. In total, the subcolumns makes up 600-1200 bytes of data when serialized. Usually a fair bit of supercolumns per row. My test data was 447 keys. I tested with the ning lzf jars and the default java.util.zip. This is not necessarily a good test, but I think json2sstable is somewhat useful to measure relative impact between implementations although not useful to determine real performance in any way. In addition, I made a simple dictionary of column names (only applied to supercolumns) as the column names was not very well compressed when looking at just a single supercolumn at a time. The result of both the digest and compression: Standard cassandra. json2sstable: real0m55.148s user1m50.023s sys 0m2.856s sstable: 190MB ning.com: real1m8.315s user2m18.361s sys 0m4.600s sstable: 108MB java.util.zip real1m35.899s user2m49.691s sys 0m2.940s sstable: 90mb As a reference, the whole sstable files compresses as follows: ning.com (command line) real0m1.803s user0m1.536s sys 0m0.396s sstable: 80MB gzip (command line) real0m6.175s user0m6.076s sys 0m0.084s sstable: 48MB I doubt this implementation has much for inclusion in a release. Just added the numbers for the reference. Of course, if requested, I could see if I could make the patch available somewhere. SSTable compression --- Key: CASSANDRA-47 URL: https://issues.apache.org/jira/browse/CASSANDRA-47 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Priority: Minor Fix For: 0.8 We should be able to do SSTable compression which would trade CPU for I/O (almost always a good trade). -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (CASSANDRA-2282) ReadCallback AssertionError: resolver.getMessageCount() = endpoints.size()
[ https://issues.apache.org/jira/browse/CASSANDRA-2282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13003946#comment-13003946 ] Terje Marthinussen commented on CASSANDRA-2282: --- From ReadCallback.java this.endpoints = repair || resolver instanceof RowRepairResolver ? endpoints : endpoints.subList(0, Math.min(endpoints.size(), blockfor)); // min so as to not throw exception until assureSufficient is called This will cut the list of endpoints to whatever is the consistency requirement (in the case where repair is false, which for instance happens all the time for a rangescan). Later: assert resolver.getMessageCount() = endpoints.size() which will cause an assert if all nodes answers on a range request (or if you have a random readrepair). I am actually not 100% sure what the fix is right now as I have not had time to scan the rest of the code and need to leave office for today, but that is the problem anyway :) To be honest, the logic here may look a bit broken. The assert happening here may good either. The error condition is not returned to the client, so it will hang around waiting for a timeout to occur. Maybe the code should throw some exception before the assert? Something like assert resolver.getMessageCount() = endpoints.size() : Got + resolver.getMessageCount() + replies but only know + endpoints.size() + endpoints; may also be good? ReadCallback AssertionError: resolver.getMessageCount() = endpoints.size() --- Key: CASSANDRA-2282 URL: https://issues.apache.org/jira/browse/CASSANDRA-2282 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.3 Reporter: Tyler Hobbs In a three node cluster with RF=2, when trying to page through all rows with get_range_slices() at CL.ONE, I get timeouts on the client side. Looking at the Cassandra logs, all of the nodes show the following AssertionError repeatedly: {noformat} ERROR [RequestResponseStage:2] 2011-03-07 19:10:27,527 DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecutor java.lang.AssertionError at org.apache.cassandra.service.ReadCallback.response(ReadCallback.java:127) at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:49) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) ERROR [RequestResponseStage:2] 2011-03-07 19:10:27,529 AbstractCassandraDaemon.java (line 114) Fatal exception in thread Thread[RequestResponseStage:2,5,main] java.lang.AssertionError at org.apache.cassandra.service.ReadCallback.response(ReadCallback.java:127) at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:49) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} The nodes are all running 0.7.3. The cluster was at size 3 before any data was inserted, and everything else appears perfectly healthy. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (CASSANDRA-2282) ReadCallback AssertionError: resolver.getMessageCount() = endpoints.size()
[ https://issues.apache.org/jira/browse/CASSANDRA-2282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13003948#comment-13003948 ] Terje Marthinussen commented on CASSANDRA-2282: --- Just to make this very clear. The list of endpoints is cut to the consistency requirement so if: - The request requires gossip with more than one node - all nodes answers - you do not use consistencylevel ALL, then the assert will trigger. ReadCallback AssertionError: resolver.getMessageCount() = endpoints.size() --- Key: CASSANDRA-2282 URL: https://issues.apache.org/jira/browse/CASSANDRA-2282 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 0.7.3 Reporter: Tyler Hobbs In a three node cluster with RF=2, when trying to page through all rows with get_range_slices() at CL.ONE, I get timeouts on the client side. Looking at the Cassandra logs, all of the nodes show the following AssertionError repeatedly: {noformat} ERROR [RequestResponseStage:2] 2011-03-07 19:10:27,527 DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecutor java.lang.AssertionError at org.apache.cassandra.service.ReadCallback.response(ReadCallback.java:127) at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:49) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) ERROR [RequestResponseStage:2] 2011-03-07 19:10:27,529 AbstractCassandraDaemon.java (line 114) Fatal exception in thread Thread[RequestResponseStage:2,5,main] java.lang.AssertionError at org.apache.cassandra.service.ReadCallback.response(ReadCallback.java:127) at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:49) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:636) {noformat} The nodes are all running 0.7.3. The cluster was at size 3 before any data was inserted, and everything else appears perfectly healthy. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] Commented: (CASSANDRA-1735) Using MessagePack for reducing data size
[ https://issues.apache.org/jira/browse/CASSANDRA-1735?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12932756#action_12932756 ] Terje Marthinussen commented on CASSANDRA-1735: --- I am very curious how the serialization in messagepack could compete with the serialization used on the data side for cassandra (SSTables) and how we could benefit from having the same serialization in both those places. Anyone has any thoughts? Using MessagePack for reducing data size Key: CASSANDRA-1735 URL: https://issues.apache.org/jira/browse/CASSANDRA-1735 Project: Cassandra Issue Type: New Feature Components: API Affects Versions: 0.7 beta 3 Environment: Fedora11, JDK1.6.0_20 Reporter: Muga Nishizawa Attachments: 0001-implement-a-Cassandra-RPC-part-with-MessagePack.patch, dependency_libs.zip For improving Cassandra performance, I implemented a Cassandra RPC part with MessagePack. The implementation details are attached as a patch. The patch works on Cassandra 0.7.0-beta3. Please check it. MessagePack is one of object serialization libraries for cross-languages like Thrift and Protocol Buffers but it is much faster, small, and easy to implement. MessagePack allows reducing serialization cost and data size in network and disk. MessagePack websites are * website: http://msgpack.org/ This website compares MessagePack, Thrift and JSON. * desing details: http://redmine.msgpack.org/projects/msgpack/wiki/FormatDesign * source code: https://github.com/msgpack/msgpack/ Performance of the data serialization library is one of the most important issues for developing a distributed database in Java. If the performance is bad, it significantly reduces the overall database performance. Java's GC also runs many times. Cassandra has this problem as well. For reducing data size in network between a client and Cassandra, I prototyped the implementation of a Cassandra RPC part with MessagePack and MessagePack-RPC. The implementation is very simple. MessagePack-RPC can reuse the existing Thrift based CassandraServer (org.apache.cassandra.thrift.CassandraServer) while adapting MessagePack's communication protocol and data serialization. Major features of MessagePack-RPC are * Asynchronous RPC * Parallel Pipelining * Connection pooling * Delayed return * Event-driven I/O * more details: http://redmine.msgpack.org/projects/msgpack/wiki/RPCDesign * source code: https://github.com/msgpack/msgpack-rpc/ The attached patch includes a ring cache program for MessagePack and its test program. You can check the behavior of the Cassandra RPC with MessagePack. Thanks in advance, -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.