dlmarion commented on code in PR #6156:
URL: https://github.com/apache/accumulo/pull/6156#discussion_r2874189385


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineTabletLocatorImpl.java:
##########
@@ -0,0 +1,365 @@
+/*
+ * 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
+ *
+ *   https://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.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.util.Timer;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Policy.Eviction;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.Ticker;
+import com.google.common.base.Preconditions;
+
+import io.micrometer.core.instrument.Metrics;
+import io.micrometer.core.instrument.binder.cache.CaffeineStatsCounter;
+
+public class OfflineTabletLocatorImpl extends TabletLocator {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(OfflineTabletLocatorImpl.class);
+
+  public static class OfflineTabletLocation extends TabletLocation {
+
+    public static final String SERVER = "offline_table_marker";
+
+    public OfflineTabletLocation(KeyExtent tablet_extent) {
+      super(tablet_extent, SERVER, SERVER);
+    }
+
+  }
+
+  private class OfflineTabletsCache implements 
RemovalListener<KeyExtent,KeyExtent> {
+
+    // This object uses a Caffeine cache to manage the duration of the extents
+    // cached in the TreeSet. The TreeSet is necessary for expedient 
operations.
+
+    private final ClientContext context;
+    private final int maxCacheSize;
+    private final int prefetch;
+    private final Cache<KeyExtent,KeyExtent> cache;
+    private final LinkedBlockingQueue<KeyExtent> evictions = new 
LinkedBlockingQueue<>();
+    private final TreeSet<KeyExtent> extents = new TreeSet<>();
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final Timer scanTimer = Timer.startNew();
+    private final AtomicInteger cacheCount = new AtomicInteger(0);
+    private final Eviction<KeyExtent,KeyExtent> evictionPolicy;
+
+    private OfflineTabletsCache(ClientContext context) {
+      this.context = context;
+      Properties clientProperties = context.getProperties();
+      Duration cacheDuration = Duration.ofMillis(
+          
ClientProperty.OFFLINE_LOCATOR_CACHE_DURATION.getTimeInMillis(clientProperties));
+      maxCacheSize =
+          
Integer.parseInt(ClientProperty.OFFLINE_LOCATOR_CACHE_SIZE.getValue(clientProperties));
+      prefetch = Integer
+          
.parseInt(ClientProperty.OFFLINE_LOCATOR_CACHE_PREFETCH.getValue(clientProperties));
+
+      // This cache is used to evict KeyExtents from the extents TreeSet when
+      // they have not been accessed in cacheDuration. We are targeting to have
+      // maxCacheSize objects in the cache, but are not using the Cache's 
maximumSize
+      // to achieve this as the Cache will remove things from the Cache that 
were
+      // newly inserted and not yet used. This negates the pre-fetching feature
+      // that we have added into this TabletLocator for offline tables. Here we
+      // set the maximum size much larger that the property and use the 
cacheCount
+      // variable to manage the max size manually.
+      cache = 
Caffeine.newBuilder().expireAfterAccess(cacheDuration).initialCapacity(maxCacheSize)
+          .maximumSize(maxCacheSize * 
2).removalListener(this).ticker(Ticker.systemTicker())
+          .recordStats(() -> new CaffeineStatsCounter(Metrics.globalRegistry,
+              OfflineTabletsCache.class.getSimpleName()))
+          .build();
+      evictionPolicy = cache.policy().eviction().orElseThrow();
+    }
+
+    @Override
+    public void onRemoval(KeyExtent key, KeyExtent value, RemovalCause cause) {
+      if (cause == RemovalCause.REPLACED) {
+        return;
+      }
+      LOG.trace("Extent {} was evicted from cache for {} ", key, cause);
+      cacheCount.decrementAndGet();
+      evictions.add(key);
+      try {
+        if (lock.writeLock().tryLock(1, TimeUnit.MILLISECONDS)) {
+          try {
+            processRecentCacheEvictions();
+          } finally {
+            lock.writeLock().unlock();
+          }
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException("Interrupted while waiting to acquire write 
lock", e);
+      }
+    }
+
+    private void processRecentCacheEvictions() {
+      Preconditions.checkArgument(lock.writeLock().isHeldByCurrentThread());
+      Set<KeyExtent> copy = new HashSet<>();
+      evictions.drainTo(copy);
+      int numEvictions = copy.size();
+      if (numEvictions > 0) {
+        LOG.trace("Processing {} prior evictions", numEvictions);
+        extents.removeAll(copy);
+      }
+    }
+
+    private KeyExtent findOrLoadExtent(KeyExtent searchKey) {
+      lock.readLock().lock();
+      try {
+        KeyExtent match = extents.ceiling(searchKey);
+        if (match != null && match.contains(searchKey.endRow())) {
+          // update access time in cache
+          @SuppressWarnings("unused")
+          var unused = cache.getIfPresent(match);
+          LOG.trace("Extent {} found in cache for start row {}", match, 
searchKey);
+          return match;
+        }
+      } finally {
+        lock.readLock().unlock();
+      }
+      lock.writeLock().lock();
+      // process prior evictions since we have the write lock
+      processRecentCacheEvictions();
+      // The following block of code fixes an issue with
+      // the cache where recently pre-fetched extents
+      // will be evicted from the cache when it reaches
+      // the maxCacheSize. This is because from the cache's
+      // perspective they are the coldest objects. The code
+      // below manually removes the coldest extents that are
+      // before the searchKey.endRow to make room for the next
+      // batch of extents that we are going to load into the
+      // cache so that they are not immediately evicted.
+      if (cacheCount.get() + prefetch + 1 >= maxCacheSize) {
+        int evictionSize = prefetch * 2;
+        Set<KeyExtent> candidates = new 
HashSet<>(evictionPolicy.coldest(evictionSize).keySet());

Review Comment:
   `evictionPolicy.coldest()` works when setting the Caffeine `maximumSize` 
option. Another approach would be to *not* set the `maximumSize` on the Cache 
at all, and use 
`cache.policy().expireAfterAccess().orElseThrow().youngest(int)` instead.  I'm 
not sure, but this may be functionally equivalent but might allow us to remove 
some code trying to deal with Caffeine removing objects prematurely based on 
size.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to