kevinrr888 commented on code in PR #28: URL: https://github.com/apache/accumulo-classloaders/pull/28#discussion_r2408004052
########## modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/HDFSContextClassLoaderFactory.java: ########## @@ -0,0 +1,399 @@ +/* + * 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.vfs; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.spi.common.ContextClassLoaderEnvironment; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.gson.Gson; + +/** + * A {@link ContextClassLoaderFactory} implementation which uses a {@link URLClassLoader} per + * defined context. To use this class, need to set the Accumulo configuration property + * <b>general.context.class.loader.factory</b> to the fully qualified name of this class. <br> + * <br> + * Configuration of this class is done by having a base directory in HDFS which stores context + * directories (e.g., "hdfs://test:1234/contexts" could be the base directory) and where the + * children are specific contexts (e.g., contextA, contextB). The "contexts" directory should not + * contain any other files or directories. Each context directory should contain a manifest file + * <b>manifest.json</b> and JAR files. The manifest file defines the context name and the JAR info + * for what JARs should be in the {@link URLClassLoader} for that context. For example: + * + * <pre> + * { + * "context": "contextA", + * "jars": [ + * { + * "name": "Iterators.jar", + * "checksum": "f2ca1bb6c7e907d06dafe4687e579fce76b37e4e93b7605022da52e6ccc26fd2" + * }, + * { + * "name": "IteratorsV2.jar", + * "checksum": "934ee77f70dc82403618c154aa63af6f4ebbe3ac1eaf22df21e7e64f0fb6643d" + * } + * ] + * } + * </pre> + * + * Two system properties need to be set to use this class: <br> + * <b>hdfs.contexts.class.loader.base.dir</b> <br> + * This defines where the context directories exist within HDFS (e.g., "hdfs://test:1234/contexts" + * in our example) <br> + * <b>local.contexts.class.loader.download.dir</b> <br> + * This defines where the context info will be locally downloaded to. Will only download the JARs. + * For example, if set to "/path/to/contexts" and + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} is called with the argument + * "contextA", will create the local directory "/path/to/contexts/contextA" (note that all parent + * directories will be created if needed), will download the JARs Iterators.jar and IteratorsV2.jar + * from HDFS to the contextA directory. The {@link URLClassLoader} will be constructed from these + * local JARs, will be associated with "contextA", and will be cached. <br> + * Another optional system property: <br> + * <b>manifest.file.check.interval.sec</b> <br> + * may be set as well. This defines how often (in sec) the cached contexts will be checked and/or + * updated based on changes to their associated manifest file. If not set, default interval is 10s. + * Changes to this will take effect on next interval. + */ +public class HDFSContextClassLoaderFactory implements ContextClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(HDFSContextClassLoaderFactory.class); + public static final String MANIFEST_FILE_NAME = "manifest.json"; + private static final String HASH_ALG = "SHA-256"; + private static final long DEFAULT_MANIFEST_CHECK_INTERVAL = TimeUnit.SECONDS.toMillis(10); + public static final String HDFS_CONTEXTS_BASE_DIR = "hdfs.contexts.class.loader.base.dir"; + public static final String LOCAL_CONTEXTS_DOWNLOAD_DIR = + "local.contexts.class.loader.download.dir"; + public static final String MANIFEST_FILE_CHECK_INTERVAL = "manifest.file.check.interval.sec"; + // Cache the class loaders for re-use + // WeakReferences are used so that the class loaders can be cleaned up when no longer needed + // Classes that are loaded contain a reference to the class loader used to load them + // so the class loader will be garbage collected when no more classes are loaded that reference it + private final Cache<Path,Pair<String,URLClassLoader>> classLoaders = + CacheBuilder.newBuilder().weakValues().build(); + private FileSystem hdfs; + private Path HDFSContextsDir; + private java.nio.file.Path localContextsDir; + private final Configuration hadoopConf = new Configuration(); + private final Thread manifestFileChecker = new Thread(new ManifestFileChecker()); + private final AtomicBoolean shutdownManifestFileChecker = new AtomicBoolean(false); + + @VisibleForTesting + public class ManifestFileChecker implements Runnable { + @Override + public void run() { + while (!shutdownManifestFileChecker.get()) { + var manifestFileCheckInterval = System.getProperty(MANIFEST_FILE_CHECK_INTERVAL); + long sleepTime = manifestFileCheckInterval == null ? DEFAULT_MANIFEST_CHECK_INTERVAL + : TimeUnit.SECONDS.toMillis(Integer.parseInt(manifestFileCheckInterval)); + + classLoaders.asMap().keySet().forEach(HDFSManifestFile -> classLoaders.asMap() + .computeIfPresent(HDFSManifestFile, (key, existingVal) -> { + var existingChecksum = existingVal.getFirst(); + try (var HDFSManifestFileIn = hdfs.open(HDFSManifestFile)) { + var HDFSManifestFileBytes = HDFSManifestFileIn.readAllBytes(); + var computedChecksum = checksum(HDFSManifestFileBytes); + if (!existingChecksum.equals(computedChecksum)) { + // This manifest file has changed since the class loader for it was computed. + // Need to update the class loader entry. + LOG.debug("HDFS manifest file {} existing checksum {} computed checksum {}", + HDFSManifestFile, existingChecksum, computedChecksum); + try (var tempState = + createTempState(HDFSManifestFile.getParent().getName(), HDFSManifestFile, + new ByteArrayInputStream(HDFSManifestFileBytes), computedChecksum)) { + return createClassLoader(tempState); + } + } + } catch (IOException | ContextClassLoaderException | ExecutionException e) { + LOG.error("Exception occurred in thread {}", Thread.currentThread().getName(), e); + } + return existingVal; + })); + + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + } + } + + /** + * Used to store info needed to create the value in the Cache + */ + private static class TempState implements AutoCloseable { + private final String contextName; + private final java.nio.file.Path tempPath; + private final java.nio.file.Path finalPath; + private final String checksum; + private final URL[] urls; + + private TempState(String contextName, java.nio.file.Path tempPath, java.nio.file.Path finalPath, + String checksum, URL[] urls) { + this.contextName = contextName; + this.tempPath = tempPath; + this.finalPath = finalPath; + this.checksum = checksum; + this.urls = urls; + } + + @Override + public void close() throws IOException { + if (tempPath.toFile().exists()) { + FileUtils.deleteDirectory(tempPath.toFile()); + LOG.debug("Creating the class loader for context {} did not fully complete. Deleted the " + + "temp directory {} and all of its contents.", contextName, tempPath); + } + } + } + + @VisibleForTesting + public void shutdownManifestFileChecker() { + shutdownManifestFileChecker.set(true); + } + + @VisibleForTesting + public static class Context { + private final String contextName; + private final List<JarInfo> jars; + + public Context(String contextName, JarInfo... jars) { + this.contextName = contextName; + this.jars = List.of(jars); + } + + public String getContextName() { + return contextName; + } + + public List<JarInfo> getJars() { + return jars; + } + } + + @VisibleForTesting + public static class JarInfo { + private final String jarName; + private final String checksum; + + public JarInfo(String jarName, String checksum) { + this.jarName = jarName; + this.checksum = checksum; + } + + public String getJarName() { + return jarName; + } + + public String getChecksum() { + return checksum; + } + } + + @Override + public void init(ContextClassLoaderEnvironment env) { + HDFSContextsDir = new Path(System.getProperty(HDFS_CONTEXTS_BASE_DIR)); + localContextsDir = java.nio.file.Path.of(System.getProperty(LOCAL_CONTEXTS_DOWNLOAD_DIR)); + // create the directory to store all the context info, if needed + if (localContextsDir.toFile().mkdirs()) { + LOG.info("Created dir(s) " + localContextsDir); + } + try { + hdfs = FileSystem.get(HDFSContextsDir.toUri(), hadoopConf); + } catch (IOException e) { + throw new IllegalStateException("could not obtain FileSystem for " + HDFSContextsDir.toUri(), + e); + } + manifestFileChecker.start(); + } + + private TempState createTempState(String contextName, Path HDFSManifestFile, + InputStream HDFSManifestFileIn, String HDFSManifestFileChecksum) + throws IOException, ContextClassLoaderException, ExecutionException { + URL[] urls; + + var localFinalContextDir = localContextsDir.resolve(contextName); + // create the temp directory, will eventually be renamed to the above or deleted + var localTempContextDir = + localContextsDir.resolve("tmp-" + contextName + "-" + UUID.randomUUID()); + + if (localTempContextDir.toFile().mkdir()) { + LOG.debug("Created dir " + localTempContextDir); + } else { + throw new IOException("Could not create dir " + localTempContextDir); + } + + // read the context info from the HDFS manifest file and download the jars referenced to the + // temp directory in the local file system + try (var reader = new InputStreamReader(HDFSManifestFileIn, StandardCharsets.UTF_8)) { + Context context = new Gson().fromJson(reader, Context.class); + urls = new URL[context.getJars().size()]; + int i = 0; + if (HDFSManifestFile.getParent() != null) { + for (var jar : context.getJars()) { + // download to temp dir + var HDFSJarPath = new Path(HDFSManifestFile.getParent(), jar.getJarName()); + var localTempJarPath = localTempContextDir.resolve(jar.getJarName()); + hdfs.copyToLocalFile(HDFSJarPath, new Path(localTempJarPath.toUri())); + LOG.info("Copied from {} to {}", HDFSJarPath, localTempJarPath); + // verify downloaded jar checksum matches what is in the manifest file + var computedChecksumLocalJar = checksumLocalFile(localTempJarPath); + if (!computedChecksumLocalJar.equals(jar.getChecksum())) { + throw new ContextClassLoaderException(contextName, + new IllegalStateException(String.format( + "checksum: %s of downloaded jar: %s (downloaded from %s to %s) did not match " + + "checksum present: %s in manifest file: %s. Consider retrying and/or " + + "updating the checksum for this jar in the manifest file", + computedChecksumLocalJar, jar.getJarName(), HDFSJarPath, localTempJarPath, + jar.getChecksum(), HDFSManifestFile))); + } + // use final (non-temp) jar path for the URL as that is what will exist if this op + // completes + var localFinalJarPath = localFinalContextDir.resolve(jar.getJarName()); + urls[i++] = localFinalJarPath.toUri().toURL(); + } + } else { + throw new ContextClassLoaderException(contextName); + } + } + + return new TempState(contextName, localTempContextDir, localFinalContextDir, + HDFSManifestFileChecksum, urls); + } + + private Pair<String,URLClassLoader> createClassLoader(TempState tempState) throws IOException { + // if the final path already exists, we are replacing it with temp path, so recursively delete + // the final path + if (Files.exists(tempState.finalPath)) { + FileUtils.deleteDirectory(tempState.finalPath.toFile()); + } + // rename temp path to final path + if (tempState.tempPath.toFile().renameTo(tempState.finalPath.toFile())) { + LOG.info("Renamed {} to {}", tempState.tempPath, tempState.finalPath); + } + var classLoader = new URLClassLoader(tempState.urls, ClassLoader.getSystemClassLoader()); + return new Pair<>(tempState.checksum, classLoader); + } + + public String checksumLocalFile(java.nio.file.Path path) throws IOException { + try (var in = FileUtils.openInputStream(path.toFile())) { + return checksum(in.readAllBytes()); + } + } + + public static String checksum(byte[] fileBytes) { + try { + StringBuilder checksum = new StringBuilder(); + for (byte b : MessageDigest.getInstance(HASH_ALG).digest(fileBytes)) { + checksum.append(String.format("%02x", b)); + } + return checksum.toString(); + } catch (NoSuchAlgorithmException e) { + throw new IllegalStateException(e); + } + } + + @Override + public ClassLoader getClassLoader(String contextName) throws ContextClassLoaderException { + Path HDFSContextPath = new Path(HDFSContextsDir, contextName); + Path HDFSManifestFile = new Path(HDFSContextPath, MANIFEST_FILE_NAME); + + try (var HDFSManifestFileIn = hdfs.open(HDFSManifestFile)) { + // if another thread has already started creating the class loader for this context: + // wait a short time for it to succeed in creating and caching the classloader, otherwise + // continue and try to create and cache the classloader ourselves (only one result will be + // cached) + try (var allContextDirs = Files.list(localContextsDir)) { + // will match temp directories or final directories for this context name + if (allContextDirs.map(java.nio.file.Path::getFileName) + .anyMatch(path -> path.toString().contains(contextName))) { + var retry = Retry.builder().maxRetries(5).retryAfter(50, TimeUnit.MILLISECONDS) + .incrementBy(50, TimeUnit.MILLISECONDS).maxWait(1, TimeUnit.SECONDS).backOffFactor(2) + .logInterval(500, TimeUnit.MILLISECONDS).createRetry(); + final String operationDesc = + "Waiting for another thread to finish creating/caching the class loader for context " + + contextName; + Pair<String,URLClassLoader> valInCache; + while ((valInCache = classLoaders.getIfPresent(HDFSManifestFile)) == null + && retry.canRetry()) { + retry.useRetry(); + try { + retry.waitForNextAttempt(LOG, operationDesc); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + + if (valInCache != null) { + retry.logCompletion(LOG, operationDesc); + return valInCache.getSecond(); + } else { + LOG.debug( + "Operation '" + operationDesc + "' has not yet succeeded. Attempting ourselves."); + } + } + } + + var HDFSManifestFileBytes = HDFSManifestFileIn.readAllBytes(); Review Comment: Commenting once but applies to all calls to `readAllBytes()`: I think readAllBytes is fine since the files should be pretty small. Thoughts? ########## modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/HDFSContextClassLoaderFactory.java: ########## @@ -0,0 +1,399 @@ +/* + * 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.vfs; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.spi.common.ContextClassLoaderEnvironment; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.gson.Gson; + +/** + * A {@link ContextClassLoaderFactory} implementation which uses a {@link URLClassLoader} per + * defined context. To use this class, need to set the Accumulo configuration property + * <b>general.context.class.loader.factory</b> to the fully qualified name of this class. <br> + * <br> + * Configuration of this class is done by having a base directory in HDFS which stores context + * directories (e.g., "hdfs://test:1234/contexts" could be the base directory) and where the + * children are specific contexts (e.g., contextA, contextB). The "contexts" directory should not + * contain any other files or directories. Each context directory should contain a manifest file + * <b>manifest.json</b> and JAR files. The manifest file defines the context name and the JAR info + * for what JARs should be in the {@link URLClassLoader} for that context. For example: + * + * <pre> + * { + * "context": "contextA", + * "jars": [ + * { + * "name": "Iterators.jar", + * "checksum": "f2ca1bb6c7e907d06dafe4687e579fce76b37e4e93b7605022da52e6ccc26fd2" + * }, + * { + * "name": "IteratorsV2.jar", + * "checksum": "934ee77f70dc82403618c154aa63af6f4ebbe3ac1eaf22df21e7e64f0fb6643d" + * } + * ] + * } + * </pre> + * + * Two system properties need to be set to use this class: <br> + * <b>hdfs.contexts.class.loader.base.dir</b> <br> + * This defines where the context directories exist within HDFS (e.g., "hdfs://test:1234/contexts" + * in our example) <br> + * <b>local.contexts.class.loader.download.dir</b> <br> + * This defines where the context info will be locally downloaded to. Will only download the JARs. + * For example, if set to "/path/to/contexts" and + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} is called with the argument + * "contextA", will create the local directory "/path/to/contexts/contextA" (note that all parent + * directories will be created if needed), will download the JARs Iterators.jar and IteratorsV2.jar + * from HDFS to the contextA directory. The {@link URLClassLoader} will be constructed from these + * local JARs, will be associated with "contextA", and will be cached. <br> + * Another optional system property: <br> + * <b>manifest.file.check.interval.sec</b> <br> + * may be set as well. This defines how often (in sec) the cached contexts will be checked and/or + * updated based on changes to their associated manifest file. If not set, default interval is 10s. + * Changes to this will take effect on next interval. + */ +public class HDFSContextClassLoaderFactory implements ContextClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(HDFSContextClassLoaderFactory.class); + public static final String MANIFEST_FILE_NAME = "manifest.json"; + private static final String HASH_ALG = "SHA-256"; + private static final long DEFAULT_MANIFEST_CHECK_INTERVAL = TimeUnit.SECONDS.toMillis(10); + public static final String HDFS_CONTEXTS_BASE_DIR = "hdfs.contexts.class.loader.base.dir"; + public static final String LOCAL_CONTEXTS_DOWNLOAD_DIR = + "local.contexts.class.loader.download.dir"; + public static final String MANIFEST_FILE_CHECK_INTERVAL = "manifest.file.check.interval.sec"; + // Cache the class loaders for re-use + // WeakReferences are used so that the class loaders can be cleaned up when no longer needed + // Classes that are loaded contain a reference to the class loader used to load them + // so the class loader will be garbage collected when no more classes are loaded that reference it + private final Cache<Path,Pair<String,URLClassLoader>> classLoaders = + CacheBuilder.newBuilder().weakValues().build(); + private FileSystem hdfs; + private Path HDFSContextsDir; + private java.nio.file.Path localContextsDir; + private final Configuration hadoopConf = new Configuration(); + private final Thread manifestFileChecker = new Thread(new ManifestFileChecker()); + private final AtomicBoolean shutdownManifestFileChecker = new AtomicBoolean(false); + + @VisibleForTesting + public class ManifestFileChecker implements Runnable { + @Override + public void run() { + while (!shutdownManifestFileChecker.get()) { + var manifestFileCheckInterval = System.getProperty(MANIFEST_FILE_CHECK_INTERVAL); + long sleepTime = manifestFileCheckInterval == null ? DEFAULT_MANIFEST_CHECK_INTERVAL + : TimeUnit.SECONDS.toMillis(Integer.parseInt(manifestFileCheckInterval)); + + classLoaders.asMap().keySet().forEach(HDFSManifestFile -> classLoaders.asMap() + .computeIfPresent(HDFSManifestFile, (key, existingVal) -> { + var existingChecksum = existingVal.getFirst(); + try (var HDFSManifestFileIn = hdfs.open(HDFSManifestFile)) { + var HDFSManifestFileBytes = HDFSManifestFileIn.readAllBytes(); + var computedChecksum = checksum(HDFSManifestFileBytes); + if (!existingChecksum.equals(computedChecksum)) { + // This manifest file has changed since the class loader for it was computed. + // Need to update the class loader entry. + LOG.debug("HDFS manifest file {} existing checksum {} computed checksum {}", + HDFSManifestFile, existingChecksum, computedChecksum); + try (var tempState = + createTempState(HDFSManifestFile.getParent().getName(), HDFSManifestFile, + new ByteArrayInputStream(HDFSManifestFileBytes), computedChecksum)) { + return createClassLoader(tempState); + } + } + } catch (IOException | ContextClassLoaderException | ExecutionException e) { + LOG.error("Exception occurred in thread {}", Thread.currentThread().getName(), e); + } + return existingVal; + })); + + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + } + } + + /** + * Used to store info needed to create the value in the Cache + */ + private static class TempState implements AutoCloseable { + private final String contextName; + private final java.nio.file.Path tempPath; + private final java.nio.file.Path finalPath; + private final String checksum; + private final URL[] urls; + + private TempState(String contextName, java.nio.file.Path tempPath, java.nio.file.Path finalPath, + String checksum, URL[] urls) { + this.contextName = contextName; + this.tempPath = tempPath; + this.finalPath = finalPath; + this.checksum = checksum; + this.urls = urls; + } + + @Override + public void close() throws IOException { + if (tempPath.toFile().exists()) { + FileUtils.deleteDirectory(tempPath.toFile()); + LOG.debug("Creating the class loader for context {} did not fully complete. Deleted the " + + "temp directory {} and all of its contents.", contextName, tempPath); + } + } + } + + @VisibleForTesting + public void shutdownManifestFileChecker() { + shutdownManifestFileChecker.set(true); + } + + @VisibleForTesting + public static class Context { + private final String contextName; + private final List<JarInfo> jars; + + public Context(String contextName, JarInfo... jars) { + this.contextName = contextName; + this.jars = List.of(jars); + } + + public String getContextName() { + return contextName; + } + + public List<JarInfo> getJars() { + return jars; + } + } + + @VisibleForTesting + public static class JarInfo { + private final String jarName; + private final String checksum; + + public JarInfo(String jarName, String checksum) { + this.jarName = jarName; + this.checksum = checksum; + } + + public String getJarName() { + return jarName; + } + + public String getChecksum() { + return checksum; + } + } + + @Override + public void init(ContextClassLoaderEnvironment env) { + HDFSContextsDir = new Path(System.getProperty(HDFS_CONTEXTS_BASE_DIR)); + localContextsDir = java.nio.file.Path.of(System.getProperty(LOCAL_CONTEXTS_DOWNLOAD_DIR)); + // create the directory to store all the context info, if needed + if (localContextsDir.toFile().mkdirs()) { + LOG.info("Created dir(s) " + localContextsDir); + } + try { + hdfs = FileSystem.get(HDFSContextsDir.toUri(), hadoopConf); + } catch (IOException e) { + throw new IllegalStateException("could not obtain FileSystem for " + HDFSContextsDir.toUri(), + e); + } + manifestFileChecker.start(); + } + + private TempState createTempState(String contextName, Path HDFSManifestFile, + InputStream HDFSManifestFileIn, String HDFSManifestFileChecksum) + throws IOException, ContextClassLoaderException, ExecutionException { + URL[] urls; + + var localFinalContextDir = localContextsDir.resolve(contextName); + // create the temp directory, will eventually be renamed to the above or deleted + var localTempContextDir = + localContextsDir.resolve("tmp-" + contextName + "-" + UUID.randomUUID()); + + if (localTempContextDir.toFile().mkdir()) { + LOG.debug("Created dir " + localTempContextDir); + } else { + throw new IOException("Could not create dir " + localTempContextDir); + } + + // read the context info from the HDFS manifest file and download the jars referenced to the + // temp directory in the local file system + try (var reader = new InputStreamReader(HDFSManifestFileIn, StandardCharsets.UTF_8)) { + Context context = new Gson().fromJson(reader, Context.class); + urls = new URL[context.getJars().size()]; + int i = 0; + if (HDFSManifestFile.getParent() != null) { + for (var jar : context.getJars()) { + // download to temp dir + var HDFSJarPath = new Path(HDFSManifestFile.getParent(), jar.getJarName()); + var localTempJarPath = localTempContextDir.resolve(jar.getJarName()); + hdfs.copyToLocalFile(HDFSJarPath, new Path(localTempJarPath.toUri())); + LOG.info("Copied from {} to {}", HDFSJarPath, localTempJarPath); + // verify downloaded jar checksum matches what is in the manifest file + var computedChecksumLocalJar = checksumLocalFile(localTempJarPath); + if (!computedChecksumLocalJar.equals(jar.getChecksum())) { + throw new ContextClassLoaderException(contextName, + new IllegalStateException(String.format( + "checksum: %s of downloaded jar: %s (downloaded from %s to %s) did not match " + + "checksum present: %s in manifest file: %s. Consider retrying and/or " + + "updating the checksum for this jar in the manifest file", + computedChecksumLocalJar, jar.getJarName(), HDFSJarPath, localTempJarPath, + jar.getChecksum(), HDFSManifestFile))); + } Review Comment: maybe want to retry download a few times in case the file got corrupted in download? (is this possible?) ########## modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/HDFSContextClassLoaderFactory.java: ########## @@ -0,0 +1,399 @@ +/* + * 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.vfs; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.spi.common.ContextClassLoaderEnvironment; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.gson.Gson; + +/** + * A {@link ContextClassLoaderFactory} implementation which uses a {@link URLClassLoader} per + * defined context. To use this class, need to set the Accumulo configuration property + * <b>general.context.class.loader.factory</b> to the fully qualified name of this class. <br> + * <br> + * Configuration of this class is done by having a base directory in HDFS which stores context + * directories (e.g., "hdfs://test:1234/contexts" could be the base directory) and where the + * children are specific contexts (e.g., contextA, contextB). The "contexts" directory should not + * contain any other files or directories. Each context directory should contain a manifest file + * <b>manifest.json</b> and JAR files. The manifest file defines the context name and the JAR info + * for what JARs should be in the {@link URLClassLoader} for that context. For example: + * + * <pre> + * { + * "context": "contextA", + * "jars": [ + * { + * "name": "Iterators.jar", + * "checksum": "f2ca1bb6c7e907d06dafe4687e579fce76b37e4e93b7605022da52e6ccc26fd2" + * }, + * { + * "name": "IteratorsV2.jar", + * "checksum": "934ee77f70dc82403618c154aa63af6f4ebbe3ac1eaf22df21e7e64f0fb6643d" + * } + * ] + * } + * </pre> + * + * Two system properties need to be set to use this class: <br> + * <b>hdfs.contexts.class.loader.base.dir</b> <br> + * This defines where the context directories exist within HDFS (e.g., "hdfs://test:1234/contexts" + * in our example) <br> + * <b>local.contexts.class.loader.download.dir</b> <br> + * This defines where the context info will be locally downloaded to. Will only download the JARs. + * For example, if set to "/path/to/contexts" and + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} is called with the argument + * "contextA", will create the local directory "/path/to/contexts/contextA" (note that all parent + * directories will be created if needed), will download the JARs Iterators.jar and IteratorsV2.jar + * from HDFS to the contextA directory. The {@link URLClassLoader} will be constructed from these + * local JARs, will be associated with "contextA", and will be cached. <br> + * Another optional system property: <br> + * <b>manifest.file.check.interval.sec</b> <br> + * may be set as well. This defines how often (in sec) the cached contexts will be checked and/or + * updated based on changes to their associated manifest file. If not set, default interval is 10s. + * Changes to this will take effect on next interval. + */ +public class HDFSContextClassLoaderFactory implements ContextClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(HDFSContextClassLoaderFactory.class); + public static final String MANIFEST_FILE_NAME = "manifest.json"; + private static final String HASH_ALG = "SHA-256"; Review Comment: do we want the hash alg to be configurable? ########## modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/HDFSContextClassLoaderFactory.java: ########## @@ -0,0 +1,399 @@ +/* + * 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.vfs; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.spi.common.ContextClassLoaderEnvironment; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.gson.Gson; + +/** + * A {@link ContextClassLoaderFactory} implementation which uses a {@link URLClassLoader} per + * defined context. To use this class, need to set the Accumulo configuration property + * <b>general.context.class.loader.factory</b> to the fully qualified name of this class. <br> + * <br> + * Configuration of this class is done by having a base directory in HDFS which stores context + * directories (e.g., "hdfs://test:1234/contexts" could be the base directory) and where the + * children are specific contexts (e.g., contextA, contextB). The "contexts" directory should not + * contain any other files or directories. Each context directory should contain a manifest file + * <b>manifest.json</b> and JAR files. The manifest file defines the context name and the JAR info + * for what JARs should be in the {@link URLClassLoader} for that context. For example: + * + * <pre> + * { + * "context": "contextA", + * "jars": [ + * { + * "name": "Iterators.jar", + * "checksum": "f2ca1bb6c7e907d06dafe4687e579fce76b37e4e93b7605022da52e6ccc26fd2" + * }, + * { + * "name": "IteratorsV2.jar", + * "checksum": "934ee77f70dc82403618c154aa63af6f4ebbe3ac1eaf22df21e7e64f0fb6643d" + * } + * ] + * } + * </pre> + * + * Two system properties need to be set to use this class: <br> + * <b>hdfs.contexts.class.loader.base.dir</b> <br> + * This defines where the context directories exist within HDFS (e.g., "hdfs://test:1234/contexts" + * in our example) <br> + * <b>local.contexts.class.loader.download.dir</b> <br> + * This defines where the context info will be locally downloaded to. Will only download the JARs. + * For example, if set to "/path/to/contexts" and + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} is called with the argument + * "contextA", will create the local directory "/path/to/contexts/contextA" (note that all parent + * directories will be created if needed), will download the JARs Iterators.jar and IteratorsV2.jar + * from HDFS to the contextA directory. The {@link URLClassLoader} will be constructed from these + * local JARs, will be associated with "contextA", and will be cached. <br> + * Another optional system property: <br> + * <b>manifest.file.check.interval.sec</b> <br> + * may be set as well. This defines how often (in sec) the cached contexts will be checked and/or + * updated based on changes to their associated manifest file. If not set, default interval is 10s. + * Changes to this will take effect on next interval. + */ +public class HDFSContextClassLoaderFactory implements ContextClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(HDFSContextClassLoaderFactory.class); + public static final String MANIFEST_FILE_NAME = "manifest.json"; + private static final String HASH_ALG = "SHA-256"; + private static final long DEFAULT_MANIFEST_CHECK_INTERVAL = TimeUnit.SECONDS.toMillis(10); + public static final String HDFS_CONTEXTS_BASE_DIR = "hdfs.contexts.class.loader.base.dir"; + public static final String LOCAL_CONTEXTS_DOWNLOAD_DIR = + "local.contexts.class.loader.download.dir"; + public static final String MANIFEST_FILE_CHECK_INTERVAL = "manifest.file.check.interval.sec"; Review Comment: which is preferred: setting these as system props or setting them as (new) Accumulo properties. For now, have them as system props ########## modules/vfs-class-loader/src/test/java/org/apache/accumulo/classloader/vfs/context/HDFSContextClassLoaderFactoryTest.java: ########## @@ -0,0 +1,319 @@ +/* + * 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.vfs.context; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.File; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import org.apache.accumulo.classloader.vfs.HDFSContextClassLoaderFactory; +import org.apache.accumulo.classloader.vfs.HDFSContextClassLoaderFactory.Context; +import org.apache.accumulo.classloader.vfs.HDFSContextClassLoaderFactory.JarInfo; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.Gson; + +public class HDFSContextClassLoaderFactoryTest { + private static final Logger LOG = + LoggerFactory.getLogger(HDFSContextClassLoaderFactoryTest.class); + private static final String CONTEXT1 = "context1"; + private static final String HELLO_WORLD_JAR = "HelloWorld.jar"; + private static final String HELLO_WORLD_JAR_RESOURCE_NAME = "/" + HELLO_WORLD_JAR; + private static final String HELLO_WORLD_CLASS = "test.HelloWorld"; + private static final String TEST_JAR = "Test.jar"; + private static final String TEST_JAR_RESOURCE_NAME = "/ClassLoaderTestA/" + TEST_JAR; + private static final String TEST_CLASS = "test.TestObject"; + private static final String HDFS_BASE_DIR = "/hdfs"; + private static final String HDFS_CONTEXTS_DIR = HDFS_BASE_DIR + "/contexts"; + private static final String HDFS_CONTEXT1_DIR = HDFS_CONTEXTS_DIR + "/" + CONTEXT1; + private static final String HDFS_MANIFEST_FILE = + HDFS_CONTEXT1_DIR + "/" + HDFSContextClassLoaderFactory.MANIFEST_FILE_NAME; + private static final long CHECK_INTERVAL_SEC = 5; + final String localContextsDir = System.getProperty("user.dir") + "/target/contexts"; + private MiniDFSCluster hdfsCluster; + private FileSystem hdfs; + + @BeforeEach + public void setup() throws Exception { + Configuration hadoopConf = new Configuration(); + hdfsCluster = new MiniDFSCluster.Builder(hadoopConf).build(); + hdfsCluster.waitClusterUp(); + hdfs = hdfsCluster.getFileSystem(); + + Path HDFSContext1Path = new Path(HDFS_CONTEXT1_DIR); + hdfs.mkdirs(HDFSContext1Path, FsPermission.getDirDefault()); + LOG.debug("Created dir(s) " + HDFSContext1Path); + + System.setProperty(HDFSContextClassLoaderFactory.HDFS_CONTEXTS_BASE_DIR, + hdfs.getUri() + HDFS_CONTEXTS_DIR); + System.setProperty(HDFSContextClassLoaderFactory.LOCAL_CONTEXTS_DOWNLOAD_DIR, localContextsDir); + System.setProperty(HDFSContextClassLoaderFactory.MANIFEST_FILE_CHECK_INTERVAL, + CHECK_INTERVAL_SEC + ""); + } + + @AfterEach + public void teardown() throws Exception { + hdfsCluster.close(); + hdfsCluster.shutdown(true); + FileUtils.deleteDirectory( + new File(System.getProperty(HDFSContextClassLoaderFactory.LOCAL_CONTEXTS_DOWNLOAD_DIR))); + } + + private void writeManifestFileToHDFS(Context context) throws Exception { + Path HDFSManifestPath = new Path(HDFS_MANIFEST_FILE); + try (var os = hdfs.create(HDFSManifestPath)) { + Gson gson = new Gson().newBuilder().setPrettyPrinting().create(); + os.write(gson.toJson(context).getBytes(StandardCharsets.UTF_8)); + LOG.debug("Wrote {}{}{} to {}", System.lineSeparator(), gson.toJson(context), + System.lineSeparator(), HDFSManifestPath); + } + } + + private void writeJarFileToHDFS(Path HDFSJarPath, String resourceName) throws Exception { + Path localJar = new Path(this.getClass().getResource(resourceName).toURI()); + hdfs.copyFromLocalFile(localJar, HDFSJarPath); + LOG.debug("Copied from {} to {}", localJar, HDFSJarPath); + } + + private void writeManfiestAndJarToHDFS(String jarName, String resourceName, String contextName, + boolean validChecksum) throws Exception { + var HDFSJarPath = new Path(HDFS_CONTEXT1_DIR, jarName); + writeJarFileToHDFS(HDFSJarPath, resourceName); + + final JarInfo jarInfo; + if (validChecksum) { + jarInfo = new JarInfo(jarName, + HDFSContextClassLoaderFactory.checksum(hdfs.open(HDFSJarPath).readAllBytes())); + } else { + jarInfo = new JarInfo(jarName, "badchecksum"); + } + final Context context = new Context(contextName, jarInfo); + + writeManifestFileToHDFS(context); + } + + @Test + public void testBasic() throws Exception { + HDFSContextClassLoaderFactory factory = new HDFSContextClassLoaderFactory(); + factory.init(null); + + writeManfiestAndJarToHDFS(HELLO_WORLD_JAR, HELLO_WORLD_JAR_RESOURCE_NAME, CONTEXT1, true); + + var classLoader = factory.getClassLoader(CONTEXT1); + var clazz = classLoader.loadClass(HELLO_WORLD_CLASS); + var methods = clazz.getMethods(); + assertEquals(1, + Arrays.stream(methods).filter(method -> method.getName().equals("validate")).count()); + } + + @Test + public void testChangingManifest() throws Exception { + HDFSContextClassLoaderFactory factory = new HDFSContextClassLoaderFactory(); + factory.init(null); + + writeManfiestAndJarToHDFS(HELLO_WORLD_JAR, HELLO_WORLD_JAR_RESOURCE_NAME, CONTEXT1, true); + + var classLoaderA = factory.getClassLoader(CONTEXT1); + assertThrows(ClassNotFoundException.class, () -> classLoaderA.loadClass(TEST_CLASS)); + var clazz = classLoaderA.loadClass(HELLO_WORLD_CLASS); + var methods = clazz.getMethods(); + assertEquals(1, + Arrays.stream(methods).filter(method -> method.getName().equals("validate")).count()); + + // note that we are writing the manifest file with the same context name but without the + // hello world jar but with the test jar + writeManfiestAndJarToHDFS(TEST_JAR, TEST_JAR_RESOURCE_NAME, CONTEXT1, true); + + // wait for manifest file check to take place + Thread.sleep(TimeUnit.SECONDS.toMillis(CHECK_INTERVAL_SEC + 1)); + + var classLoaderB = factory.getClassLoader(CONTEXT1); + assertThrows(ClassNotFoundException.class, () -> classLoaderB.loadClass(HELLO_WORLD_CLASS)); + clazz = classLoaderB.loadClass(TEST_CLASS); + methods = clazz.getMethods(); + assertEquals(1, + Arrays.stream(methods).filter(method -> method.getName().equals("add")).count()); + } + + @Test + public void testIncorrectChecksum() throws Exception { + HDFSContextClassLoaderFactory factory = new HDFSContextClassLoaderFactory(); + factory.init(null); + + writeManfiestAndJarToHDFS(HELLO_WORLD_JAR, HELLO_WORLD_JAR_RESOURCE_NAME, CONTEXT1, false); + + assertThrows(ContextClassLoaderFactory.ContextClassLoaderException.class, + () -> factory.getClassLoader(CONTEXT1)); + } + + /** + * Tests many threads running calls to + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} and many threads running the + * {@link HDFSContextClassLoaderFactory.ManifestFileChecker}. This is to test that there are no + * race conditions for shared resources of this factory (the file systems and the internal store + * for the class loaders). In the implementation, just one + * {@link HDFSContextClassLoaderFactory.ManifestFileChecker} runs, but this tests many of these + * running to more thoroughly test for race conditions. This test will also change the manifest + * file for the context to reference a new jar, which should be picked up by a checker and cause + * the context entry in the internal store to be updated. + */ + @Test + public void testConcurrent() throws Exception { + // have the manifest file checks run more often + final int manifestFileCheckIntervalSec = 1; + System.setProperty(HDFSContextClassLoaderFactory.MANIFEST_FILE_CHECK_INTERVAL, + manifestFileCheckIntervalSec + ""); + final int numTotalThreads = 128; + final int numClassLoaderThreads = numTotalThreads / 2; + final int numManifestFileCheckerThreads = numTotalThreads / 2; + ExecutorService classLoaderPool = Executors.newFixedThreadPool(numClassLoaderThreads); + ExecutorService manifestFileCheckerPool = + Executors.newFixedThreadPool(numManifestFileCheckerThreads); + + try { + CountDownLatch latch = new CountDownLatch(numTotalThreads); + List<Future<ClassLoader>> classLoaderFutures = new ArrayList<>(numClassLoaderThreads); + List<Future<?>> manifestFileCheckerFutures = new ArrayList<>(numManifestFileCheckerThreads); + Set<ClassLoader> classLoaderPoolResults = new HashSet<>(); + HDFSContextClassLoaderFactory factory = new HDFSContextClassLoaderFactory(); + + factory.init(null); + + // context1 -> HelloWorld jar + writeManfiestAndJarToHDFS(HELLO_WORLD_JAR, HELLO_WORLD_JAR_RESOURCE_NAME, CONTEXT1, true); + + for (int i = 0; i < numClassLoaderThreads; i++) { + classLoaderFutures.add(classLoaderPool.submit(() -> { + try { + latch.countDown(); + latch.await(); + // make ~1/2 the threads race to call getClassLoader immediately (racing to get the + // class loader for the HelloWorld jar), make the other ~1/2 race to call after + // the manifest file change occurs and would be picked up (racing to get the class + // loader for the Test jar) + if (new Random().nextInt(2) == 0) { + Thread.sleep(TimeUnit.SECONDS.toMillis(manifestFileCheckIntervalSec + 1)); + } + var classLoader = factory.getClassLoader(CONTEXT1); + return classLoader; + } catch (ContextClassLoaderFactory.ContextClassLoaderException e) { + throw new RuntimeException(e); + } + })); + } + + for (int i = 0; i < numManifestFileCheckerThreads; i++) { + manifestFileCheckerFutures.add(manifestFileCheckerPool.submit(() -> { + latch.countDown(); + try { + latch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + factory.new ManifestFileChecker().run(); + })); + } + + // context1 -> Test jar + writeManfiestAndJarToHDFS(TEST_JAR, TEST_JAR_RESOURCE_NAME, CONTEXT1, true); + + for (var future : classLoaderFutures) { + classLoaderPoolResults.add(future.get()); + } + factory.shutdownManifestFileChecker(); + for (var future : manifestFileCheckerFutures) { + future.get(); + } + + // only two unique class loaders should have been created + assertEquals(2, classLoaderPoolResults.size()); Review Comment: **BUG!** this is occasionally failing with >2 classloaders... Have not been able to figure out why. Will continue to try to figure this out... ########## modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/HDFSContextClassLoaderFactory.java: ########## @@ -0,0 +1,399 @@ +/* + * 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.vfs; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.spi.common.ContextClassLoaderEnvironment; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.gson.Gson; + +/** + * A {@link ContextClassLoaderFactory} implementation which uses a {@link URLClassLoader} per + * defined context. To use this class, need to set the Accumulo configuration property + * <b>general.context.class.loader.factory</b> to the fully qualified name of this class. <br> + * <br> + * Configuration of this class is done by having a base directory in HDFS which stores context + * directories (e.g., "hdfs://test:1234/contexts" could be the base directory) and where the + * children are specific contexts (e.g., contextA, contextB). The "contexts" directory should not + * contain any other files or directories. Each context directory should contain a manifest file + * <b>manifest.json</b> and JAR files. The manifest file defines the context name and the JAR info + * for what JARs should be in the {@link URLClassLoader} for that context. For example: + * + * <pre> + * { + * "context": "contextA", + * "jars": [ + * { + * "name": "Iterators.jar", + * "checksum": "f2ca1bb6c7e907d06dafe4687e579fce76b37e4e93b7605022da52e6ccc26fd2" + * }, + * { + * "name": "IteratorsV2.jar", + * "checksum": "934ee77f70dc82403618c154aa63af6f4ebbe3ac1eaf22df21e7e64f0fb6643d" + * } + * ] + * } + * </pre> + * + * Two system properties need to be set to use this class: <br> + * <b>hdfs.contexts.class.loader.base.dir</b> <br> + * This defines where the context directories exist within HDFS (e.g., "hdfs://test:1234/contexts" + * in our example) <br> + * <b>local.contexts.class.loader.download.dir</b> <br> + * This defines where the context info will be locally downloaded to. Will only download the JARs. + * For example, if set to "/path/to/contexts" and + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} is called with the argument + * "contextA", will create the local directory "/path/to/contexts/contextA" (note that all parent + * directories will be created if needed), will download the JARs Iterators.jar and IteratorsV2.jar + * from HDFS to the contextA directory. The {@link URLClassLoader} will be constructed from these + * local JARs, will be associated with "contextA", and will be cached. <br> + * Another optional system property: <br> + * <b>manifest.file.check.interval.sec</b> <br> + * may be set as well. This defines how often (in sec) the cached contexts will be checked and/or + * updated based on changes to their associated manifest file. If not set, default interval is 10s. + * Changes to this will take effect on next interval. + */ +public class HDFSContextClassLoaderFactory implements ContextClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(HDFSContextClassLoaderFactory.class); + public static final String MANIFEST_FILE_NAME = "manifest.json"; + private static final String HASH_ALG = "SHA-256"; + private static final long DEFAULT_MANIFEST_CHECK_INTERVAL = TimeUnit.SECONDS.toMillis(10); + public static final String HDFS_CONTEXTS_BASE_DIR = "hdfs.contexts.class.loader.base.dir"; + public static final String LOCAL_CONTEXTS_DOWNLOAD_DIR = + "local.contexts.class.loader.download.dir"; + public static final String MANIFEST_FILE_CHECK_INTERVAL = "manifest.file.check.interval.sec"; + // Cache the class loaders for re-use + // WeakReferences are used so that the class loaders can be cleaned up when no longer needed + // Classes that are loaded contain a reference to the class loader used to load them + // so the class loader will be garbage collected when no more classes are loaded that reference it + private final Cache<Path,Pair<String,URLClassLoader>> classLoaders = + CacheBuilder.newBuilder().weakValues().build(); + private FileSystem hdfs; + private Path HDFSContextsDir; + private java.nio.file.Path localContextsDir; + private final Configuration hadoopConf = new Configuration(); + private final Thread manifestFileChecker = new Thread(new ManifestFileChecker()); + private final AtomicBoolean shutdownManifestFileChecker = new AtomicBoolean(false); + + @VisibleForTesting + public class ManifestFileChecker implements Runnable { + @Override + public void run() { + while (!shutdownManifestFileChecker.get()) { + var manifestFileCheckInterval = System.getProperty(MANIFEST_FILE_CHECK_INTERVAL); + long sleepTime = manifestFileCheckInterval == null ? DEFAULT_MANIFEST_CHECK_INTERVAL + : TimeUnit.SECONDS.toMillis(Integer.parseInt(manifestFileCheckInterval)); + + classLoaders.asMap().keySet().forEach(HDFSManifestFile -> classLoaders.asMap() + .computeIfPresent(HDFSManifestFile, (key, existingVal) -> { + var existingChecksum = existingVal.getFirst(); + try (var HDFSManifestFileIn = hdfs.open(HDFSManifestFile)) { + var HDFSManifestFileBytes = HDFSManifestFileIn.readAllBytes(); + var computedChecksum = checksum(HDFSManifestFileBytes); + if (!existingChecksum.equals(computedChecksum)) { + // This manifest file has changed since the class loader for it was computed. + // Need to update the class loader entry. + LOG.debug("HDFS manifest file {} existing checksum {} computed checksum {}", + HDFSManifestFile, existingChecksum, computedChecksum); + try (var tempState = + createTempState(HDFSManifestFile.getParent().getName(), HDFSManifestFile, + new ByteArrayInputStream(HDFSManifestFileBytes), computedChecksum)) { + return createClassLoader(tempState); + } + } + } catch (IOException | ContextClassLoaderException | ExecutionException e) { + LOG.error("Exception occurred in thread {}", Thread.currentThread().getName(), e); + } + return existingVal; + })); + + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + } + } + + /** + * Used to store info needed to create the value in the Cache + */ + private static class TempState implements AutoCloseable { + private final String contextName; + private final java.nio.file.Path tempPath; + private final java.nio.file.Path finalPath; + private final String checksum; + private final URL[] urls; + + private TempState(String contextName, java.nio.file.Path tempPath, java.nio.file.Path finalPath, + String checksum, URL[] urls) { + this.contextName = contextName; + this.tempPath = tempPath; + this.finalPath = finalPath; + this.checksum = checksum; + this.urls = urls; + } + + @Override + public void close() throws IOException { + if (tempPath.toFile().exists()) { + FileUtils.deleteDirectory(tempPath.toFile()); + LOG.debug("Creating the class loader for context {} did not fully complete. Deleted the " + + "temp directory {} and all of its contents.", contextName, tempPath); + } + } + } + + @VisibleForTesting + public void shutdownManifestFileChecker() { + shutdownManifestFileChecker.set(true); + } + + @VisibleForTesting + public static class Context { + private final String contextName; + private final List<JarInfo> jars; + + public Context(String contextName, JarInfo... jars) { + this.contextName = contextName; + this.jars = List.of(jars); + } + + public String getContextName() { + return contextName; + } + + public List<JarInfo> getJars() { + return jars; + } + } + + @VisibleForTesting + public static class JarInfo { + private final String jarName; + private final String checksum; + + public JarInfo(String jarName, String checksum) { + this.jarName = jarName; + this.checksum = checksum; + } + + public String getJarName() { + return jarName; + } + + public String getChecksum() { + return checksum; + } + } + + @Override + public void init(ContextClassLoaderEnvironment env) { + HDFSContextsDir = new Path(System.getProperty(HDFS_CONTEXTS_BASE_DIR)); + localContextsDir = java.nio.file.Path.of(System.getProperty(LOCAL_CONTEXTS_DOWNLOAD_DIR)); + // create the directory to store all the context info, if needed + if (localContextsDir.toFile().mkdirs()) { + LOG.info("Created dir(s) " + localContextsDir); + } + try { + hdfs = FileSystem.get(HDFSContextsDir.toUri(), hadoopConf); + } catch (IOException e) { + throw new IllegalStateException("could not obtain FileSystem for " + HDFSContextsDir.toUri(), + e); + } + manifestFileChecker.start(); + } Review Comment: currently not using env... Will change if we want to use accumulo props instead of system props ########## modules/vfs-class-loader/src/main/java/org/apache/accumulo/classloader/vfs/HDFSContextClassLoaderFactory.java: ########## @@ -0,0 +1,399 @@ +/* + * 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.vfs; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.accumulo.core.spi.common.ContextClassLoaderEnvironment; +import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.util.Retry; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.gson.Gson; + +/** + * A {@link ContextClassLoaderFactory} implementation which uses a {@link URLClassLoader} per + * defined context. To use this class, need to set the Accumulo configuration property + * <b>general.context.class.loader.factory</b> to the fully qualified name of this class. <br> + * <br> + * Configuration of this class is done by having a base directory in HDFS which stores context + * directories (e.g., "hdfs://test:1234/contexts" could be the base directory) and where the + * children are specific contexts (e.g., contextA, contextB). The "contexts" directory should not + * contain any other files or directories. Each context directory should contain a manifest file + * <b>manifest.json</b> and JAR files. The manifest file defines the context name and the JAR info + * for what JARs should be in the {@link URLClassLoader} for that context. For example: + * + * <pre> + * { + * "context": "contextA", + * "jars": [ + * { + * "name": "Iterators.jar", + * "checksum": "f2ca1bb6c7e907d06dafe4687e579fce76b37e4e93b7605022da52e6ccc26fd2" + * }, + * { + * "name": "IteratorsV2.jar", + * "checksum": "934ee77f70dc82403618c154aa63af6f4ebbe3ac1eaf22df21e7e64f0fb6643d" + * } + * ] + * } + * </pre> + * + * Two system properties need to be set to use this class: <br> + * <b>hdfs.contexts.class.loader.base.dir</b> <br> + * This defines where the context directories exist within HDFS (e.g., "hdfs://test:1234/contexts" + * in our example) <br> + * <b>local.contexts.class.loader.download.dir</b> <br> + * This defines where the context info will be locally downloaded to. Will only download the JARs. + * For example, if set to "/path/to/contexts" and + * {@link HDFSContextClassLoaderFactory#getClassLoader(String)} is called with the argument + * "contextA", will create the local directory "/path/to/contexts/contextA" (note that all parent + * directories will be created if needed), will download the JARs Iterators.jar and IteratorsV2.jar + * from HDFS to the contextA directory. The {@link URLClassLoader} will be constructed from these + * local JARs, will be associated with "contextA", and will be cached. <br> + * Another optional system property: <br> + * <b>manifest.file.check.interval.sec</b> <br> + * may be set as well. This defines how often (in sec) the cached contexts will be checked and/or + * updated based on changes to their associated manifest file. If not set, default interval is 10s. + * Changes to this will take effect on next interval. + */ +public class HDFSContextClassLoaderFactory implements ContextClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(HDFSContextClassLoaderFactory.class); + public static final String MANIFEST_FILE_NAME = "manifest.json"; + private static final String HASH_ALG = "SHA-256"; + private static final long DEFAULT_MANIFEST_CHECK_INTERVAL = TimeUnit.SECONDS.toMillis(10); + public static final String HDFS_CONTEXTS_BASE_DIR = "hdfs.contexts.class.loader.base.dir"; + public static final String LOCAL_CONTEXTS_DOWNLOAD_DIR = + "local.contexts.class.loader.download.dir"; + public static final String MANIFEST_FILE_CHECK_INTERVAL = "manifest.file.check.interval.sec"; + // Cache the class loaders for re-use + // WeakReferences are used so that the class loaders can be cleaned up when no longer needed + // Classes that are loaded contain a reference to the class loader used to load them + // so the class loader will be garbage collected when no more classes are loaded that reference it + private final Cache<Path,Pair<String,URLClassLoader>> classLoaders = + CacheBuilder.newBuilder().weakValues().build(); + private FileSystem hdfs; + private Path HDFSContextsDir; + private java.nio.file.Path localContextsDir; + private final Configuration hadoopConf = new Configuration(); + private final Thread manifestFileChecker = new Thread(new ManifestFileChecker()); + private final AtomicBoolean shutdownManifestFileChecker = new AtomicBoolean(false); + + @VisibleForTesting + public class ManifestFileChecker implements Runnable { + @Override + public void run() { + while (!shutdownManifestFileChecker.get()) { + var manifestFileCheckInterval = System.getProperty(MANIFEST_FILE_CHECK_INTERVAL); + long sleepTime = manifestFileCheckInterval == null ? DEFAULT_MANIFEST_CHECK_INTERVAL + : TimeUnit.SECONDS.toMillis(Integer.parseInt(manifestFileCheckInterval)); + + classLoaders.asMap().keySet().forEach(HDFSManifestFile -> classLoaders.asMap() + .computeIfPresent(HDFSManifestFile, (key, existingVal) -> { + var existingChecksum = existingVal.getFirst(); + try (var HDFSManifestFileIn = hdfs.open(HDFSManifestFile)) { + var HDFSManifestFileBytes = HDFSManifestFileIn.readAllBytes(); + var computedChecksum = checksum(HDFSManifestFileBytes); + if (!existingChecksum.equals(computedChecksum)) { + // This manifest file has changed since the class loader for it was computed. + // Need to update the class loader entry. + LOG.debug("HDFS manifest file {} existing checksum {} computed checksum {}", + HDFSManifestFile, existingChecksum, computedChecksum); + try (var tempState = + createTempState(HDFSManifestFile.getParent().getName(), HDFSManifestFile, + new ByteArrayInputStream(HDFSManifestFileBytes), computedChecksum)) { + return createClassLoader(tempState); + } + } + } catch (IOException | ContextClassLoaderException | ExecutionException e) { + LOG.error("Exception occurred in thread {}", Thread.currentThread().getName(), e); + } + return existingVal; + })); + + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + } + } + + /** + * Used to store info needed to create the value in the Cache + */ + private static class TempState implements AutoCloseable { Review Comment: Not a great name -- 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]
