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

Mithun Radhakrishnan commented on HIVE-19261:
---------------------------------------------

Hello, [~shengzhixia], et al.

This looks like a good fix, generally. A couple of suggestions:
 # Please consider the rephrasing the ending of {{InstanceCache::retrieve()}} 
as follows:
 ## {code:java}
Instance newInstance = makeInstance(hv, seenSchemas);
instance = cache.putIfAbsent(hv, newInstance);
return instance == null? newInstance : instance;{code}
 # Please remove the documentation regarding 
{{ConcurrentHashMap::computeIfAbsent()}}; it won't work here. 
{{makeInstance()}} lands up making reentrant calls to 
{{InstanceCache::retrieve()}}. {{computeIfAbsent()}} requires that the 
{{cache}} not be modified within a single call to the {{putIfAbsent()}} lambda 
(i.e. {{hv -> makeInstance(hv, seenSchemas)}}. As such, this will cause 
{{retrieve()}} to hang on the first reentrant call.

> 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
(v8.3.2#803003)

Reply via email to