hawq-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From lavjain <...@git.apache.org>
Subject [GitHub] incubator-hawq pull request #1379: WIP: Cache UGI objects and clean them per...
Date Wed, 04 Jul 2018 16:19:13 GMT
Github user lavjain commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1379#discussion_r200169586
  
    --- Diff: pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/servlet/SecurityServletFilter.java
---
    @@ -89,32 +188,103 @@ public Boolean run() throws IOException, ServletException {
                 };
     
                 // create proxy user UGI from the UGI of the logged in user and execute the
servlet chain as that user
    -            UserGroupInformation proxyUGI = null;
    +            TimedProxyUGI timedProxyUGI = getTimedProxyUGI(user, session);
                 try {
    -                LOG.debug("Creating proxy user = " + user);
    -                proxyUGI = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser());
    -                proxyUGI.doAs(action);
    +                timedProxyUGI.proxyUGI.doAs(action);
                 } catch (UndeclaredThrowableException ute) {
                     // unwrap the real exception thrown by the action
                     throw new ServletException(ute.getCause());
                 } catch (InterruptedException ie) {
                     throw new ServletException(ie);
    -            } finally {
    -                try {
    -                    if (proxyUGI != null) {
    -                        LOG.debug("Closing FileSystem for proxy user = " + proxyUGI.getUserName());
    -                        FileSystem.closeAllForUGI(proxyUGI);
    -                    }
    -                } catch (Throwable t) {
    -                    LOG.warn("Error closing FileSystem for proxy user = " + proxyUGI.getUserName());
    -                }
    +            }
    +            finally {
    +                release(timedProxyUGI, fragmentIndex, fragmentCount);
                 }
             } else {
                 // no user impersonation is configured
                 chain.doFilter(request, response);
             }
         }
     
    +   private TimedProxyUGI getTimedProxyUGI(String user, SegmentTransactionId session)
throws IOException {
    +        synchronized (session.segmentTransactionId.intern()) {
    +            TimedProxyUGI timedProxyUGI = cache.get(session);
    +            if (timedProxyUGI == null || timedProxyUGI.getDelayMillis() < 0) {
    +                cleanup();
    +                LOG.info(session.toString() + " Creating proxy user = " + user);
    +                UserGroupInformation proxyUGI =
    +                        UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser());
    +                timedProxyUGI = new TimedProxyUGI(proxyUGI, session);
    +                delayQueue.offer(timedProxyUGI);
    +                cache.put(session, timedProxyUGI);
    +            } else {
    +                timedProxyUGI.incrementCounter();
    +            }
    +            return timedProxyUGI;
    +        }
    +    }
    +
    +    private void release(TimedProxyUGI timedProxyUGI, Integer fragmentIndex, Integer
fragmentCount) {
    +        synchronized (timedProxyUGI.session.segmentTransactionId.intern()) {
    +            timedProxyUGI.resetTime();
    +            timedProxyUGI.decrementCounter();
    +            if (fragmentIndex != null && fragmentCount.equals(fragmentIndex))
    +                closeUGI(timedProxyUGI);
    +        }
    +    }
    +
    +    private void cleanup() {
    +        TimedProxyUGI timedProxyUGI = delayQueue.poll();
    +        while (timedProxyUGI != null) {
    +            closeUGI(timedProxyUGI);
    +            LOG.info(timedProxyUGI.session.toString() + " Delay Queue Size = " + delayQueue.size());
    +            timedProxyUGI = delayQueue.poll();
    +        }
    +    }
    +
    +    private void closeUGI(TimedProxyUGI timedProxyUGI) {
    +        synchronized (timedProxyUGI.session.segmentTransactionId.intern()) {
    --- End diff --
    
    Good catch. The locking should only operate on one lock at a time. This could be accomplished
if the clean method is called by a reaper thread.


---

Mime
View raw message