[ https://issues.apache.org/jira/browse/HIVE-19261?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16507934#comment-16507934 ]
Ganesha Shreedhara commented on HIVE-19261: ------------------------------------------- [~shengzhixia] You might have to change the way of initializing cache variable as below if you want to use ConcurrentHashMap's putIfAbsent method. {code:java} ConcurrentHashMap<SeedObject, Instance> cache = new ConcurrentHashMap<SeedObject, Instance>();{code} putIfAbsent method was added to ConcurrentHashMap class in java 1.7 version but it was added to Map class in java 1.8 version. > Avro SerDe's InstanceCache should not be synchronized on retrieve > ----------------------------------------------------------------- > > Key: HIVE-19261 > URL: https://issues.apache.org/jira/browse/HIVE-19261 > Project: Hive > Issue Type: Improvement > Reporter: Fangshi Li > Assignee: Fangshi Li > Priority: Major > Attachments: HIVE-19261.1.patch > > > In HIVE-16175, upstream made a patch to fix the thread safety issue in > AvroSerDe's InstanceCache. This fix made the retrieve method in InstanceCache > synchronized. While it should make InstanceCache thread-safe, making retrieve > synchronized for the cache can be expensive in highly concurrent environment > like Spark, as multiple threads need to be synchronized on entering the > entire retrieve method. > We are proposing another way to fix this thread safety issue by making the > underlying map of InstanceCache as ConcurrentHashMap. Ideally, we can use > atomic computeIfAbsent in the retrieve method to avoid synchronizing the > entire method. > While computeIfAbsent is only available on java 8 and java 7 is still > supported in Hive, > we use a pattern to simulate the behavior of computeIfAbsent. In the future, > we should move to computeIfAbsent when Hive requires java 8. -- This message was sent by Atlassian JIRA (v7.6.3#76005)