hawq-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From benchristel <...@git.apache.org>
Subject [GitHub] incubator-hawq pull request #1379: HAWQ-1622. Cache PXF proxy UGI so that cl...
Date Thu, 12 Jul 2018 22:02:27 GMT
Github user benchristel commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1379#discussion_r202193131
  
    --- Diff: pxf/pxf-service/src/main/java/org/apache/hawq/pxf/service/UGICache.java ---
    @@ -0,0 +1,318 @@
    +package org.apache.hawq.pxf.service;
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.DelayQueue;
    +import java.util.concurrent.Delayed;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import com.google.common.base.Ticker;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.security.UserGroupInformation;
    +
    +/**
    + * Stores UserGroupInformation instances for each active session. The UGIs are cleaned
up if they
    + * have not been accessed for 15 minutes.
    + * <p>
    + * The motivation for caching is that destroying UGIs is slow. The alternative, creating
and
    + * destroying a UGI per-request, is wasteful.
    + */
    +public class UGICache {
    +
    +    private static final Log LOG = LogFactory.getLog(UGICache.class);
    +    private Map<SessionId, Entry> cache = new ConcurrentHashMap<>();
    +    @SuppressWarnings("unchecked")
    +    // There is a separate DelayQueue for each segment (also being used for locking)
    +    private final Map<Integer, DelayQueue<Entry>> queueMap = new HashMap<>();
    +    private final UGIProvider ugiProvider;
    +    private Ticker ticker;
    +    private final static long UGI_CACHE_EXPIRY = 15 * 60 * 1000L; // 15 Minutes
    +
    +    /**
    +     * Create a UGICache with the given {@link UGIProvider}. Intended for use by tests
which need
    +     * to substitute a mock UGIProvider.
    +     */
    +    UGICache(UGIProvider provider, Ticker ticker) {
    +        this.ticker = ticker;
    +        this.ugiProvider = provider;
    +    }
    +
    +    /**
    +     * Create a UGICache. Automatically creates a {@link UGIProvider} that this cache
will use to
    +     * create and destroy UserGroupInformation instances.
    +     */
    +    public UGICache() {
    +        this(new UGIProvider(), Ticker.systemTicker());
    +    }
    +
    +    /**
    +     * Create new proxy UGI if not found in cache and increment reference count
    +     */
    +    public UserGroupInformation getUserGroupInformation(SessionId session) throws IOException
{
    +        Integer segmentId = session.getSegmentId();
    +        String user = session.getUser();
    +        DelayQueue<Entry> delayQueue = getExpirationQueue(segmentId);
    +        synchronized (delayQueue) {
    +            // Use the opportunity to cleanup any expired entries
    +            cleanup(session);
    +            Entry entry = cache.get(session);
    +            if (entry == null) {
    +                LOG.info(session.toString() + " Creating proxy user = " + user);
    +                entry = new Entry(ticker, ugiProvider.createProxyUGI(user));
    +                delayQueue.offer(entry);
    +                cache.put(session, entry);
    +            }
    +            entry.acquireReference();
    +            return entry.getUGI();
    +        }
    +    }
    +
    +    /**
    +     * Decrement reference count for the given session's UGI. Resets the time at which
the UGI will
    +     * expire to 15 minutes in the future.
    +     *
    +     * @param session                  the session for which we want to release the UGI.
    +     * @param cleanImmediatelyIfNoRefs if true, destroys the UGI for the given session
(only if it is
    +     *                                 now unreferenced).
    +     */
    +    public void release(SessionId session, boolean cleanImmediatelyIfNoRefs) {
    --- End diff --
    
    I'd like the parameter names to allow someone reading this code to consider the cache
in isolation from its callers, and from that perspective, `cleanImmediatelyIfNoRefs` or some
variant seems like a good choice. The cache can't know or control whether it's the "lastCall"
to `release`.
    
    What if, instead of using a boolean parameter to change the behavior of this function,
we had a separate method `destroyUnreferencedUGIs` that clients could call to clean things
up? Then that method could be called in the `SecurityServletFilter` and the concept of "lastCall"
could be represented there.


---

Mime
View raw message