[ 
https://issues.apache.org/jira/browse/CASSANDRA-13271?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15886318#comment-15886318
 ] 

Robert Stupp edited comment on CASSANDRA-13271 at 2/27/17 7:33 PM:
-------------------------------------------------------------------

The changes for {{ListType}} and {{SetType}} look good so far. However, you 
miss some similar one in {{MapType}} and {{ReversedType}} plus the 
corresponding {{List/Set/MapSerializer}} classes, that use the same pattern.
Can you provide a branch on GitHub instead of a patch file?

EDIT: Also {{DynamicCompositeType}}. You might also switch the pattern to 
something like this:
{code}
X t t = internMap.get(key);
if (t == null)
    internMap.computeIfAbsent(key, (key) -> ...);
{code}
In the rare case when an instance is missing, we can afford the extra overhead 
but favor the cleaner code.


was (Author: snazy):
The changes for {{ListType}} and {{SetType}} look good so far. However, you 
miss some similar one in {{MapType}} and {{ReversedType}} plus the 
corresponding {{List/Set/MapSerializer}} classes, that use the same pattern.
Can you provide a branch on GitHub instead of a patch file?

> Reduce lock contention on instance factories of ListType and SetType
> --------------------------------------------------------------------
>
>                 Key: CASSANDRA-13271
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13271
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: vincent royer
>            Priority: Minor
>              Labels: performance
>             Fix For: 4.x
>
>         Attachments: 
> 0001-CASSANDRA-13271-singleton-factory-concurrency-opimiz.patch
>
>
> By doing some performance tests, i noticed that getInstance() in 
> org.apache.cassandra.db.marshal.ListType and SetType could suffer from lock 
> contention on the singleton factory getInstance(). Here is a proposal to 
> reduce lock contention by using a ConcurrentMap and the putIfAbsent method 
> rather than a synchronized method.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to