[jira] [Comment Edited] (CASSANDRA-11206) Support large partitions on the 3.0 sstable format
[ https://issues.apache.org/jira/browse/CASSANDRA-11206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15328785#comment-15328785 ] Michael Kjellman edited comment on CASSANDRA-11206 at 6/14/16 2:14 AM: --- going thru the changes and have some questions :) # RowIndexEntry$serializedSize used to return the size of the index for the entire row. As the size of the IndexInfo elements are variable length I'm having trouble understanding how the new/current implementation does this: {code} private static int serializedSize(DeletionTime deletionTime, long headerLength, int columnIndexCount) { return TypeSizes.sizeofUnsignedVInt(headerLength) + (int) DeletionTime.serializer.serializedSize(deletionTime) + TypeSizes.sizeofUnsignedVInt(columnIndexCount); } {code} # In the class level Javadoc for IndexInfo there is a lot of comment about serialization format changes and even a comment "Serialization format changed in 3.0" yet I don't see any corresponding changes in BigFormat$BigVersion # I see a class named **Pre_C_11206_RowIndexEntry** in RowIndexEntryTest which has a lot of the logic that used to be in RowIndexEntry. I don't see the logic outside of the test classes though. was (Author: mkjellman): going thru the changes and have some questions :) # RowIndexEntry$serializedSize used to return the size of the index for the entire row. As the size of the IndexInfo elements are variable length I'm having trouble understanding how the new/current implementation does this: {code} private static int serializedSize(DeletionTime deletionTime, long headerLength, int columnIndexCount) { return TypeSizes.sizeofUnsignedVInt(headerLength) + (int) DeletionTime.serializer.serializedSize(deletionTime) + TypeSizes.sizeofUnsignedVInt(columnIndexCount); } {code} # In the class level Javadoc for IndexInfo there is a lot of comment about serialization format changes and even a comment "Serialization format changed in 3.0" yet I don't see any corresponding changes in BigFormat$BigVersion > Support large partitions on the 3.0 sstable format > -- > > Key: CASSANDRA-11206 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11206 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths >Reporter: Jonathan Ellis >Assignee: Robert Stupp > Labels: docs-impacting > Fix For: 3.6 > > Attachments: 11206-gc.png, trunk-gc.png > > > Cassandra saves a sample of IndexInfo objects that store the offset within > each partition of every 64KB (by default) range of rows. To find a row, we > binary search this sample, then scan the partition of the appropriate range. > The problem is that this scales poorly as partitions grow: on a cache miss, > we deserialize the entire set of IndexInfo, which both creates a lot of GC > overhead (as noted in CASSANDRA-9754) but is also non-negligible i/o activity > (relative to reading a single 64KB row range) as partitions get truly large. > We introduced an "offset map" in CASSANDRA-10314 that allows us to perform > the IndexInfo bsearch while only deserializing IndexInfo that we need to > compare against, i.e. log(N) deserializations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-11206) Support large partitions on the 3.0 sstable format
[ https://issues.apache.org/jira/browse/CASSANDRA-11206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15203258#comment-15203258 ] Robert Stupp edited comment on CASSANDRA-11206 at 3/20/16 12:02 PM: I just finished with most of the coding for this ticket - i.e. "shallow" RowIndexEntry without {{IndexInfo}} - and ran a poor-man's comparison of current trunk against 11206 using different partition sizes covering writes, a major compaction and reads. The results are really promising especially with big and huge partitions (tested up to 8G partitions). Reads against big partitions really benefit from 11206. For example, with 11206 it takes a couple of seconds for 5000 random reads against 8G partitions vs. many minutes (not a typo) on current trunk). At the same time, the heap is quite full and causes a lot of GC pressure. Compactions also benefit from 11206 GC-wise - but not CPU- or I/O-wise since it's still the same amount of work to be done. 11206 "just" reduces GC pressure. Flushes also benefit, since it can "forget" IndexInfo objects sooner. This ticket will *not* raise the limit on cells. [~doanduyhai], you're right. Having the ability to handle big partitions has a direct influence to data modeling. I'd not say "you are not longer limited by the size of your partitions". This ticket _raises_ the current limitation WRT GC pressure and read performance. In theory the limit went away, but as you say, compaction gets even more important and other operational tasks like replacing nodes or changing topology need to be considered. My next steps are: * fix some unit tests that no longer work as they relied on the old implementation (expected to have IndexInfo on heap) * cleanup the code * run some tests on cstar I only ran a poor-man's comparison - on my laptop with small-ish 3G heap with default unit test settings. That's why I did not note exact numbers. But I'd like to show the GC pressure of the same test ran against trunk (took 3 hours) and 11206 (took 1 hour): !trunk-gc.png|GC on current trunk, width=800! !11206-gc.png|GC on 11206, width=800! was (Author: snazy): I just finished with most of the coding for this ticket - i.e. "shallow" RowIndexEntry without {{IndexInfo}} - and ran a poor-man's comparison of current trunk against 11206 using different partition sizes covering writes, a major compaction and reads. The results are really promising especially with big and huge partitions (tested up to 8G partitions). Reads against big partitions really benefit from 11206. For example, with 11206 it takes a couple of seconds for 5000 random reads against 8G partitions vs. many minutes (not a typo) on current trunk). At the same time, the heap is quite full and causes a lot of GC pressure. Compactions also benefit from 11206 GC-wise - but not CPU- or I/O-wise since it's still the same amount of work to be done. 11206 "just" reduces GC pressure. Flushes also benefit, since it can "forget" IndexInfo objects sooner. This ticket will *not* raise the limit on cells. [~doanduyhai], you're right. Having the ability to handle big partitions has a direct influence to data modeling. I'd not say "you are not longer limited by the size of your partitions". This ticket _raises_ the current limitation WRT GC pressure and read performance. In theory the limit went away, but as you say, compaction gets even more important and other operational tasks like replacing nodes or changing topology need to be considered. My next steps are: * fix some unit tests that no longer work as they relied on the old implementation (expected to have IndexInfo on heap) * cleanup the code * run some tests on cstar I only ran a poor-man's comparison - on my laptop with small-ish 3G heap with default unit test settings. That's why I did not note exact numbers. But I'd like to show the GC pressure of the same test ran against trunk and 11206: !trunk-gc.png|GC on current trunk! !11206-gc.png|GC on 11206! > Support large partitions on the 3.0 sstable format > -- > > Key: CASSANDRA-11206 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11206 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Robert Stupp > Fix For: 3.x > > Attachments: 11206-gc.png, trunk-gc.png > > > Cassandra saves a sample of IndexInfo objects that store the offset within > each partition of every 64KB (by default) range of rows. To find a row, we > binary search this sample, then scan the partition of the appropriate range. > The problem is that this scales poorly as partitions grow: on a cache miss, > we deserialize the entire set of IndexInfo, which both creates a lot of GC > overhead (as noted in CASSANDRA-9754) but is also non-negligible
[jira] [Comment Edited] (CASSANDRA-11206) Support large partitions on the 3.0 sstable format
[ https://issues.apache.org/jira/browse/CASSANDRA-11206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15203258#comment-15203258 ] Robert Stupp edited comment on CASSANDRA-11206 at 3/20/16 11:59 AM: I just finished with most of the coding for this ticket - i.e. "shallow" RowIndexEntry without {{IndexInfo}} - and ran a poor-man's comparison of current trunk against 11206 using different partition sizes covering writes, a major compaction and reads. The results are really promising especially with big and huge partitions (tested up to 8G partitions). Reads against big partitions really benefit from 11206. For example, with 11206 it takes a couple of seconds for 5000 random reads against 8G partitions vs. many minutes (not a typo) on current trunk). At the same time, the heap is quite full and causes a lot of GC pressure. Compactions also benefit from 11206 GC-wise - but not CPU- or I/O-wise since it's still the same amount of work to be done. 11206 "just" reduces GC pressure. Flushes also benefit, since it can "forget" IndexInfo objects sooner. This ticket will *not* raise the limit on cells. [~doanduyhai], you're right. Having the ability to handle big partitions has a direct influence to data modeling. I'd not say "you are not longer limited by the size of your partitions". This ticket _raises_ the current limitation WRT GC pressure and read performance. In theory the limit went away, but as you say, compaction gets even more important and other operational tasks like replacing nodes or changing topology need to be considered. My next steps are: * fix some unit tests that no longer work as they relied on the old implementation (expected to have IndexInfo on heap) * cleanup the code * run some tests on cstar I only ran a poor-man's comparison - on my laptop with small-ish 3G heap with default unit test settings. That's why I did not note exact numbers. But I'd like to show the GC pressure of the same test ran against trunk and 11206: !trunk-gc.png|GC on current trunk! !11206-gc.png|GC on 11206! was (Author: snazy): I just finished with most of the coding for this ticket - i.e. "shallow" RowIndexEntry without {{IndexInfo}} - and ran a poor-man's comparison of current trunk against 11206 using different partition sizes covering writes, a major compaction and reads. The results are really promising especially with big and huge partitions (tested up to 8G partitions). Reads against big partitions really benefit from 11206. For example, with 11206 it takes a couple of seconds for 5000 random reads against 8G partitions vs. many minutes (not a typo) on current trunk). At the same time, the heap is quite full and causes a lot of GC pressure. Compactions also benefit from 11206 GC-wise - but not CPU- or I/O-wise since it's still the same amount of work to be done. 11206 "just" reduces GC pressure. Flushes also benefit, since it can "forget" IndexInfo objects sooner. This ticket will *not* raise the limit on cells. [~doanduyhai], you're right. Having the ability to handle big partitions has a direct influence to data modeling. I'd not say "you are not longer limited by the size of your partitions". This ticket _raises_ the current limitation WRT GC pressure and read performance. In theory the limit went away, but as you say, compaction gets even more important and other operational tasks like replacing nodes or changing topology need to be considered. My next steps are: * fix some unit tests that no longer work as they relied on the old implementation (expected to have IndexInfo on heap) * cleanup the code * run some tests on cstar I only ran a poor-man's comparison - on my laptop with small-ish 3G heap with default unit test settings. That's why I did not note exact numbers. But I'd like to show the GC pressure of the same test ran against trunk and 11206: !trunk-gc.png!GC on current trunk! !11206-gc.png!GC on 11206! > Support large partitions on the 3.0 sstable format > -- > > Key: CASSANDRA-11206 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11206 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Robert Stupp > Fix For: 3.x > > Attachments: 11206-gc.png, trunk-gc.png > > > Cassandra saves a sample of IndexInfo objects that store the offset within > each partition of every 64KB (by default) range of rows. To find a row, we > binary search this sample, then scan the partition of the appropriate range. > The problem is that this scales poorly as partitions grow: on a cache miss, > we deserialize the entire set of IndexInfo, which both creates a lot of GC > overhead (as noted in CASSANDRA-9754) but is also non-negligible i/o activity > (relative to reading a single 64KB
[jira] [Comment Edited] (CASSANDRA-11206) Support large partitions on the 3.0 sstable format
[ https://issues.apache.org/jira/browse/CASSANDRA-11206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15175634#comment-15175634 ] Jonathan Ellis edited comment on CASSANDRA-11206 at 3/2/16 2:07 PM: bq. For partitions < 64k (partitions without an IndexInfo object) we could skip the indirection during reads via RowIndexEntry at all by extending the IndexSummary and directly store the offset into the data file Since the idea here is to do something simple that we can be confident about shipping in 3.6 if CASSANDRA-9754 isn't ready, let's avoid making changes to the on disk layout. To clarify for others following along, bq. Remove IndexInfo from the key cache (not from the index file on disk, of course) This sounds scary but it's core to the goal here: if we're going to support large partitions, we can't afford the overhead either of keeping the entire summary on heap, or of reading it from disk in the first place. (If we're reading a 1KB row, then reading 2MB of summary first on a cache miss is a huge overhead.) Moving the key cache off heap (CASSANDRA-9738) would have helped with the first but not the second. So one approach is to go back to the old strategy of only caching the partition key location, and then go through the index bsearch using the offsets map every time. For small partitions this will be trivial and I hope negligible to the performance story vs the current cache. (If not, we can look at a hybrid strategy, but I'd like to avoid that complexity if possible.) bq. what I was thinking was that the key cache instead of storing a copy of the RIE it would store an offset into the index that is the location of the RIE. Then the RIE could be accessed off heap via a memory mapping without doing any allocations or copies I was thinking that even the offsets alone for a 4GB partition are going to be 256KB, so we don't want to cache the entire offsets map. But the other side there is that if you have a bunch of 4GB partitions you won't have very many of them. 16TB of data would be 1GB of offsets which is within the bounds of reasonable when off heap. And your approach may require less logic changes than the one above, since we're still "caching" the entire summary, sort of; only adding an extra indirection to read the IndexInfo entries. So that might well be simpler. Edit: but switching to a per-row cache (from per-partition) would be a much bigger change and I don't see the performance implications as straightforward at all, so let's not do that. was (Author: jbellis): bq. For partitions < 64k (partitions without an IndexInfo object) we could skip the indirection during reads via RowIndexEntry at all by extending the IndexSummary and directly store the offset into the data file Since the idea here is to do something simple that we can be confident about shipping in 3.6 if CASSANDRA-9754 isn't ready, let's avoid making changes to the on disk layout. To clarify for others following along, bq. Remove IndexInfo from the key cache (not from the index file on disk, of course) This sounds scary but it's core to the goal here: if we're going to support large partitions, we can't afford the overhead either of keeping the entire summary on heap, or of reading it from disk in the first place. (If we're reading a 1KB row, then reading 2MB of summary first on a cache miss is a huge overhead.) Moving the key cache off heap (CASSANDRA-9738) would have helped with the first but not the second. So one approach is to go back to the old strategy of only caching the partition key location, and then go through the index bsearch using the offsets map every time. For small partitions this will be trivial and I hope negligible to the performance story vs the current cache. (If not, we can look at a hybrid strategy, but I'd like to avoid that complexity if possible.) bq. what I was thinking was that the key cache instead of storing a copy of the RIE it would store an offset into the index that is the location of the RIE. Then the RIE could be accessed off heap via a memory mapping without doing any allocations or copies I was thinking that even the offsets alone for a 4GB partition are going to be 256KB, so we don't want to cache the entire offsets map. But the other side there is that if you have a bunch of 4GB partitions you won't have very many of them. 16TB of data would be 1GB of offsets which is within the bounds of reasonable when off heap. And your approach may require less logic changes than the one above, since we're still "caching" the entire summary, sort of; only adding an extra indirection to read the IndexInfo entries. So that might well be simpler. > Support large partitions on the 3.0 sstable format > -- > > Key: CASSANDRA-11206 > URL:
[jira] [Comment Edited] (CASSANDRA-11206) Support large partitions on the 3.0 sstable format
[ https://issues.apache.org/jira/browse/CASSANDRA-11206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15175634#comment-15175634 ] Jonathan Ellis edited comment on CASSANDRA-11206 at 3/2/16 2:06 PM: bq. For partitions < 64k (partitions without an IndexInfo object) we could skip the indirection during reads via RowIndexEntry at all by extending the IndexSummary and directly store the offset into the data file Since the idea here is to do something simple that we can be confident about shipping in 3.6 if CASSANDRA-9754 isn't ready, let's avoid making changes to the on disk layout. To clarify for others following along, bq. Remove IndexInfo from the key cache (not from the index file on disk, of course) This sounds scary but it's core to the goal here: if we're going to support large partitions, we can't afford the overhead either of keeping the entire summary on heap, or of reading it from disk in the first place. (If we're reading a 1KB row, then reading 2MB of summary first on a cache miss is a huge overhead.) Moving the key cache off heap (CASSANDRA-9738) would have helped with the first but not the second. So one approach is to go back to the old strategy of only caching the partition key location, and then go through the index bsearch using the offsets map every time. For small partitions this will be trivial and I hope negligible to the performance story vs the current cache. (If not, we can look at a hybrid strategy, but I'd like to avoid that complexity if possible.) bq. what I was thinking was that the key cache instead of storing a copy of the RIE it would store an offset into the index that is the location of the RIE. Then the RIE could be accessed off heap via a memory mapping without doing any allocations or copies I was thinking that even the offsets alone for a 4GB partition are going to be 256KB, so we don't want to cache the entire offsets map. But the other side there is that if you have a bunch of 4GB partitions you won't have very many of them. 16TB of data would be 1GB of offsets which is within the bounds of reasonable when off heap. And your approach may require less logic changes than the one above, since we're still "caching" the entire summary, sort of; only adding an extra indirection to read the IndexInfo entries. So that might well be simpler. was (Author: jbellis): bq. For partitions < 64k (partitions without an IndexInfo object) we could skip the indirection during reads via RowIndexEntry at all by extending the IndexSummary and directly store the offset into the data file Since the idea here is to do something simple that we can be confident about shipping in 3.6 if CASSANDRA-9754 isn't ready, let's avoid making changes to the on disk layout, i.e., your Plan B. To clarify for others following along, bq. Remove IndexInfo from the key cache (not from the index file on disk, of course) This sounds scary but it's core to the goal here: if we're going to support large partitions, we can't afford the overhead either of keeping the entire summary on heap, or of reading it from disk in the first place. (If we're reading a 1KB row, then reading 2MB of summary first on a cache miss is a huge overhead.) Moving the key cache off heap (CASSANDRA-9738) would have helped with the first but not the second. So one approach is to go back to the old strategy of only caching the partition key location, and then go through the index bsearch using the offsets map every time. For small partitions this will be trivial and I hope negligible to the performance story vs the current cache. (If not, we can look at a hybrid strategy, but I'd like to avoid that complexity if possible.) bq. what I was thinking was that the key cache instead of storing a copy of the RIE it would store an offset into the index that is the location of the RIE. Then the RIE could be accessed off heap via a memory mapping without doing any allocations or copies I was thinking that even the offsets alone for a 4GB partition are going to be 256KB, so we don't want to cache the entire offsets map. But the other side there is that if you have a bunch of 4GB partitions you won't have very many of them. 16TB of data would be 1GB of offsets which is within the bounds of reasonable when off heap. And your approach may require less logic changes than the one above, since we're still "caching" the entire summary, sort of; only adding an extra indirection to read the IndexInfo entries. So that might well be simpler. > Support large partitions on the 3.0 sstable format > -- > > Key: CASSANDRA-11206 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11206 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >
[jira] [Comment Edited] (CASSANDRA-11206) Support large partitions on the 3.0 sstable format
[ https://issues.apache.org/jira/browse/CASSANDRA-11206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15174309#comment-15174309 ] Ariel Weisberg edited comment on CASSANDRA-11206 at 3/1/16 7:49 PM: I'm summarizing to make sure I remember correctly what the key cache miss read path for a table looks like. 1. Binary search index summary to find location of partition index entry in index 2. Lookup index entry which may just be a pointer to the data file, or it may be a sampled index of rows in the partition 3. Look up the partition contents based on the index entry The index summary is a sampling of the index so most of the time we aren't going to get a hit into the data file right? We have to scan the index to find the RIE and that entire process is what the key cache saves us from. If I remember correctly what I was thinking was that the key cache instead of storing a copy of the RIE it would store an offset into the index that is the location of the RIE. Then the RIE could be accessed off heap via a memory mapping without doing any allocations or copies. I agree that for partitions that aren't indexed the key cache could point straight to the data file and skip the index lookup since there doesn't need to be additional data there. I don't follow the path you are describing to completely removing the key cache without restructuring index summaries and indexes into something that is either traversed differently or doesn't summarize/sample. An aside. Is {{RowIndexEntry}} named incorrectly? Should it be {{PartitionIndexEntry}}? was (Author: aweisberg): I'm summarizing to make sure I remember correctly what the key cache miss read path for a table looks like. 1. Binary search index summary to find location of partition index entry in index 2. Lookup index entry which may just be a pointer to the data file, or it may be a sampled index of rows in the partition 3. Look up the partition contents based on the index entry The index summary is a sampling of the index so most of the time we aren't going to get a hit into the data file right? We have to scan the index to find the RIE and that entire process is what the key cache saves us from. If I remember correctly what I was thinking was that the key cache instead of storing a copy of the RIE it would store an offset into the index that is the location of the RIE. Then the RIE could be accessed off heap via a memory mapping without doing any allocations or copies. I agree that for partitions that aren't indexed the key cache could point straight to the data file and skip the index lookup since there doesn't need to be additional data there. I don't follow the path you are describing to completely removing the key cache without restructuring index summaries and indexes. An aside. Is {{RowIndexEntry}} named incorrectly? Should it be {{PartitionIndexEntry}}? > Support large partitions on the 3.0 sstable format > -- > > Key: CASSANDRA-11206 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11206 > Project: Cassandra > Issue Type: Improvement >Reporter: Jonathan Ellis >Assignee: Robert Stupp > Fix For: 3.x > > > Cassandra saves a sample of IndexInfo objects that store the offset within > each partition of every 64KB (by default) range of rows. To find a row, we > binary search this sample, then scan the partition of the appropriate range. > The problem is that this scales poorly as partitions grow: on a cache miss, > we deserialize the entire set of IndexInfo, which both creates a lot of GC > overhead (as noted in CASSANDRA-9754) but is also non-negligible i/o activity > (relative to reading a single 64KB row range) as partitions get truly large. > We introduced an "offset map" in CASSANDRA-10314 that allows us to perform > the IndexInfo bsearch while only deserializing IndexInfo that we need to > compare against, i.e. log(N) deserializations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)