[ https://issues.apache.org/jira/browse/CASSANDRA-7438?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14228563#comment-14228563 ]
Benedict commented on CASSANDRA-7438: ------------------------------------- [~aweisberg]: In my experience segments tend to be imperfectly distributed, so whilst there is bunching of resizes simply because they take so long, with real work going on at the same time they should be a _little_ spread out. Though with murmur3 the distribution may be significantly more uniform than my prior experiments. Either way, they're performed in parallel (without coordination) if they coincide, so it's still an improvement. [~vijay2...@yahoo.com]: When I talk about complexity, I mean the difficulties of concurrent programming magnified without the normal tools. For instance, there are the following concerns: * We have a spin-lock - admittedly one that should _generally_ be uncontended, but on a grow or a small map this is certainly not the case, which could result in really problematic behaviour. Pure spin locks should not be used outside of the kernel. * The queue is maintained by a separate thread that requires signalling if it isn't currently performing work - which, in a real C* instance where the cost of linking the queue item is a fraction of the other work done to service a request means we are likely to incur a costly unpark() for a majority of operations * Reads can interleave with put/replace/remove and abort the removal of an item from the queue, resulting in a memory leak. * We perform the grow on a separate thread, but prevent all reader _or_ writer threads from making progress by taking the locks for all buckets immediately. * Freeing of oldSegments is still dangerous, it's just probabilistically less likely to happen. * During a grow, we can lose puts because we unlock the old segments, so with the right (again, unlikely) interleaving of events a writer can think the old table is still valid * When growing, we only double the size of the backing table, however since grows happen in the background the updater can get ahead, meaning we remain behind and multiply the constant factor overheads, collisions and contention until total size tails off. These are only the obvious problems that spring to mind from 15m perusing the code, I'm sure there are others. This kind of stuff is really hard, and the approach I'm suggesting is comparatively a doddle to get right, and is likely faster to boot. I'm not sure I understand your concern with segmentation creating complexity with the hashing... I'm proposing the exact method used by CHM. We have an excellent hash algorithm to distribute the data over the segments: murmurhash3. Although we need to be careful to not use the bits that don't have the correct entropy for selecting a segment. It's really no more than a two-tier hash table. The user doesn't need to know anything about this. > Serializing Row cache alternative (Fully off heap) > -------------------------------------------------- > > Key: CASSANDRA-7438 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7438 > Project: Cassandra > Issue Type: Improvement > Components: Core > Environment: Linux > Reporter: Vijay > Assignee: Vijay > Labels: performance > Fix For: 3.0 > > Attachments: 0001-CASSANDRA-7438.patch, tests.zip > > > Currently SerializingCache is partially off heap, keys are still stored in > JVM heap as BB, > * There is a higher GC costs for a reasonably big cache. > * Some users have used the row cache efficiently in production for better > results, but this requires careful tunning. > * Overhead in Memory for the cache entries are relatively high. > So the proposal for this ticket is to move the LRU cache logic completely off > heap and use JNI to interact with cache. We might want to ensure that the new > implementation match the existing API's (ICache), and the implementation > needs to have safe memory access, low overhead in memory and less memcpy's > (As much as possible). > We might also want to make this cache configurable. -- This message was sent by Atlassian JIRA (v6.3.4#6332)