This is an automated email from the ASF dual-hosted git repository.

ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo-classloaders.git


The following commit(s) were added to refs/heads/main by this push:
     new 264faa7  Use a deduplicating cache implementation (#34)
264faa7 is described below

commit 264faa708c0a326bfd8a9dd6c197b282f97da818
Author: Christopher Tubbs <[email protected]>
AuthorDate: Tue Dec 16 14:36:38 2025 -0500

    Use a deduplicating cache implementation (#34)
    
    Core changes:
    
    * Use a generic DeduplicationCache implementation with the property of
      guaranteeing no duplicate classloader instances for any given context
      version, but also with a minimum on-access lifetime
    * Track the current context versions separately from the classloader
      instances, so there's only one update task per context, but immutable
      instances of URLClassloader in the deduplication cache are independent
      from the updating (makes it easier to reason about the lifetime of
      objects, and the update code); this removes the update method from the
      LocalCachingContext object and replaces it with a new instance (TODO:
      this could be further cleaned up since LocalCachingContext is now just
      a bag of utility code rather than something that holds classloader
      state to be updated)
    * Use compute method to atomically update the current context
      definitions and access the deduplciated URLClassloader instances to
      ensure that the two structures are always coherent
    * Use LinkedHashSet instead of TreeSet to preserve Resource ordering, as
      specified in the user's context file
    * Remove the unused LocalCachingContext.update method, and ensure that
      LocalCachingContext.initialize creates unique local directory names, to
      support distinct versions of each context.
      Currently, this results in duplicate downloads, but later, this will be
      addressed by having a common pool of files.
    * Saves the context definition json file to the local context cache 
directory.
    
    Other changes:
    
    * update to latest spotbugs to remove some spammy console output
    * remove use of deprecated doPrivileged and suppress spotbugs warning
    * add justifications to spotbugs warnings about reading from user
      provided URL
    * remove some spotbugs warnings by removing setters to make
      ContextDefinition and Resource objects immutable
    * use asSubclass to avoid warnings about casting during dynamic class
      loading
    * Create WrappedException for use in lambdas to make wrapping/unwrapping
      more predictable
    * (unrelated) Update VFSManager to guard the initialization of the VFS
      static field with an AtomicBoolean to avoid a spotbugs warning
    * suppress Eclipse warnings about maven-dependency-plugin executions
---
 .../classloader/lcc/LocalCachingContext.java       |  50 ++------
 .../lcc/LocalCachingContextClassLoaderFactory.java | 137 +++++++++++++--------
 ...HttpFileResolver.java => WrappedException.java} |  27 +---
 .../classloader/lcc/cache/DeduplicationCache.java  |  65 ++++++++++
 .../lcc/definition/ContextDefinition.java          |  36 ++----
 .../classloader/lcc/definition/Resource.java       |   8 --
 .../lcc/resolvers/HttpFileResolver.java            |   3 +-
 .../LocalCachingContextClassLoaderFactoryTest.java |  59 +++------
 .../classloader/lcc/LocalCachingContextTest.java   |  25 ++--
 .../apache/accumulo/classloader/lcc/TestUtils.java |   7 +-
 .../accumulo/classloader/vfs/VFSManager.java       |   4 +-
 pom.xml                                            |  16 ++-
 12 files changed, 236 insertions(+), 201 deletions(-)

diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContext.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContext.java
index 43184f4..38457e0 100644
--- 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContext.java
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContext.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.classloader.lcc;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
 import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
 import static java.util.Objects.hash;
@@ -31,8 +32,6 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -52,6 +51,8 @@ import org.apache.accumulo.core.util.Retry.RetryFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
 public final class LocalCachingContext {
 
   private static class ClassPathElement {
@@ -118,7 +119,8 @@ public final class LocalCachingContext {
       throws IOException, ContextClassLoaderException {
     this.definition.set(requireNonNull(contextDefinition, "definition must be 
supplied"));
     this.contextName = this.definition.get().getContextName();
-    this.contextCacheDir = CacheUtils.createOrGetContextCacheDir(baseCacheDir, 
contextName);
+    this.contextCacheDir = CacheUtils.createOrGetContextCacheDir(baseCacheDir,
+        contextName + "_" + this.definition.get().getChecksum());
   }
 
   public ContextDefinition getDefinition() {
@@ -187,6 +189,8 @@ public final class LocalCachingContext {
       throws InterruptedException, IOException, ContextClassLoaderException, 
URISyntaxException {
     try {
       LockInfo lockInfo = CacheUtils.lockContextCacheDir(contextCacheDir);
+      Files.write(contextCacheDir.resolve("definition_" + 
definition.get().getChecksum()),
+          definition.get().toJson().getBytes(UTF_8));
       while (lockInfo == null) {
         // something else is updating this directory
         LOG.info("Directory {} locked, another process must be updating the 
class loader contents. "
@@ -207,39 +211,11 @@ public final class LocalCachingContext {
     }
   }
 
-  public void update(final ContextDefinition update)
-      throws InterruptedException, IOException, ContextClassLoaderException, 
URISyntaxException {
-    requireNonNull(update, "definition must be supplied");
-    if (definition.get().getResources().equals(update.getResources())) {
-      return;
-    }
-    try {
-      LockInfo lockInfo = CacheUtils.lockContextCacheDir(contextCacheDir);
-      while (lockInfo == null) {
-        // something else is updating this directory
-        LOG.info("Directory {} locked, another process must be updating the 
class loader contents. "
-            + "Retrying in 1 second", contextCacheDir);
-        Thread.sleep(1000);
-        lockInfo = CacheUtils.lockContextCacheDir(contextCacheDir);
-      }
-      synchronized (elements) {
-        try {
-          elements.clear();
-          cacheResources(update);
-          this.definition.set(update);
-        } finally {
-          lockInfo.unlock();
-        }
-      }
-    } catch (Exception e) {
-      LOG.error("Error updating context: " + contextName, e);
-      throw e;
-    }
-  }
-
-  public ClassLoader getClassloader() {
+  @SuppressFBWarnings(value = "DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED",
+      justification = "doPrivileged is deprecated without replacement and 
removed in newer Java")
+  public URLClassLoader getClassloader() {
 
-    ClassLoader currentCL = classloader.get();
+    URLClassLoader currentCL = classloader.get();
     if (currentCL != null) {
       return currentCL;
     }
@@ -258,9 +234,7 @@ public final class LocalCachingContext {
         urls[x] = iter.next().getLocalCachedCopyLocation();
       }
       final URLClassLoader cl =
-          AccessController.doPrivileged((PrivilegedAction<URLClassLoader>) () 
-> {
-            return new URLClassLoader(contextName, urls, 
this.getClass().getClassLoader());
-          });
+          new URLClassLoader(contextName, urls, 
this.getClass().getClassLoader());
       classloader.set(cl);
       LOG.trace("New classloader created from URLs: {}",
           Arrays.asList(classloader.get().getURLs()));
diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactory.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactory.java
index 14e7c32..b3dd6fe 100644
--- 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactory.java
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactory.java
@@ -27,15 +27,18 @@ import java.io.InputStreamReader;
 import java.net.MalformedURLException;
 import java.net.URISyntaxException;
 import java.net.URL;
-import java.security.NoSuchAlgorithmException;
+import java.net.URLClassLoader;
 import java.time.Duration;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 
 import org.apache.accumulo.classloader.lcc.cache.CacheUtils;
+import org.apache.accumulo.classloader.lcc.cache.DeduplicationCache;
 import org.apache.accumulo.classloader.lcc.definition.ContextDefinition;
 import org.apache.accumulo.classloader.lcc.definition.Resource;
 import org.apache.accumulo.classloader.lcc.resolvers.FileResolver;
@@ -45,8 +48,6 @@ import org.apache.accumulo.core.util.Timer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.github.benmanes.caffeine.cache.Cache;
-import com.github.benmanes.caffeine.cache.Caffeine;
 import com.google.common.base.Preconditions;
 
 /**
@@ -80,16 +81,24 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
   private static final Logger LOG =
       LoggerFactory.getLogger(LocalCachingContextClassLoaderFactory.class);
 
-  private final Cache<String,LocalCachingContext> contexts =
-      Caffeine.newBuilder().expireAfterAccess(24, TimeUnit.HOURS).build();
+  private final DeduplicationCache<String,LocalCachingContext,URLClassLoader> 
classloaders =
+      new DeduplicationCache<>((String key, LocalCachingContext lcc) -> 
lcc.getClassloader(),
+          Duration.ofHours(24));
 
   private final Map<String,Timer> classloaderFailures = new HashMap<>();
   private volatile String baseCacheDir;
   private volatile Duration updateFailureGracePeriodMins;
 
-  private ContextDefinition parseContextDefinition(final URL url)
+  private ContextDefinition parseContextDefinition(final String 
contextLocation)
       throws ContextClassLoaderException {
-    LOG.trace("Retrieving context definition file from {}", url);
+    URL url;
+    try {
+      url = new URL(contextLocation);
+    } catch (MalformedURLException e) {
+      throw new ContextClassLoaderException(
+          "Expected valid URL to context definition file but received: " + 
contextLocation, e);
+    }
+    LOG.trace("Retrieving context definition file from {}", contextLocation);
     final FileResolver resolver = FileResolver.resolve(url);
     try {
       try (InputStream is = resolver.getInputStream()) {
@@ -113,21 +122,30 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
    * (if it changed).
    */
   private void monitorContext(final String contextLocation, int interval) {
-    Constants.EXECUTOR.schedule(() -> {
-      final LocalCachingContext classLoader =
-          contexts.policy().getIfPresentQuietly(contextLocation);
-      if (classLoader == null) {
+    final Runnable updateTask = () -> {
+      ContextDefinition currentDef = contextDefs.compute(contextLocation, (k, 
v) -> {
+        if (v == null) {
+          return null;
+        }
+        if (!classloaders
+            .anyMatch(k2 -> k2.substring(0, 
k2.lastIndexOf('-')).equals(v.getContextName()))) {
+          // context has been removed from the map, no need to check for update
+          LOG.debug("ClassLoader for context {} not present, no longer 
monitoring for changes",
+              contextLocation);
+          return null;
+        }
+        return v;
+      });
+      if (currentDef == null) {
         // context has been removed from the map, no need to check for update
-        LOG.debug("ClassLoader for context {} not present, no longer 
monitoring for changes",
+        LOG.debug("ContextDefinition for context {} not present, no longer 
monitoring for changes",
             contextLocation);
         return;
       }
       int nextInterval = interval;
-      final ContextDefinition currentDef = classLoader.getDefinition();
       try {
-        final URL contextLocationUrl = new URL(contextLocation);
-        final ContextDefinition update = 
parseContextDefinition(contextLocationUrl);
-        if (!Arrays.equals(currentDef.getChecksum(), update.getChecksum())) {
+        final ContextDefinition update = 
parseContextDefinition(contextLocation);
+        if (!currentDef.getChecksum().equals(update.getChecksum())) {
           LOG.debug("Context definition for {} has changed", contextLocation);
           if (!currentDef.getContextName().equals(update.getContextName())) {
             LOG.warn(
@@ -135,14 +153,16 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
                 contextLocation, currentDef.getContextName(), 
currentDef.getContextName(),
                 update.getContextName());
           }
-          classLoader.update(update);
+          LocalCachingContext newCcl = new LocalCachingContext(baseCacheDir, 
update);
+          newCcl.initialize();
+          contextDefs.put(contextLocation, update);
           nextInterval = update.getMonitorIntervalSeconds();
           classloaderFailures.remove(contextLocation);
         } else {
           LOG.trace("Context definition for {} has not changed", 
contextLocation);
         }
-      } catch (ContextClassLoaderException | InterruptedException | IOException
-          | NoSuchAlgorithmException | URISyntaxException | RuntimeException 
e) {
+      } catch (ContextClassLoaderException | InterruptedException | 
IOException | URISyntaxException
+          | RuntimeException e) {
         LOG.error("Error parsing updated context definition at {}. Classloader 
NOT updated!",
             contextLocation, e);
         final Timer failureTimer = classloaderFailures.get(contextLocation);
@@ -162,7 +182,7 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
           // will return from getClassLoader in the calling thread
           LOG.info("Grace period for failing classloader has elapsed for 
context {}",
               contextLocation);
-          contexts.invalidate(contextLocation);
+          contextDefs.remove(contextLocation);
           classloaderFailures.remove(contextLocation);
         } else {
           LOG.trace("Failing to update classloader for context {} within the 
grace period",
@@ -171,7 +191,8 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
       } finally {
         monitorContext(contextLocation, nextInterval);
       }
-    }, interval, TimeUnit.SECONDS);
+    };
+    Constants.EXECUTOR.schedule(updateTask, interval, TimeUnit.SECONDS);
     LOG.trace("Monitoring context definition file {} for changes at {} second 
intervals",
         contextLocation, interval);
   }
@@ -180,8 +201,7 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
   void resetForTests() {
     // Removing the contexts will cause the
     // background monitor task to end
-    contexts.invalidateAll();
-    contexts.cleanUp();
+    contextDefs.clear();
   }
 
   @Override
@@ -198,47 +218,58 @@ public class LocalCachingContextClassLoaderFactory 
implements ContextClassLoader
     }
   }
 
+  ConcurrentHashMap<String,ContextDefinition> contextDefs = new 
ConcurrentHashMap<>();
+
   @Override
   public ClassLoader getClassLoader(final String contextLocation)
       throws ContextClassLoaderException {
     Preconditions.checkState(baseCacheDir != null, "init not called before 
calling getClassLoader");
     requireNonNull(contextLocation, "context name must be supplied");
+    final AtomicBoolean newlyCreated = new AtomicBoolean(false);
+    final AtomicReference<URLClassLoader> cl = new AtomicReference<>();
+    ContextDefinition def;
     try {
-      final URL contextLocationUrl = new URL(contextLocation);
-      final AtomicBoolean newlyCreated = new AtomicBoolean(false);
-      final LocalCachingContext ccl = contexts.get(contextLocation, cn -> {
-        try {
-          ContextDefinition def = parseContextDefinition(contextLocationUrl);
-          LocalCachingContext newCcl = new LocalCachingContext(baseCacheDir, 
def);
-          newCcl.initialize();
+      def = contextDefs.compute(contextLocation, (k, v) -> {
+        ContextDefinition def2;
+        if (v == null) {
           newlyCreated.set(true);
-          return newCcl;
-        } catch (Exception e) {
-          throw new RuntimeException("Error creating context classloader", e);
+          try {
+            def2 = parseContextDefinition(k);
+          } catch (ContextClassLoaderException e) {
+            throw new WrappedException(e);
+          }
+        } else {
+          def2 = v;
         }
+        final URLClassLoader cl2 =
+            classloaders.computeIfAbsent(def2.getContextName() + "-" + 
def2.getChecksum(),
+                (Supplier<LocalCachingContext>) () -> {
+                  try {
+                    LocalCachingContext newCcl = new 
LocalCachingContext(baseCacheDir, def2);
+                    newCcl.initialize();
+                    return newCcl;
+                  } catch (Exception e) {
+                    throw new WrappedException(e);
+                  }
+                });
+        cl.set(cl2);
+        return def2;
       });
-      if (newlyCreated.get()) {
-        monitorContext(contextLocation, 
ccl.getDefinition().getMonitorIntervalSeconds());
-      }
-      return ccl.getClassloader();
-    } catch (MalformedURLException e) {
-      throw new ContextClassLoaderException(
-          "Expected valid URL to context definition file but received: " + 
contextLocation, e);
-    } catch (RuntimeException re) {
-      Throwable t = re.getCause();
-      if (t != null && t instanceof InterruptedException) {
+    } catch (WrappedException e) {
+      Throwable t = e.getCause();
+      if (t instanceof InterruptedException) {
         Thread.currentThread().interrupt();
+      } else if (t instanceof ContextClassLoaderException) {
+        throw (ContextClassLoaderException) t;
       }
-      if (t != null) {
-        if (t instanceof ContextClassLoaderException) {
-          throw (ContextClassLoaderException) t;
-        } else {
-          throw new ContextClassLoaderException(t.getMessage(), t);
-        }
-      } else {
-        throw new ContextClassLoaderException(re.getMessage(), re);
-      }
+      throw new ContextClassLoaderException(t.getMessage(), t);
+    } catch (RuntimeException e) {
+      throw new ContextClassLoaderException(e.getMessage(), e);
+    }
+    if (newlyCreated.get()) {
+      monitorContext(contextLocation, def.getMonitorIntervalSeconds());
     }
+    return cl.get();
   }
 
 }
diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/WrappedException.java
similarity index 52%
copy from 
modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
copy to 
modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/WrappedException.java
index 76f8442..f1a3f25 100644
--- 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/WrappedException.java
@@ -16,31 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.classloader.lcc.resolvers;
+package org.apache.accumulo.classloader.lcc;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
+import static java.util.Objects.requireNonNull;
 
-import 
org.apache.accumulo.core.spi.common.ContextClassLoaderFactory.ContextClassLoaderException;
+public class WrappedException extends RuntimeException {
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+  private static final long serialVersionUID = 1L;
 
-public final class HttpFileResolver extends FileResolver {
-
-  protected HttpFileResolver(URL url) throws ContextClassLoaderException {
-    super(url);
-  }
-
-  @Override
-  public String getFileName() {
-    String path = getURL().getPath();
-    return path.substring(path.lastIndexOf("/") + 1);
+  public WrappedException(final Exception e) {
+    super(requireNonNull(e));
   }
 
-  @Override
-  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD")
-  public InputStream getInputStream() throws IOException {
-    return getURL().openStream();
-  }
 }
diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/cache/DeduplicationCache.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/cache/DeduplicationCache.java
new file mode 100644
index 0000000..a04bb2d
--- /dev/null
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/cache/DeduplicationCache.java
@@ -0,0 +1,65 @@
+/*
+ * 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.classloader.lcc.cache;
+
+import java.time.Duration;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+/**
+ * A simple de-duplication cache of weakly referenced values that retains a 
strong reference for a
+ * minimum amount of time specified, to prevent garbage collection too 
frequently for objects that
+ * may be used again.
+ */
+public class DeduplicationCache<KEY,PARAMS,VALUE> {
+
+  private Cache<KEY,VALUE> canonical;
+  private Cache<KEY,VALUE> onAccess;
+  private BiFunction<KEY,PARAMS,VALUE> loaderFunction;
+
+  public DeduplicationCache(BiFunction<KEY,PARAMS,VALUE> loaderFunction, 
Duration minLifetime) {
+    this.loaderFunction = loaderFunction;
+    this.canonical = Caffeine.newBuilder().weakValues().build();
+    this.onAccess = 
Caffeine.newBuilder().expireAfterAccess(minLifetime).build();
+  }
+
+  public VALUE computeIfAbsent(KEY key, Supplier<PARAMS> params) {
+    var cl = canonical.get(key, k -> loaderFunction.apply(k, params.get()));
+    onAccess.put(key, cl);
+    return cl;
+  }
+
+  public Set<KEY> keySet() {
+    return Set.copyOf(canonical.asMap().keySet());
+  }
+
+  public boolean anyMatch(Predicate<KEY> keyPredicate) {
+    return canonical.asMap().keySet().stream().anyMatch(keyPredicate);
+  }
+
+  public boolean containsKey(KEY key) {
+    return canonical.getIfPresent(key) != null;
+  }
+
+}
diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/ContextDefinition.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/ContextDefinition.java
index d1d180c..9641943 100644
--- 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/ContextDefinition.java
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/ContextDefinition.java
@@ -24,9 +24,10 @@ import static java.util.Objects.requireNonNull;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
-import java.security.NoSuchAlgorithmException;
+import java.util.Collections;
+import java.util.LinkedHashSet;
 import java.util.Objects;
-import java.util.TreeSet;
+import java.util.Set;
 
 import org.apache.accumulo.classloader.lcc.Constants;
 import org.apache.accumulo.classloader.lcc.resolvers.FileResolver;
@@ -34,14 +35,11 @@ import 
org.apache.accumulo.core.spi.common.ContextClassLoaderFactory.ContextClas
 
 import com.google.common.base.Preconditions;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@SuppressFBWarnings(value = {"EI_EXPOSE_REP"})
 public class ContextDefinition {
 
   public static ContextDefinition create(String contextName, int 
monitorIntervalSecs,
       URL... sources) throws ContextClassLoaderException, IOException {
-    TreeSet<Resource> resources = new TreeSet<>();
+    LinkedHashSet<Resource> resources = new LinkedHashSet<>();
     for (URL u : sources) {
       FileResolver resolver = FileResolver.resolve(u);
       try (InputStream is = resolver.getInputStream()) {
@@ -54,13 +52,13 @@ public class ContextDefinition {
 
   private String contextName;
   private volatile int monitorIntervalSeconds;
-  private TreeSet<Resource> resources;
-  private volatile transient byte[] checksum = null;
+  private LinkedHashSet<Resource> resources;
+  private volatile transient String checksum = null;
 
   public ContextDefinition() {}
 
   public ContextDefinition(String contextName, int monitorIntervalSeconds,
-      TreeSet<Resource> resources) {
+      LinkedHashSet<Resource> resources) {
     this.contextName = requireNonNull(contextName, "context name must be 
supplied");
     Preconditions.checkArgument(monitorIntervalSeconds > 0,
         "monitor interval must be greater than zero");
@@ -76,20 +74,8 @@ public class ContextDefinition {
     return monitorIntervalSeconds;
   }
 
-  public TreeSet<Resource> getResources() {
-    return resources;
-  }
-
-  public void setContextName(String contextName) {
-    this.contextName = contextName;
-  }
-
-  public void setMonitorIntervalSeconds(int monitorIntervalSeconds) {
-    this.monitorIntervalSeconds = monitorIntervalSeconds;
-  }
-
-  public void setResources(TreeSet<Resource> resources) {
-    this.resources = resources;
+  public Set<Resource> getResources() {
+    return Collections.unmodifiableSet(resources);
   }
 
   @Override
@@ -114,9 +100,9 @@ public class ContextDefinition {
         && Objects.equals(resources, other.resources);
   }
 
-  public synchronized byte[] getChecksum() throws NoSuchAlgorithmException {
+  public synchronized String getChecksum() {
     if (checksum == null) {
-      checksum = Constants.getChecksummer().digest(toJson());
+      checksum = Constants.getChecksummer().digestAsHex(toJson());
     }
     return checksum;
   }
diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/Resource.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/Resource.java
index 119824e..e4fd69b 100644
--- 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/Resource.java
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/definition/Resource.java
@@ -42,14 +42,6 @@ public class Resource implements Comparable<Resource> {
     return checksum;
   }
 
-  public void setLocation(String location) {
-    this.location = location;
-  }
-
-  public void setChecksum(String checksum) {
-    this.checksum = checksum;
-  }
-
   public URL getURL() throws MalformedURLException {
     return new URL(location);
   }
diff --git 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
index 76f8442..351120f 100644
--- 
a/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
+++ 
b/modules/local-caching-classloader/src/main/java/org/apache/accumulo/classloader/lcc/resolvers/HttpFileResolver.java
@@ -39,7 +39,8 @@ public final class HttpFileResolver extends FileResolver {
   }
 
   @Override
-  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD")
+  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
+      justification = "user-supplied URL is the intended functionality")
   public InputStream getInputStream() throws IOException {
     return getURL().openStream();
   }
diff --git 
a/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactoryTest.java
 
b/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactoryTest.java
index 587de84..aba6c66 100644
--- 
a/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactoryTest.java
+++ 
b/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextClassLoaderFactoryTest.java
@@ -37,9 +37,9 @@ import java.nio.file.StandardCopyOption;
 import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.TreeSet;
 
 import org.apache.accumulo.classloader.lcc.TestUtils.TestClassInfo;
 import org.apache.accumulo.classloader.lcc.definition.ContextDefinition;
@@ -226,8 +226,10 @@ public class LocalCachingContextClassLoaderFactoryTest {
 
     ContextClassLoaderException ex = 
assertThrows(ContextClassLoaderException.class,
         () -> FACTORY.getClassLoader(invalidDefUrl.toString()));
-    assertTrue(ex.getMessage().startsWith(
-        "Error getting classloader for context: 
com.google.gson.stream.MalformedJsonException"));
+    assertTrue(
+        ex.getMessage().startsWith(
+            "Error getting classloader for context: 
com.google.gson.stream.MalformedJsonException"),
+        ex::getMessage);
   }
 
   @Test
@@ -274,17 +276,12 @@ public class LocalCachingContextClassLoaderFactoryTest {
 
   @Test
   public void testInitialBadResourceURL() throws Exception {
-    Resource r = new Resource();
     // remove the file:// prefix from the URL
-    r.setLocation(jarAOrigLocation.toString().substring(6));
-    r.setChecksum("1234");
-    TreeSet<Resource> resources = new TreeSet<>();
+    Resource r = new Resource(jarAOrigLocation.toString().substring(6), 
"1234");
+    LinkedHashSet<Resource> resources = new LinkedHashSet<>();
     resources.add(r);
 
-    ContextDefinition def = new ContextDefinition();
-    def.setContextName("initial");
-    def.setMonitorIntervalSeconds(MONITOR_INTERVAL_SECS);
-    def.setResources(resources);
+    ContextDefinition def = new ContextDefinition("initial", 
MONITOR_INTERVAL_SECS, resources);
 
     final Path initial = createContextDefinitionFile(fs,
         "InitialContextDefinitionBadResourceURL.json", def.toJson());
@@ -292,7 +289,8 @@ public class LocalCachingContextClassLoaderFactoryTest {
 
     ContextClassLoaderException ex = 
assertThrows(ContextClassLoaderException.class,
         () -> FACTORY.getClassLoader(initialDefUrl.toString()));
-    assertTrue(ex.getMessage().startsWith("Error getting classloader for 
context: no protocol"));
+    assertTrue(ex.getMessage().startsWith("Error getting classloader for 
context: no protocol"),
+        ex::getMessage);
     Throwable t = ex.getCause();
     assertTrue(t instanceof MalformedURLException);
     assertTrue(t.getMessage().startsWith("no protocol"));
@@ -300,16 +298,11 @@ public class LocalCachingContextClassLoaderFactoryTest {
 
   @Test
   public void testInitialBadResourceChecksum() throws Exception {
-    Resource r = new Resource();
-    r.setLocation(jarAOrigLocation.toString());
-    r.setChecksum("1234");
-    TreeSet<Resource> resources = new TreeSet<>();
+    Resource r = new Resource(jarAOrigLocation.toString(), "1234");
+    LinkedHashSet<Resource> resources = new LinkedHashSet<>();
     resources.add(r);
 
-    ContextDefinition def = new ContextDefinition();
-    def.setContextName("initial");
-    def.setMonitorIntervalSeconds(MONITOR_INTERVAL_SECS);
-    def.setResources(resources);
+    ContextDefinition def = new ContextDefinition("initial", 
MONITOR_INTERVAL_SECS, resources);
 
     final Path initial = createContextDefinitionFile(fs,
         "InitialContextDefinitionBadResourceChecksum.json", def.toJson());
@@ -384,9 +377,7 @@ public class LocalCachingContextClassLoaderFactoryTest {
 
     assertNotEquals(cl, cl2);
 
-    @SuppressWarnings("unchecked")
-    Class<? extends test.Test> clazz =
-        (Class<? extends test.Test>) cl2.loadClass(classA.getClassName());
+    var clazz = 
cl2.loadClass(classA.getClassName()).asSubclass(test.Test.class);
     test.Test impl = clazz.getDeclaredConstructor().newInstance();
     assertEquals("Hello from E", impl.hello());
     testClassFailsToLoad(cl2, classB);
@@ -486,16 +477,11 @@ public class LocalCachingContextClassLoaderFactoryTest {
     testClassFailsToLoad(cl, classC);
     testClassFailsToLoad(cl, classD);
 
-    Resource r = new Resource();
-    r.setLocation(jarAOrigLocation.toString());
-    r.setChecksum("1234");
-    TreeSet<Resource> resources = new TreeSet<>();
+    Resource r = new Resource(jarAOrigLocation.toString(), "1234");
+    LinkedHashSet<Resource> resources = new LinkedHashSet<>();
     resources.add(r);
 
-    ContextDefinition def2 = new ContextDefinition();
-    def2.setContextName("update");
-    def2.setMonitorIntervalSeconds(MONITOR_INTERVAL_SECS);
-    def2.setResources(resources);
+    ContextDefinition def2 = new ContextDefinition("update", 
MONITOR_INTERVAL_SECS, resources);
 
     updateContextDefinitionFile(fs, defFilePath, def2.toJson());
 
@@ -527,17 +513,12 @@ public class LocalCachingContextClassLoaderFactoryTest {
     testClassFailsToLoad(cl, classC);
     testClassFailsToLoad(cl, classD);
 
-    Resource r = new Resource();
     // remove the file:// prefix from the URL
-    r.setLocation(jarAOrigLocation.toString().substring(6));
-    r.setChecksum("1234");
-    TreeSet<Resource> resources = new TreeSet<>();
+    Resource r = new Resource(jarAOrigLocation.toString().substring(6), 
"1234");
+    LinkedHashSet<Resource> resources = new LinkedHashSet<>();
     resources.add(r);
 
-    ContextDefinition def2 = new ContextDefinition();
-    def2.setContextName("initial");
-    def2.setMonitorIntervalSeconds(MONITOR_INTERVAL_SECS);
-    def2.setResources(resources);
+    ContextDefinition def2 = new ContextDefinition("initial", 
MONITOR_INTERVAL_SECS, resources);
 
     updateContextDefinitionFile(fs, defFilePath, def2.toJson());
 
diff --git 
a/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextTest.java
 
b/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextTest.java
index b79f516..9c12ed5 100644
--- 
a/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextTest.java
+++ 
b/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/LocalCachingContextTest.java
@@ -28,7 +28,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.net.URL;
 import java.nio.file.Files;
-import java.util.TreeSet;
+import java.util.LinkedHashSet;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.classloader.lcc.TestUtils.TestClassInfo;
 import org.apache.accumulo.classloader.lcc.definition.ContextDefinition;
@@ -91,7 +92,7 @@ public class LocalCachingContextTest {
     final URL jarCNewLocation = jetty.getURI().resolve("TestC.jar").toURL();
 
     // Create ContextDefinition with all three resources
-    final TreeSet<Resource> resources = new TreeSet<>();
+    final LinkedHashSet<Resource> resources = new LinkedHashSet<>();
     resources.add(new Resource(jarAOrigLocation.toString(),
         TestUtils.computeResourceChecksum(jarAOrigLocation)));
     resources.add(new Resource(jarBNewLocation.toString(),
@@ -125,11 +126,12 @@ public class LocalCachingContextTest {
     // Confirm the 3 jars are cached locally
     final java.nio.file.Path base = 
java.nio.file.Path.of(tempDir.resolve("base").toUri());
     assertTrue(Files.exists(base));
-    assertTrue(Files.exists(base.resolve(CONTEXT_NAME)));
+    assertTrue(Files.exists(base.resolve(CONTEXT_NAME + "_" + 
def.getChecksum())));
     for (Resource r : def.getResources()) {
       String filename = TestUtils.getFileName(r.getURL());
       String checksum = r.getChecksum();
-      assertTrue(Files.exists(base.resolve(CONTEXT_NAME).resolve(filename + 
"_" + checksum)));
+      assertTrue(Files.exists(
+          base.resolve(CONTEXT_NAME + "_" + 
def.getChecksum()).resolve(filename + "_" + checksum)));
     }
   }
 
@@ -157,9 +159,10 @@ public class LocalCachingContextTest {
     testClassLoads(contextClassLoader, classB);
     testClassLoads(contextClassLoader, classC);
 
-    TreeSet<Resource> updatedResources = new TreeSet<>(def.getResources());
-    assertEquals(3, updatedResources.size());
-    updatedResources.remove(updatedResources.last()); // remove C
+    // keep all but C
+    var updatedResources = 
def.getResources().stream().limit(def.getResources().size() - 1)
+        .collect(Collectors.toCollection(LinkedHashSet::new));
+    assertEquals(def.getResources().size() - 1, updatedResources.size());
 
     // Add D
     final URL jarDOrigLocation =
@@ -170,17 +173,19 @@ public class LocalCachingContextTest {
 
     ContextDefinition updatedDef =
         new ContextDefinition(CONTEXT_NAME, MONITOR_INTERVAL_SECS, 
updatedResources);
-    lcccl.update(updatedDef);
+    lcccl = new LocalCachingContext(baseCacheDir, updatedDef);
+    lcccl.initialize();
 
     // Confirm the 3 jars are cached locally
     final java.nio.file.Path base = 
java.nio.file.Path.of(tempDir.resolve("base").toUri());
     assertTrue(Files.exists(base));
-    assertTrue(Files.exists(base.resolve(CONTEXT_NAME)));
+    assertTrue(Files.exists(base.resolve(CONTEXT_NAME + "_" + 
updatedDef.getChecksum())));
     for (Resource r : updatedDef.getResources()) {
       String filename = TestUtils.getFileName(r.getURL());
       assertFalse(filename.contains("C"));
       String checksum = r.getChecksum();
-      assertTrue(Files.exists(base.resolve(CONTEXT_NAME).resolve(filename + 
"_" + checksum)));
+      assertTrue(Files.exists(base.resolve(CONTEXT_NAME + "_" + 
updatedDef.getChecksum())
+          .resolve(filename + "_" + checksum)));
     }
 
     final ClassLoader updatedContextClassLoader = lcccl.getClassloader();
diff --git 
a/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/TestUtils.java
 
b/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/TestUtils.java
index 65b7c3e..3d994fb 100644
--- 
a/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/TestUtils.java
+++ 
b/modules/local-caching-classloader/src/test/java/org/apache/accumulo/classloader/lcc/TestUtils.java
@@ -99,9 +99,7 @@ public class TestUtils {
   }
 
   public static void testClassLoads(ClassLoader cl, TestClassInfo tci) throws 
Exception {
-    @SuppressWarnings("unchecked")
-    Class<? extends test.Test> clazz =
-        (Class<? extends test.Test>) cl.loadClass(tci.getClassName());
+    var clazz = cl.loadClass(tci.getClassName()).asSubclass(test.Test.class);
     test.Test impl = clazz.getDeclaredConstructor().newInstance();
     assertEquals(tci.getHelloOutput(), impl.hello());
   }
@@ -166,7 +164,8 @@ public class TestUtils {
     return jetty;
   }
 
-  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD")
+  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
+      justification = "user-supplied URL is the intended functionality")
   public static String computeResourceChecksum(URL resourceLocation) throws 
IOException {
     try (InputStream is = resourceLocation.openStream()) {
       return Constants.getChecksummer().digestAsHex(is);
diff --git 
a/modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/VFSManager.java
 
b/modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/VFSManager.java
index e7c00e3..eed96f3 100644
--- 
a/modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/VFSManager.java
+++ 
b/modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/VFSManager.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.vfs2.CacheStrategy;
@@ -50,6 +51,7 @@ public class VFSManager {
     }
   }
 
+  private static final AtomicBoolean initialized = new AtomicBoolean(false);
   private static DefaultFileSystemManager VFS = null;
   private static volatile boolean DEBUG = false;
 
@@ -145,7 +147,7 @@ public class VFSManager {
   }
 
   public static void initialize() throws FileSystemException {
-    if (null == VFS) {
+    if (initialized.compareAndSet(false, true)) {
       VFS = new DefaultFileSystemManager();
       VFS.addProvider("res", new 
org.apache.commons.vfs2.provider.res.ResourceFileProvider());
       VFS.addProvider("zip", new 
org.apache.commons.vfs2.provider.zip.ZipFileProvider());
diff --git a/pom.xml b/pom.xml
index 125cb8f..e18f552 100644
--- a/pom.xml
+++ b/pom.xml
@@ -205,7 +205,7 @@ under the License.
         <plugin>
           <groupId>com.github.spotbugs</groupId>
           <artifactId>spotbugs-maven-plugin</artifactId>
-          <version>4.9.5.0</version>
+          <version>4.9.8.2</version>
           <configuration>
             <xmlOutput>true</xmlOutput>
             <effort>Max</effort>
@@ -750,6 +750,20 @@ under the License.
               <configuration>
                 <lifecycleMappingMetadata>
                   <pluginExecutions>
+                    <pluginExecution>
+                      <pluginExecutionFilter>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-dependency-plugin</artifactId>
+                        <versionRange>[0,)</versionRange>
+                        <goals>
+                          <goal>analyze-only</goal>
+                          <goal>copy</goal>
+                        </goals>
+                      </pluginExecutionFilter>
+                      <action>
+                        <ignore />
+                      </action>
+                    </pluginExecution>
                     <pluginExecution>
                       <pluginExecutionFilter>
                         <groupId>org.codehaus.mojo</groupId>


Reply via email to