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

ASF GitHub Bot logged work on HIVE-19261:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 10/Oct/19 08:59
            Start Date: 10/Oct/19 08:59
    Worklog Time Spent: 10m 
      Work Description: dlavati commented on pull request #807: HIVE-19261: 
Avro SerDe's InstanceCache should not be synchronized on retrieve
URL: https://github.com/apache/hive/pull/807#discussion_r333403943
 
 

 ##########
 File path: serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
 ##########
 @@ -51,20 +51,19 @@ public Instance retrieve(SeedObject hv) throws 
AvroSerdeException {
    * Retrieve (or create if it doesn't exist) the correct Instance for this
    * SeedObject using 'seenSchemas' to resolve circular references
    */
-  public synchronized Instance retrieve(SeedObject hv,
-      Set<SeedObject> seenSchemas) throws AvroSerdeException {
+  public Instance retrieve(SeedObject hv, Set<SeedObject> seenSchemas)
+    throws AvroSerdeException {
     if(LOG.isDebugEnabled()) LOG.debug("Checking for hv: " + hv.toString());
 
     if(cache.containsKey(hv)) {
       if(LOG.isDebugEnabled()) LOG.debug("Returning cache result.");
       return cache.get(hv);
+    } else {
+      if(LOG.isDebugEnabled()) LOG.debug("Creating new instance and storing in 
cache");
 
 Review comment:
   @xhumanoid thanks for the contribution! I don't have committer rights to 
approve your patch, but LGTM with a few comments:
   - a small nitpick, but AFAIK, `isDebugEnabled` doesn't provide benefits 
here, only if you'd append/toString an expensive operation's result (yes, this 
is also ignored in a lot of places 😄)
   - to approve (+1) your changes, you'll have to become the assignee of the 
jira ticket that's currently under Fangshi Li (if that's fine with him) and 
change it's state with `Submit a patch` with an attached patch file (the 
previously attached file didn't get checked, as the issue state didn't change)
   - once all tests pass for it (might require a few reattachments/reruns), 
then someone will be able to approve it (also see 
https://cwiki.apache.org/confluence/display/Hive/HowToContribute#HowToContribute-CreatingaPatch)
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 326191)
    Time Spent: 40m  (was: 0.5h)

> 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
>              Labels: pull-request-available
>         Attachments: HIVE-19261.1.patch
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> 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.4#803005)

Reply via email to