From 09c089898707e76b20f417fed58d27c0cabb0b95 Mon Sep 17 00:00:00 2001 From: George Wu Date: Tue, 20 Aug 2024 15:50:18 -0400 Subject: [PATCH 01/20] update docs for kafka lookup extension to specify correct extension ordering --- docs/querying/kafka-extraction-namespace.md | 3 +++ docs/querying/lookups-cached-global.md | 3 +++ 2 files changed, 6 insertions(+) diff --git a/docs/querying/kafka-extraction-namespace.md b/docs/querying/kafka-extraction-namespace.md index 1cfa91aac554..b8fdbf9c9b87 100644 --- a/docs/querying/kafka-extraction-namespace.md +++ b/docs/querying/kafka-extraction-namespace.md @@ -24,6 +24,9 @@ title: "Apache Kafka Lookups" To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` in the extensions load list. +This extension provides all the functionality of `druid-lookups-cached-global`. You should only load one of the two extensions. +(load `druid-lookups-cached-global` if you only want global cached lookup functionality, load `druid-kafka-extraction-namespace` if you also want kafka lookup functionality) + If you need updates to populate as promptly as possible, it is possible to plug into a Kafka topic whose key is the old value and message is the desired new value (both in UTF-8) as a LookupExtractorFactory. ```json diff --git a/docs/querying/lookups-cached-global.md b/docs/querying/lookups-cached-global.md index 72c4189c2dad..8db1a4333d61 100644 --- a/docs/querying/lookups-cached-global.md +++ b/docs/querying/lookups-cached-global.md @@ -24,6 +24,9 @@ title: "Globally Cached Lookups" To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` in the extensions load list. +The `druid-kafka-extraction-namespace` extension provides all the functionality of `druid-lookups-cached-global`. You should only load one of the two extensions. +(load `druid-lookups-cached-global` if you only want global cached lookup functionality, load `druid-kafka-extraction-namespace` if you also want kafka lookup functionality) + ## Configuration :::info Static configuration is no longer supported. Lookups can be configured through From cfa23fa5d4e4541ceaf65d7b687416d743d98538 Mon Sep 17 00:00:00 2001 From: George Wu Date: Tue, 20 Aug 2024 15:51:35 -0400 Subject: [PATCH 02/20] fix first line --- docs/querying/kafka-extraction-namespace.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/querying/kafka-extraction-namespace.md b/docs/querying/kafka-extraction-namespace.md index b8fdbf9c9b87..e7d02babdd66 100644 --- a/docs/querying/kafka-extraction-namespace.md +++ b/docs/querying/kafka-extraction-namespace.md @@ -22,7 +22,7 @@ title: "Apache Kafka Lookups" ~ under the License. --> -To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` in the extensions load list. +To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-kafka-extraction-namespace` in the extensions load list. This extension provides all the functionality of `druid-lookups-cached-global`. You should only load one of the two extensions. (load `druid-lookups-cached-global` if you only want global cached lookup functionality, load `druid-kafka-extraction-namespace` if you also want kafka lookup functionality) From 2ccaf611efdbf5e3256a0862671664c524ebb6d4 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 22 Aug 2024 12:39:50 -0400 Subject: [PATCH 03/20] test with extension dependencies --- .../kafka-extraction-namespace/pom.xml | 1 + .../guice/ExtensionFirstClassLoader.java | 56 +++++++++++++++++-- .../apache/druid/guice/ExtensionsLoader.java | 36 +++++++++++- 3 files changed, 84 insertions(+), 9 deletions(-) diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index d3cf5d3216fd..a23bd115da8f 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -44,6 +44,7 @@ org.apache.druid.extensions druid-lookups-cached-global ${project.parent.version} + provided org.apache.druid diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java index 1a2944b2bdb0..0ffc8fdbeb75 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java @@ -35,11 +35,13 @@ public class ExtensionFirstClassLoader extends URLClassLoader { private final ClassLoader druidLoader; + private List extensionDependencyClassLoaders; - public ExtensionFirstClassLoader(final URL[] urls, final ClassLoader druidLoader) + public ExtensionFirstClassLoader(final URL[] urls, final ClassLoader druidLoader, final List extensionDependencyClassLoaders) { super(urls, null); this.druidLoader = Preconditions.checkNotNull(druidLoader, "druidLoader"); + this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders"); } @Override @@ -60,8 +62,13 @@ protected Class loadClass(final String name, final boolean resolve) throws Cl clazz = findClass(name); } catch (ClassNotFoundException e) { - // Try the Druid classloader. Will throw ClassNotFoundException if the class can't be loaded. - return druidLoader.loadClass(name); + try { + return loadClassFromExtensionDependencies(name); + } + catch (ClassNotFoundException e2) { + // Try the Druid classloader. Will throw ClassNotFoundException if the class can't be loaded. + return druidLoader.loadClass(name); + } } } @@ -73,24 +80,61 @@ protected Class loadClass(final String name, final boolean resolve) throws Cl } } + protected Class loadClassFromExtensionDependencies(final String name) throws ClassNotFoundException + { + for (ClassLoader classLoader : extensionDependencyClassLoaders) { + try { + return classLoader.loadClass(name); + } + catch (ClassNotFoundException ignored) { + } + } + throw new ClassNotFoundException(); + } + + @Override public URL getResource(final String name) { - final URL resourceFromExtension = super.getResource(name); + URL resourceFromExtension = super.getResource(name); + + if (resourceFromExtension != null) { + return resourceFromExtension; + } + resourceFromExtension = getResourceFromExtensionsDependencies(name); if (resourceFromExtension != null) { return resourceFromExtension; - } else { - return druidLoader.getResource(name); } + + return druidLoader.getResource(name); + } + protected URL getResourceFromExtensionsDependencies(final String name) + { + URL resourceFromExtension = null; + for (ClassLoader classLoader : extensionDependencyClassLoaders) { + resourceFromExtension = classLoader.getResource(name); + if (resourceFromExtension != null) { + break; + } + } + return resourceFromExtension; } + @Override public Enumeration getResources(final String name) throws IOException { final List urls = new ArrayList<>(); Iterators.addAll(urls, Iterators.forEnumeration(super.getResources(name))); Iterators.addAll(urls, Iterators.forEnumeration(druidLoader.getResources(name))); + for (ClassLoader classLoader : extensionDependencyClassLoaders) { + Iterators.addAll(urls, Iterators.forEnumeration(classLoader.getResources(name))); + } return Iterators.asEnumeration(urls.iterator()); } + + public void setExtensionDependencyClassLoaders(List extensionDependencyClassLoaders) { + this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders"); + } } diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 0bdbddfa5a3f..1d30fcdf7ec9 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -20,6 +20,7 @@ package org.apache.druid.guice; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import org.apache.commons.io.FileUtils; import org.apache.druid.initialization.DruidModule; @@ -224,7 +225,7 @@ private static URLClassLoader makeClassLoaderForExtension( } if (useExtensionClassloaderFirst) { - return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader()); + return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader(), ImmutableList.of()); } else { return new URLClassLoader(urls, ExtensionsLoader.class.getClassLoader()); } @@ -278,6 +279,7 @@ private ServiceLoadingFromExtensions(Class serviceClass) if (extensionsConfig.searchCurrentClassloader()) { addAllFromCurrentClassLoader(); } + addAllFromFileSystem(); } @@ -291,13 +293,41 @@ private void addAllFromCurrentClassLoader() private void addAllFromFileSystem() { for (File extension : getExtensionFilesToLoad()) { - log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { - final URLClassLoader loader = getClassLoaderForExtension( + getClassLoaderForExtension( extension, extensionsConfig.isUseExtensionClassloaderFirst() ); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + ExtensionFirstClassLoader kafkaLoader = null; + ExtensionFirstClassLoader globalLoader = null; + for (Pair extension : loaders.keySet()) { + String extensionName = extension.lhs.getName(); + log.info("Extension name %s", extensionName); + if (extensionName.equals("druid-kafka-extraction-namespace")) { + kafkaLoader = (ExtensionFirstClassLoader) loaders.get(extension); + } + + if (extensionName.equals("druid-lookups-cached-global")) { + globalLoader = (ExtensionFirstClassLoader) loaders.get(extension); + } + } + + if (kafkaLoader != null && globalLoader != null) { + log.info("Setting extension dependency %s %s", kafkaLoader.getURLs(), globalLoader.getURLs()); + kafkaLoader.setExtensionDependencyClassLoaders(ImmutableList.of(globalLoader)); + } + for (File extension : getExtensionFilesToLoad()) { + try { + URLClassLoader loader = getClassLoaderForExtension( + extension, + extensionsConfig.isUseExtensionClassloaderFirst() + ); log.info( "Loading extension [%s], jars: %s", extension.getName(), From 5980b7056f4b6c63317ce017f4deecdb455f3e00 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 26 Aug 2024 09:30:58 -0400 Subject: [PATCH 04/20] save work on dependency management --- distribution/pom.xml | 3 + .../extension-dependencies.json | 4 ++ .../kafka-extraction-namespace/pom.xml | 1 - processing/pom.xml | 2 - .../druid/guice/ExtensionDependencies.java | 53 ++++++++++++++++ .../apache/druid/guice/ExtensionsLoader.java | 42 ++++++++----- .../apache/druid/cli/PullDependencies.java | 60 ++++++++++++++++++- 7 files changed, 146 insertions(+), 19 deletions(-) create mode 100644 extensions-core/kafka-extraction-namespace/extension-dependencies.json create mode 100644 processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java diff --git a/distribution/pom.xml b/distribution/pom.xml index 136935433e93..3e0eb12a7c00 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -178,6 +178,7 @@ ${project.parent.basedir}/examples/bin/run-java + ${project.parent.basedir}/build_output -classpath @@ -258,6 +259,8 @@ -c org.apache.druid.extensions:druid-catalog ${druid.distribution.pulldeps.opts} + --projectBaseDir + ${project.parent.basedir} diff --git a/extensions-core/kafka-extraction-namespace/extension-dependencies.json b/extensions-core/kafka-extraction-namespace/extension-dependencies.json new file mode 100644 index 000000000000..b166d55a97a8 --- /dev/null +++ b/extensions-core/kafka-extraction-namespace/extension-dependencies.json @@ -0,0 +1,4 @@ +{ + "name": "kafka-extraction-namespace", + "dependencies": ["lookups-cached-global"] +} \ No newline at end of file diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index a23bd115da8f..7c6edf8ee323 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -32,7 +32,6 @@ 31.0.0-SNAPSHOT ../../pom.xml - org.apache.druid diff --git a/processing/pom.xml b/processing/pom.xml index bfb39cf2f774..b352101431d7 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -31,7 +31,6 @@ druid 31.0.0-SNAPSHOT - UTF-8 1.6.5 @@ -578,5 +577,4 @@ - diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java b/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java new file mode 100644 index 000000000000..851d313d8dc8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java @@ -0,0 +1,53 @@ +package org.apache.druid.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.ArrayList; + +public class ExtensionDependencies { + @JsonProperty("name") + private String name; + + @JsonProperty("dependencies") + private List dependencies; + + // Default constructor + public ExtensionDependencies() { + this.dependencies = new ArrayList<>(); + } + + // Constructor with parameters + public ExtensionDependencies(String name, List dependencies) { + this.name = name; + this.dependencies = dependencies != null ? dependencies : new ArrayList<>(); + } + + // Getter for name + public String getName() { + return name; + } + + // Setter for name + public void setName(String name) { + this.name = name; + } + + // Getter for dependencies + public List getDependencies() { + return dependencies; + } + + // Setter for dependencies + public void setDependencies(List dependencies) { + this.dependencies = dependencies != null ? dependencies : new ArrayList<>(); + } + + // toString method for easy printing + @Override + public String toString() { + return "ExtensionDependencies{" + + "name='" + name + '\'' + + ", dependencies=" + dependencies + + '}'; + } +} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 1d30fcdf7ec9..1f2f0e596e8d 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -19,6 +19,7 @@ package org.apache.druid.guice; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.inject.Injector; @@ -36,10 +37,14 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; @@ -60,6 +65,7 @@ public class ExtensionsLoader { private static final Logger log = new Logger(ExtensionsLoader.class); + public static final String EXTENSION_DEPENDENCIES_JSON = "extension-dependencies.json"; private final ExtensionsConfig extensionsConfig; private final ConcurrentHashMap, URLClassLoader> loaders = new ConcurrentHashMap<>(); @@ -303,23 +309,31 @@ private void addAllFromFileSystem() } } - ExtensionFirstClassLoader kafkaLoader = null; - ExtensionFirstClassLoader globalLoader = null; - for (Pair extension : loaders.keySet()) { - String extensionName = extension.lhs.getName(); - log.info("Extension name %s", extensionName); - if (extensionName.equals("druid-kafka-extraction-namespace")) { - kafkaLoader = (ExtensionFirstClassLoader) loaders.get(extension); + try { + Map extensionClassLoaderMap = new HashMap<>(); + for (Pair extension : loaders.keySet()) { + extensionClassLoaderMap.put( + extension.lhs.getName(), + loaders.get(extension) + ); } - - if (extensionName.equals("druid-lookups-cached-global")) { - globalLoader = (ExtensionFirstClassLoader) loaders.get(extension); + ObjectMapper objectMapper = new ObjectMapper(); + for (Pair extension : loaders.keySet()) { + log.info("Setting extension dependncies for %s", extension.lhs.getName()); + Path dependenciesPath = Paths.get(extension.lhs.getAbsolutePath() + EXTENSION_DEPENDENCIES_JSON); + if (extension.rhs && Files.exists(dependenciesPath)) { + log.info("Found extension dependencies for %s", extension.lhs.getName()); + ExtensionDependencies extensionDependencies = objectMapper.readValue(dependenciesPath.toFile(), ExtensionDependencies.class); + log.info("Extension dependencies for %s [%s]", extension.lhs.getName(), extensionDependencies.getDependencies()); + + ExtensionFirstClassLoader classLoader = (ExtensionFirstClassLoader) loaders.get(extension); + classLoader.setExtensionDependencyClassLoaders(extensionDependencies.getDependencies().stream().map(extensionClassLoaderMap::get).collect(Collectors.toList())); + } } } - - if (kafkaLoader != null && globalLoader != null) { - log.info("Setting extension dependency %s %s", kafkaLoader.getURLs(), globalLoader.getURLs()); - kafkaLoader.setExtensionDependencyClassLoaders(ImmutableList.of(globalLoader)); + catch (Exception e) { + log.error("Failed to set extension dependencies"); + throw new RuntimeException(e); } for (File extension : getExtensionFilesToLoad()) { diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java index 6ea8626ba7e7..34e51faa0f22 100644 --- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java +++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java @@ -53,12 +53,14 @@ import org.eclipse.aether.util.repository.AuthenticationBuilder; import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.io.PrintStream; import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; +import java.nio.file.Files; import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -71,7 +73,7 @@ public class PullDependencies implements Runnable { private static final Logger log = new Logger(PullDependencies.class); - + private static final List DEFAULT_REMOTE_REPOSITORIES = ImmutableList.of( "https://repo1.maven.org/maven2/" ); @@ -251,6 +253,12 @@ on the class path and propagated around the system. Most places TRY to make sure ) public String proxyPassword = ""; + @Option( + name = {"--projectBaseDir"}, + title = "Absolute path to the project base directory" + ) + public String projectBaseDir = ""; + @SuppressWarnings("unused") // used by com.github.rvesse.airline public PullDependencies() { @@ -305,7 +313,7 @@ public void run() File currExtensionDir = new File(extensionsDir, versionedArtifact.getArtifactId()); createExtensionDirectory(coordinate, currExtensionDir); - + downloadExtensionDependencies(versionedArtifact.getArtifactId(), currExtensionDir); downloadExtension(versionedArtifact, currExtensionDir); } log.info("Finish downloading dependencies for extension coordinates: [%s]", coordinates); @@ -352,6 +360,54 @@ private Artifact getArtifact(String coordinate) return versionedArtifact; } + private void downloadExtensionDependencies(String artifactId, File extensionOutputDir) + { + log.info("Downloading extension dependencies for [%s] [%s], baseDir [%s]", artifactId, extensionOutputDir, projectBaseDir); + String extensionInputDirectoryName = artifactId; + File coreExtensionsDir = new File(projectBaseDir + "/extensions-core/" + extensionInputDirectoryName); + File contribExtensionsDir = new File(projectBaseDir + "/extensions-contrib/" + extensionInputDirectoryName); + + if (coreExtensionsDir.exists() && coreExtensionsDir.isDirectory()) { + copyExtensionDependencyFile(coreExtensionsDir, extensionOutputDir); + return; + } + + if (contribExtensionsDir.exists() && contribExtensionsDir.isDirectory()) { + copyExtensionDependencyFile(contribExtensionsDir, extensionOutputDir); + return; + } + + final String druidPrefix = "druid-"; + if (!extensionInputDirectoryName.startsWith(druidPrefix)) { + return; + } + extensionInputDirectoryName = extensionInputDirectoryName.substring(druidPrefix.length()); + coreExtensionsDir = new File(projectBaseDir + "/extensions-core/" + extensionInputDirectoryName); + contribExtensionsDir = new File(projectBaseDir + "/extensions-contrib/" + extensionInputDirectoryName); + + if (coreExtensionsDir.exists() && coreExtensionsDir.isDirectory()) { + copyExtensionDependencyFile(coreExtensionsDir, extensionOutputDir); + return; + } + + if (contribExtensionsDir.exists() && contribExtensionsDir.isDirectory()) { + copyExtensionDependencyFile(contribExtensionsDir, extensionOutputDir); + return; + } + } + + private void copyExtensionDependencyFile(File extensionDirectory, File extensionOutputDir) + { + try { + File dependenciesFile = new File(extensionDirectory, "extension-dependencies.json"); + if (dependenciesFile.exists()) { + Files.copy(dependenciesFile.toPath(), extensionOutputDir.toPath()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + /** * Download the extension given its maven coordinate * From 95e3803b23b751c8d0b8633614e81cdacc1340d2 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 26 Aug 2024 13:01:42 -0400 Subject: [PATCH 05/20] working dependency graph --- distribution/pom.xml | 25 +++++++++++++++++-- .../extension-dependencies.json | 2 +- .../apache/druid/guice/ExtensionsLoader.java | 15 ++++++----- .../apache/druid/cli/PullDependencies.java | 13 +++++----- 4 files changed, 40 insertions(+), 15 deletions(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index 3e0eb12a7c00..79ea1208244c 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -122,6 +122,27 @@ + + org.commonjava.maven.plugins + directory-maven-plugin + 0.1 + + + directories + + directory-of + + initialize + + druid.basedir + + org.apache.druid + druid + + + + + org.codehaus.mojo exec-maven-plugin @@ -194,6 +215,8 @@ --clean --defaultVersion ${project.parent.version} + --project-base-dir + ${druid.basedir} -l ${settings.localRepository} -h @@ -259,8 +282,6 @@ -c org.apache.druid.extensions:druid-catalog ${druid.distribution.pulldeps.opts} - --projectBaseDir - ${project.parent.basedir} diff --git a/extensions-core/kafka-extraction-namespace/extension-dependencies.json b/extensions-core/kafka-extraction-namespace/extension-dependencies.json index b166d55a97a8..017c1f52892c 100644 --- a/extensions-core/kafka-extraction-namespace/extension-dependencies.json +++ b/extensions-core/kafka-extraction-namespace/extension-dependencies.json @@ -1,4 +1,4 @@ { "name": "kafka-extraction-namespace", - "dependencies": ["lookups-cached-global"] + "dependencies": ["druid-lookups-cached-global"] } \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 1f2f0e596e8d..62ce92bd8bc2 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -319,15 +319,18 @@ private void addAllFromFileSystem() } ObjectMapper objectMapper = new ObjectMapper(); for (Pair extension : loaders.keySet()) { - log.info("Setting extension dependncies for %s", extension.lhs.getName()); - Path dependenciesPath = Paths.get(extension.lhs.getAbsolutePath() + EXTENSION_DEPENDENCIES_JSON); + Path dependenciesPath = Paths.get(extension.lhs.getAbsolutePath() + "/" + EXTENSION_DEPENDENCIES_JSON); if (extension.rhs && Files.exists(dependenciesPath)) { - log.info("Found extension dependencies for %s", extension.lhs.getName()); ExtensionDependencies extensionDependencies = objectMapper.readValue(dependenciesPath.toFile(), ExtensionDependencies.class); - log.info("Extension dependencies for %s [%s]", extension.lhs.getName(), extensionDependencies.getDependencies()); - ExtensionFirstClassLoader classLoader = (ExtensionFirstClassLoader) loaders.get(extension); - classLoader.setExtensionDependencyClassLoaders(extensionDependencies.getDependencies().stream().map(extensionClassLoaderMap::get).collect(Collectors.toList())); + List extensionClassLoaders = new ArrayList<>(); + for (String dependency : extensionDependencies.getDependencies()) { + if (!extensionClassLoaderMap.containsKey(dependency)) { + throw new RuntimeException(String.format("%s depends on %s which is not a valid extension or not loaded.", extensionDependencies.getName(), dependency)); + } + extensionClassLoaders.add(extensionClassLoaderMap.get(dependency)); + } + classLoader.setExtensionDependencyClassLoaders(extensionClassLoaders); } } } diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java index 34e51faa0f22..6262c905fe2b 100644 --- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java +++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java @@ -73,7 +73,7 @@ public class PullDependencies implements Runnable { private static final Logger log = new Logger(PullDependencies.class); - + private static final List DEFAULT_REMOTE_REPOSITORIES = ImmutableList.of( "https://repo1.maven.org/maven2/" ); @@ -159,6 +159,7 @@ on the class path and propagated around the system. Most places TRY to make sure Dependencies.builder() .put("commons-beanutils", "commons-beanutils-core") .build(); + public static final String EXTENSION_DEPENDENCIES_JSON = "extension-dependencies.json"; private final Dependencies hadoopExclusions; @@ -254,7 +255,7 @@ on the class path and propagated around the system. Most places TRY to make sure public String proxyPassword = ""; @Option( - name = {"--projectBaseDir"}, + name = {"--project-base-dir"}, title = "Absolute path to the project base directory" ) public String projectBaseDir = ""; @@ -306,7 +307,7 @@ public void run() ); try { - log.info("Start downloading dependencies for extension coordinates: [%s]", coordinates); + System.out.println("Start downloading dependencies for extension coordinates: " + coordinates); for (String coordinate : coordinates) { coordinate = coordinate.trim(); final Artifact versionedArtifact = getArtifact(coordinate); @@ -362,7 +363,7 @@ private Artifact getArtifact(String coordinate) private void downloadExtensionDependencies(String artifactId, File extensionOutputDir) { - log.info("Downloading extension dependencies for [%s] [%s], baseDir [%s]", artifactId, extensionOutputDir, projectBaseDir); + System.out.println("Downloading extension dependencies for " + artifactId + " " + extensionOutputDir + " " + projectBaseDir); String extensionInputDirectoryName = artifactId; File coreExtensionsDir = new File(projectBaseDir + "/extensions-core/" + extensionInputDirectoryName); File contribExtensionsDir = new File(projectBaseDir + "/extensions-contrib/" + extensionInputDirectoryName); @@ -399,9 +400,9 @@ private void downloadExtensionDependencies(String artifactId, File extensionOutp private void copyExtensionDependencyFile(File extensionDirectory, File extensionOutputDir) { try { - File dependenciesFile = new File(extensionDirectory, "extension-dependencies.json"); + File dependenciesFile = new File(extensionDirectory, EXTENSION_DEPENDENCIES_JSON); if (dependenciesFile.exists()) { - Files.copy(dependenciesFile.toPath(), extensionOutputDir.toPath()); + Files.copy(dependenciesFile.toPath(), new File(extensionOutputDir, EXTENSION_DEPENDENCIES_JSON).toPath()); } } catch (Exception e) { throw new RuntimeException(e); From 3f948687b672a7c7857ee013eb342a92325a0ca2 Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 28 Aug 2024 16:13:29 -0400 Subject: [PATCH 06/20] working pull --- .../extension-dependencies.json | 4 - .../resources/extension-dependencies.json | 4 + .../druid/guice/ExtensionDependencies.java | 46 ++----- .../guice/ExtensionFirstClassLoader.java | 46 +------ .../apache/druid/guice/ExtensionsLoader.java | 116 ++++++++++------- .../druid/guice/StandardClassLoader.java | 121 ++++++++++++++++++ .../druid/guice/ExtensionsLoaderTest.java | 20 +-- .../apache/druid/cli/PullDependencies.java | 50 -------- 8 files changed, 223 insertions(+), 184 deletions(-) delete mode 100644 extensions-core/kafka-extraction-namespace/extension-dependencies.json create mode 100644 extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json create mode 100644 processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java diff --git a/extensions-core/kafka-extraction-namespace/extension-dependencies.json b/extensions-core/kafka-extraction-namespace/extension-dependencies.json deleted file mode 100644 index 017c1f52892c..000000000000 --- a/extensions-core/kafka-extraction-namespace/extension-dependencies.json +++ /dev/null @@ -1,4 +0,0 @@ -{ - "name": "kafka-extraction-namespace", - "dependencies": ["druid-lookups-cached-global"] -} \ No newline at end of file diff --git a/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json b/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json new file mode 100644 index 000000000000..d8224ab352f6 --- /dev/null +++ b/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json @@ -0,0 +1,4 @@ +{ + "name": "kafka-extraction-namespace", + "dependsOnDruidExtensions": ["druid-lookups-cached-global"] +} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java b/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java index 851d313d8dc8..c8f2036dd254 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java @@ -1,53 +1,25 @@ package org.apache.druid.guice; import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nonnull; import java.util.List; import java.util.ArrayList; public class ExtensionDependencies { - @JsonProperty("name") - private String name; - @JsonProperty("dependencies") - private List dependencies; + @JsonProperty("dependsOnDruidExtensions") + private List dependsOnDruidExtensions; - // Default constructor public ExtensionDependencies() { - this.dependencies = new ArrayList<>(); - } - - // Constructor with parameters - public ExtensionDependencies(String name, List dependencies) { - this.name = name; - this.dependencies = dependencies != null ? dependencies : new ArrayList<>(); - } - - // Getter for name - public String getName() { - return name; - } - - // Setter for name - public void setName(String name) { - this.name = name; - } - - // Getter for dependencies - public List getDependencies() { - return dependencies; + this.dependsOnDruidExtensions = new ArrayList<>(); } - // Setter for dependencies - public void setDependencies(List dependencies) { - this.dependencies = dependencies != null ? dependencies : new ArrayList<>(); + public ExtensionDependencies(@Nonnull final List dependsOnDruidExtensions) { + this.dependsOnDruidExtensions = dependsOnDruidExtensions; } - // toString method for easy printing - @Override - public String toString() { - return "ExtensionDependencies{" + - "name='" + name + '\'' + - ", dependencies=" + dependencies + - '}'; + public List getDependsOnDruidExtensions() { + return dependsOnDruidExtensions; } } \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java index 0ffc8fdbeb75..f529eec95435 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.net.URL; -import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Enumeration; import java.util.List; @@ -32,16 +31,14 @@ /** * The ClassLoader that gets used when druid.extensions.useExtensionClassloaderFirst = true. */ -public class ExtensionFirstClassLoader extends URLClassLoader +public class ExtensionFirstClassLoader extends StandardClassLoader { private final ClassLoader druidLoader; - private List extensionDependencyClassLoaders; public ExtensionFirstClassLoader(final URL[] urls, final ClassLoader druidLoader, final List extensionDependencyClassLoaders) { - super(urls, null); + super(urls, null, extensionDependencyClassLoaders); this.druidLoader = Preconditions.checkNotNull(druidLoader, "druidLoader"); - this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders"); } @Override @@ -63,11 +60,11 @@ protected Class loadClass(final String name, final boolean resolve) throws Cl } catch (ClassNotFoundException e) { try { - return loadClassFromExtensionDependencies(name); + clazz = loadClassFromExtensionDependencies(name); } catch (ClassNotFoundException e2) { // Try the Druid classloader. Will throw ClassNotFoundException if the class can't be loaded. - return druidLoader.loadClass(name); + clazz = druidLoader.loadClass(name); } } } @@ -80,19 +77,6 @@ protected Class loadClass(final String name, final boolean resolve) throws Cl } } - protected Class loadClassFromExtensionDependencies(final String name) throws ClassNotFoundException - { - for (ClassLoader classLoader : extensionDependencyClassLoaders) { - try { - return classLoader.loadClass(name); - } - catch (ClassNotFoundException ignored) { - } - } - throw new ClassNotFoundException(); - } - - @Override public URL getResource(final String name) { @@ -109,18 +93,6 @@ public URL getResource(final String name) return druidLoader.getResource(name); } - protected URL getResourceFromExtensionsDependencies(final String name) - { - URL resourceFromExtension = null; - for (ClassLoader classLoader : extensionDependencyClassLoaders) { - resourceFromExtension = classLoader.getResource(name); - if (resourceFromExtension != null) { - break; - } - } - return resourceFromExtension; - } - @Override public Enumeration getResources(final String name) throws IOException @@ -128,13 +100,7 @@ public Enumeration getResources(final String name) throws IOException final List urls = new ArrayList<>(); Iterators.addAll(urls, Iterators.forEnumeration(super.getResources(name))); Iterators.addAll(urls, Iterators.forEnumeration(druidLoader.getResources(name))); - for (ClassLoader classLoader : extensionDependencyClassLoaders) { - Iterators.addAll(urls, Iterators.forEnumeration(classLoader.getResources(name))); - } + addExtensionResources(name, urls); return Iterators.asEnumeration(urls.iterator()); } - - public void setExtensionDependencyClassLoaders(List extensionDependencyClassLoaders) { - this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders"); - } -} +} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 62ce92bd8bc2..10ae1373289f 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -27,6 +27,8 @@ import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import javax.inject.Inject; @@ -37,21 +39,22 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.ServiceLoader; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; import java.util.stream.Collectors; /** @@ -66,8 +69,9 @@ public class ExtensionsLoader { private static final Logger log = new Logger(ExtensionsLoader.class); public static final String EXTENSION_DEPENDENCIES_JSON = "extension-dependencies.json"; - private final ExtensionsConfig extensionsConfig; + private final ObjectMapper objectMapper; + private final ConcurrentHashMap, URLClassLoader> loaders = new ConcurrentHashMap<>(); /** @@ -76,9 +80,10 @@ public class ExtensionsLoader private final ConcurrentHashMap, Collection> extensions = new ConcurrentHashMap<>(); @Inject - public ExtensionsLoader(ExtensionsConfig config) + public ExtensionsLoader(ExtensionsConfig config, ObjectMapper objectMapper) { this.extensionsConfig = config; + this.objectMapper = objectMapper; } public static ExtensionsLoader instance(Injector injector) @@ -233,7 +238,7 @@ private static URLClassLoader makeClassLoaderForExtension( if (useExtensionClassloaderFirst) { return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader(), ImmutableList.of()); } else { - return new URLClassLoader(urls, ExtensionsLoader.class.getClassLoader()); + return new StandardClassLoader(urls, ExtensionsLoader.class.getClassLoader(), ImmutableList.of()); } } @@ -298,45 +303,16 @@ private void addAllFromCurrentClassLoader() private void addAllFromFileSystem() { - for (File extension : getExtensionFilesToLoad()) { - try { - getClassLoaderForExtension( - extension, - extensionsConfig.isUseExtensionClassloaderFirst() - ); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + Map extensionClassLoaderMap = new HashMap<>(); - try { - Map extensionClassLoaderMap = new HashMap<>(); - for (Pair extension : loaders.keySet()) { - extensionClassLoaderMap.put( - extension.lhs.getName(), - loaders.get(extension) - ); - } - ObjectMapper objectMapper = new ObjectMapper(); - for (Pair extension : loaders.keySet()) { - Path dependenciesPath = Paths.get(extension.lhs.getAbsolutePath() + "/" + EXTENSION_DEPENDENCIES_JSON); - if (extension.rhs && Files.exists(dependenciesPath)) { - ExtensionDependencies extensionDependencies = objectMapper.readValue(dependenciesPath.toFile(), ExtensionDependencies.class); - ExtensionFirstClassLoader classLoader = (ExtensionFirstClassLoader) loaders.get(extension); - List extensionClassLoaders = new ArrayList<>(); - for (String dependency : extensionDependencies.getDependencies()) { - if (!extensionClassLoaderMap.containsKey(dependency)) { - throw new RuntimeException(String.format("%s depends on %s which is not a valid extension or not loaded.", extensionDependencies.getName(), dependency)); - } - extensionClassLoaders.add(extensionClassLoaderMap.get(dependency)); - } - classLoader.setExtensionDependencyClassLoaders(extensionClassLoaders); - } - } - } - catch (Exception e) { - log.error("Failed to set extension dependencies"); - throw new RuntimeException(e); + for (File extension : getExtensionFilesToLoad()) { + extensionClassLoaderMap.put( + extension.getName(), + getClassLoaderForExtension( + extension, + extensionsConfig.isUseExtensionClassloaderFirst() + ) + ); } for (File extension : getExtensionFilesToLoad()) { @@ -345,6 +321,30 @@ private void addAllFromFileSystem() extension, extensionsConfig.isUseExtensionClassloaderFirst() ); + Optional extensionDependencies = getExtensionDependencies(loader); + List extensionDruidExtensionDependencies; + if (extensionDependencies.isPresent()) { + extensionDruidExtensionDependencies = extensionDependencies.get().getDependsOnDruidExtensions(); + } else { + extensionDruidExtensionDependencies = ImmutableList.of(); + } + + log.info("Discovered extension dependencies %s", extensionDruidExtensionDependencies); + List chainedClassLoadersForExtension = new ArrayList<>(); + for (String druidExtensionDependency : extensionDruidExtensionDependencies) { + if (!extensionClassLoaderMap.containsKey(druidExtensionDependency)) { + throw new RE( + StringUtils.format( + "%s depends on %s which is not a valid extension or not loaded.", + extension.getName(), + druidExtensionDependency + ) + ); + } + chainedClassLoadersForExtension.add(extensionClassLoaderMap.get(druidExtensionDependency)); + } + ((StandardClassLoader) loader).setExtensionDependencyClassLoaders(chainedClassLoadersForExtension); + log.info( "Loading extension [%s], jars: %s", extension.getName(), @@ -381,5 +381,33 @@ private void tryAdd(T serviceImpl, String extensionType) implsToLoad.add(serviceImpl); } } + + private Optional getExtensionDependencies(URLClassLoader loader) + { + for (URL url: loader.getURLs()) { + File jarFileLocation = new File(url.getPath()); + if (jarFileLocation.getName().startsWith("druid")) { + try (JarFile jarFile = new JarFile(url.getPath())) { + Enumeration entries = jarFile.entries(); + + while (entries.hasMoreElements()) { + JarEntry entry = entries.nextElement(); + String entryName = entry.getName(); + + if (!entry.isDirectory() && entryName.equals(EXTENSION_DEPENDENCIES_JSON)) { + log.info("Found extension dependency entry in druid jar %s", url.getPath()); + return Optional.of(objectMapper.readValue( + jarFile.getInputStream(entry), + ExtensionDependencies.class + )); + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + return Optional.empty(); + } } } diff --git a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java b/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java new file mode 100644 index 000000000000..cce8d281ee02 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.guice; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterators; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.List; + + +/** + * The ClassLoader that gets used when druid.extensions.useExtensionClassloaderFirst = false. + */ +public class StandardClassLoader extends URLClassLoader +{ + private List extensionDependencyClassLoaders; + + public StandardClassLoader(final URL[] urls, final ClassLoader druidLoader, final List extensionDependencyClassLoaders) + { + super(urls, druidLoader); + this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders"); + } + + @Override + protected Class loadClass(final String name, final boolean resolve) throws ClassNotFoundException + { + Class clazz; + try { + clazz = super.loadClass(name, resolve); + } + catch (ClassNotFoundException e) { + clazz = loadClassFromExtensionDependencies(name); + } + if (resolve) { + resolveClass(clazz); + } + + return clazz; + } + + @Override + public URL getResource(final String name) + { + URL resource = super.getResource(name); + + if (resource != null) { + return resource; + } + + return getResourceFromExtensionsDependencies(name); + } + + @Override + public Enumeration getResources(final String name) throws IOException + { + final List urls = new ArrayList<>(); + Iterators.addAll(urls, Iterators.forEnumeration(super.getResources(name))); + addExtensionResources(name, urls); + return Iterators.asEnumeration(urls.iterator()); + } + + protected URL getResourceFromExtensionsDependencies(final String name) + { + URL resourceFromExtension = null; + for (ClassLoader classLoader : extensionDependencyClassLoaders) { + resourceFromExtension = classLoader.getResource(name); + if (resourceFromExtension != null) { + break; + } + } + return resourceFromExtension; + } + + protected Class loadClassFromExtensionDependencies(final String name) throws ClassNotFoundException + { + for (ClassLoader classLoader : extensionDependencyClassLoaders) { + try { + return classLoader.loadClass(name); + } + catch (ClassNotFoundException ignored) { + } + } + throw new ClassNotFoundException(); + } + + protected void addExtensionResources(final String name, List urls) throws IOException + { + for (ClassLoader classLoader : extensionDependencyClassLoaders) { + Iterators.addAll(urls, Iterators.forEnumeration(classLoader.getResources(name))); + } + } + + public void setExtensionDependencyClassLoaders(@Nonnull List extensionDependencyClassLoaders) + { + this.extensionDependencyClassLoaders = extensionDependencyClassLoaders; + } +} \ No newline at end of file diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 44b7f06fb3d8..3fa93066b2d4 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -48,6 +48,8 @@ public class ExtensionsLoaderTest @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + private final ObjectMapper objectMapper = new ObjectMapper(); + private Injector startupInjector() { return new StartupInjectorBuilder() @@ -90,7 +92,7 @@ public void test04DuplicateClassLoaderExtensions() throws Exception @Test public void test06GetClassLoaderForExtension() throws IOException { - final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig()); + final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); final File some_extension_dir = temporaryFolder.newFolder(); final File a_jar = new File(some_extension_dir, "a.jar"); @@ -109,7 +111,7 @@ public void test06GetClassLoaderForExtension() throws IOException @Test public void testGetLoadedModules() { - final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig()); + final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); Collection modules = extnLoader.getModules(); HashSet moduleSet = new HashSet<>(modules); @@ -134,7 +136,7 @@ public String getDirectory() { return tmpDir.getAbsolutePath(); } - }); + }, objectMapper); Assert.assertArrayEquals( "Non-exist root extensionsDir should return an empty array of File", new File[]{}, @@ -155,7 +157,7 @@ public String getDirectory() return extensionsDir.getAbsolutePath(); } }; - final ExtensionsLoader extnLoader = new ExtensionsLoader(config); + final ExtensionsLoader extnLoader = new ExtensionsLoader(config, objectMapper); extnLoader.getExtensionFilesToLoad(); } @@ -172,7 +174,7 @@ public String getDirectory() } }; - final ExtensionsLoader extnLoader = new ExtensionsLoader(config); + final ExtensionsLoader extnLoader = new ExtensionsLoader(config, objectMapper); Assert.assertArrayEquals( "Empty root extensionsDir should return an empty array of File", new File[]{}, @@ -196,7 +198,7 @@ public String getDirectory() return extensionsDir.getAbsolutePath(); } }; - final ExtensionsLoader extnLoader = new ExtensionsLoader(config); + final ExtensionsLoader extnLoader = new ExtensionsLoader(config, objectMapper); final File mysql_metadata_storage = new File(extensionsDir, "mysql-metadata-storage"); mysql_metadata_storage.mkdir(); @@ -231,7 +233,7 @@ public String getDirectory() return extensionsDir.getAbsolutePath(); } }; - final ExtensionsLoader extnLoader = new ExtensionsLoader(config); + final ExtensionsLoader extnLoader = new ExtensionsLoader(config, objectMapper); final File mysql_metadata_storage = new File(extensionsDir, "mysql-metadata-storage"); final File random_extension = new File(extensionsDir, "random-extensions"); @@ -267,7 +269,7 @@ public String getDirectory() }; final File random_extension = new File(extensionsDir, "random-extensions"); random_extension.mkdir(); - final ExtensionsLoader extnLoader = new ExtensionsLoader(config); + final ExtensionsLoader extnLoader = new ExtensionsLoader(config, objectMapper); extnLoader.getExtensionFilesToLoad(); } @@ -323,7 +325,7 @@ public void testExtensionsWithSameDirName() throws Exception Assert.assertTrue(jar1.createNewFile()); Assert.assertTrue(jar2.createNewFile()); - final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig()); + final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false); diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java index 6262c905fe2b..19eb5c138a63 100644 --- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java +++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java @@ -307,14 +307,12 @@ public void run() ); try { - System.out.println("Start downloading dependencies for extension coordinates: " + coordinates); for (String coordinate : coordinates) { coordinate = coordinate.trim(); final Artifact versionedArtifact = getArtifact(coordinate); File currExtensionDir = new File(extensionsDir, versionedArtifact.getArtifactId()); createExtensionDirectory(coordinate, currExtensionDir); - downloadExtensionDependencies(versionedArtifact.getArtifactId(), currExtensionDir); downloadExtension(versionedArtifact, currExtensionDir); } log.info("Finish downloading dependencies for extension coordinates: [%s]", coordinates); @@ -361,54 +359,6 @@ private Artifact getArtifact(String coordinate) return versionedArtifact; } - private void downloadExtensionDependencies(String artifactId, File extensionOutputDir) - { - System.out.println("Downloading extension dependencies for " + artifactId + " " + extensionOutputDir + " " + projectBaseDir); - String extensionInputDirectoryName = artifactId; - File coreExtensionsDir = new File(projectBaseDir + "/extensions-core/" + extensionInputDirectoryName); - File contribExtensionsDir = new File(projectBaseDir + "/extensions-contrib/" + extensionInputDirectoryName); - - if (coreExtensionsDir.exists() && coreExtensionsDir.isDirectory()) { - copyExtensionDependencyFile(coreExtensionsDir, extensionOutputDir); - return; - } - - if (contribExtensionsDir.exists() && contribExtensionsDir.isDirectory()) { - copyExtensionDependencyFile(contribExtensionsDir, extensionOutputDir); - return; - } - - final String druidPrefix = "druid-"; - if (!extensionInputDirectoryName.startsWith(druidPrefix)) { - return; - } - extensionInputDirectoryName = extensionInputDirectoryName.substring(druidPrefix.length()); - coreExtensionsDir = new File(projectBaseDir + "/extensions-core/" + extensionInputDirectoryName); - contribExtensionsDir = new File(projectBaseDir + "/extensions-contrib/" + extensionInputDirectoryName); - - if (coreExtensionsDir.exists() && coreExtensionsDir.isDirectory()) { - copyExtensionDependencyFile(coreExtensionsDir, extensionOutputDir); - return; - } - - if (contribExtensionsDir.exists() && contribExtensionsDir.isDirectory()) { - copyExtensionDependencyFile(contribExtensionsDir, extensionOutputDir); - return; - } - } - - private void copyExtensionDependencyFile(File extensionDirectory, File extensionOutputDir) - { - try { - File dependenciesFile = new File(extensionDirectory, EXTENSION_DEPENDENCIES_JSON); - if (dependenciesFile.exists()) { - Files.copy(dependenciesFile.toPath(), new File(extensionOutputDir, EXTENSION_DEPENDENCIES_JSON).toPath()); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - /** * Download the extension given its maven coordinate * From 5a5802d1c61c2286743b05a1e455877841b0dfce Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 28 Aug 2024 16:32:10 -0400 Subject: [PATCH 07/20] fix style --- .../druid/guice/ExtensionDependencies.java | 36 +++++++++++++++---- .../guice/ExtensionFirstClassLoader.java | 2 +- .../apache/druid/guice/ExtensionsLoader.java | 7 ++-- .../druid/guice/StandardClassLoader.java | 2 +- .../apache/druid/cli/PullDependencies.java | 2 -- 5 files changed, 36 insertions(+), 13 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java b/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java index c8f2036dd254..100cc87f08f8 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java @@ -1,25 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.guice; import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nonnull; -import java.util.List; + import java.util.ArrayList; +import java.util.List; -public class ExtensionDependencies { +public class ExtensionDependencies +{ @JsonProperty("dependsOnDruidExtensions") private List dependsOnDruidExtensions; - public ExtensionDependencies() { + public ExtensionDependencies() + { this.dependsOnDruidExtensions = new ArrayList<>(); } - public ExtensionDependencies(@Nonnull final List dependsOnDruidExtensions) { + public ExtensionDependencies(@Nonnull final List dependsOnDruidExtensions) + { this.dependsOnDruidExtensions = dependsOnDruidExtensions; } - public List getDependsOnDruidExtensions() { + public List getDependsOnDruidExtensions() + { return dependsOnDruidExtensions; } -} \ No newline at end of file +} diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java index f529eec95435..5ad4be95da1c 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java @@ -103,4 +103,4 @@ public Enumeration getResources(final String name) throws IOException addExtensionResources(name, urls); return Iterators.asEnumeration(urls.iterator()); } -} \ No newline at end of file +} diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 10ae1373289f..4e594977bb46 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -384,7 +384,7 @@ private void tryAdd(T serviceImpl, String extensionType) private Optional getExtensionDependencies(URLClassLoader loader) { - for (URL url: loader.getURLs()) { + for (URL url : loader.getURLs()) { File jarFileLocation = new File(url.getPath()); if (jarFileLocation.getName().startsWith("druid")) { try (JarFile jarFile = new JarFile(url.getPath())) { @@ -394,7 +394,7 @@ private Optional getExtensionDependencies(URLClassLoader JarEntry entry = entries.nextElement(); String entryName = entry.getName(); - if (!entry.isDirectory() && entryName.equals(EXTENSION_DEPENDENCIES_JSON)) { + if (!entry.isDirectory() && EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { log.info("Found extension dependency entry in druid jar %s", url.getPath()); return Optional.of(objectMapper.readValue( jarFile.getInputStream(entry), @@ -402,7 +402,8 @@ private Optional getExtensionDependencies(URLClassLoader )); } } - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } diff --git a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java b/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java index cce8d281ee02..f535bbde908e 100644 --- a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java @@ -118,4 +118,4 @@ public void setExtensionDependencyClassLoaders(@Nonnull List extens { this.extensionDependencyClassLoaders = extensionDependencyClassLoaders; } -} \ No newline at end of file +} diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java index 19eb5c138a63..643a2f337ebe 100644 --- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java +++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java @@ -53,14 +53,12 @@ import org.eclipse.aether.util.repository.AuthenticationBuilder; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.io.PrintStream; import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; -import java.nio.file.Files; import java.util.ArrayList; import java.util.List; import java.util.Set; From 406f2e6056c3eb87796a4bf3adc51ac1c4507910 Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 28 Aug 2024 16:39:37 -0400 Subject: [PATCH 08/20] fix style --- distribution/pom.xml | 24 ------------------- docs/querying/kafka-extraction-namespace.md | 5 +--- docs/querying/lookups-cached-global.md | 3 --- .../kafka-extraction-namespace/pom.xml | 1 + processing/pom.xml | 2 ++ .../apache/druid/guice/ExtensionsLoader.java | 14 ++++------- .../apache/druid/cli/PullDependencies.java | 9 ++----- 7 files changed, 11 insertions(+), 47 deletions(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index 79ea1208244c..136935433e93 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -122,27 +122,6 @@ - - org.commonjava.maven.plugins - directory-maven-plugin - 0.1 - - - directories - - directory-of - - initialize - - druid.basedir - - org.apache.druid - druid - - - - - org.codehaus.mojo exec-maven-plugin @@ -199,7 +178,6 @@ ${project.parent.basedir}/examples/bin/run-java - ${project.parent.basedir}/build_output -classpath @@ -215,8 +193,6 @@ --clean --defaultVersion ${project.parent.version} - --project-base-dir - ${druid.basedir} -l ${settings.localRepository} -h diff --git a/docs/querying/kafka-extraction-namespace.md b/docs/querying/kafka-extraction-namespace.md index e7d02babdd66..1cfa91aac554 100644 --- a/docs/querying/kafka-extraction-namespace.md +++ b/docs/querying/kafka-extraction-namespace.md @@ -22,10 +22,7 @@ title: "Apache Kafka Lookups" ~ under the License. --> -To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-kafka-extraction-namespace` in the extensions load list. - -This extension provides all the functionality of `druid-lookups-cached-global`. You should only load one of the two extensions. -(load `druid-lookups-cached-global` if you only want global cached lookup functionality, load `druid-kafka-extraction-namespace` if you also want kafka lookup functionality) +To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` in the extensions load list. If you need updates to populate as promptly as possible, it is possible to plug into a Kafka topic whose key is the old value and message is the desired new value (both in UTF-8) as a LookupExtractorFactory. diff --git a/docs/querying/lookups-cached-global.md b/docs/querying/lookups-cached-global.md index 8db1a4333d61..72c4189c2dad 100644 --- a/docs/querying/lookups-cached-global.md +++ b/docs/querying/lookups-cached-global.md @@ -24,9 +24,6 @@ title: "Globally Cached Lookups" To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` in the extensions load list. -The `druid-kafka-extraction-namespace` extension provides all the functionality of `druid-lookups-cached-global`. You should only load one of the two extensions. -(load `druid-lookups-cached-global` if you only want global cached lookup functionality, load `druid-kafka-extraction-namespace` if you also want kafka lookup functionality) - ## Configuration :::info Static configuration is no longer supported. Lookups can be configured through diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 7c6edf8ee323..a23bd115da8f 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -32,6 +32,7 @@ 31.0.0-SNAPSHOT ../../pom.xml + org.apache.druid diff --git a/processing/pom.xml b/processing/pom.xml index b352101431d7..bfb39cf2f774 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -31,6 +31,7 @@ druid 31.0.0-SNAPSHOT + UTF-8 1.6.5 @@ -577,4 +578,5 @@ + diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 4e594977bb46..b096efc8da85 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -290,7 +290,6 @@ private ServiceLoadingFromExtensions(Class serviceClass) if (extensionsConfig.searchCurrentClassloader()) { addAllFromCurrentClassLoader(); } - addAllFromFileSystem(); } @@ -316,20 +315,17 @@ private void addAllFromFileSystem() } for (File extension : getExtensionFilesToLoad()) { + log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { - URLClassLoader loader = getClassLoaderForExtension( + final URLClassLoader loader = getClassLoaderForExtension( extension, extensionsConfig.isUseExtensionClassloaderFirst() ); Optional extensionDependencies = getExtensionDependencies(loader); - List extensionDruidExtensionDependencies; - if (extensionDependencies.isPresent()) { - extensionDruidExtensionDependencies = extensionDependencies.get().getDependsOnDruidExtensions(); - } else { - extensionDruidExtensionDependencies = ImmutableList.of(); - } + List extensionDruidExtensionDependencies = extensionDependencies.isPresent() + ? extensionDependencies.get().getDependsOnDruidExtensions() + : ImmutableList.of(); - log.info("Discovered extension dependencies %s", extensionDruidExtensionDependencies); List chainedClassLoadersForExtension = new ArrayList<>(); for (String druidExtensionDependency : extensionDruidExtensionDependencies) { if (!extensionClassLoaderMap.containsKey(druidExtensionDependency)) { diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java index 643a2f337ebe..6ea8626ba7e7 100644 --- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java +++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java @@ -157,7 +157,6 @@ on the class path and propagated around the system. Most places TRY to make sure Dependencies.builder() .put("commons-beanutils", "commons-beanutils-core") .build(); - public static final String EXTENSION_DEPENDENCIES_JSON = "extension-dependencies.json"; private final Dependencies hadoopExclusions; @@ -252,12 +251,6 @@ on the class path and propagated around the system. Most places TRY to make sure ) public String proxyPassword = ""; - @Option( - name = {"--project-base-dir"}, - title = "Absolute path to the project base directory" - ) - public String projectBaseDir = ""; - @SuppressWarnings("unused") // used by com.github.rvesse.airline public PullDependencies() { @@ -305,12 +298,14 @@ public void run() ); try { + log.info("Start downloading dependencies for extension coordinates: [%s]", coordinates); for (String coordinate : coordinates) { coordinate = coordinate.trim(); final Artifact versionedArtifact = getArtifact(coordinate); File currExtensionDir = new File(extensionsDir, versionedArtifact.getArtifactId()); createExtensionDirectory(coordinate, currExtensionDir); + downloadExtension(versionedArtifact, currExtensionDir); } log.info("Finish downloading dependencies for extension coordinates: [%s]", coordinates); From e30c6bd64b77030b9c6307d1165a9e9519ac28eb Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 28 Aug 2024 16:50:26 -0400 Subject: [PATCH 09/20] remove name --- .../src/main/resources/extension-dependencies.json | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json b/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json index d8224ab352f6..8b9d1666509a 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json +++ b/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json @@ -1,4 +1,3 @@ { - "name": "kafka-extraction-namespace", "dependsOnDruidExtensions": ["druid-lookups-cached-global"] } \ No newline at end of file From 536d72f0db73699467516cb137e8981eb495aa1c Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 11 Sep 2024 15:42:24 -0400 Subject: [PATCH 10/20] load extension dependencies recursively --- .../indexing/common/task/HadoopTask.java | 17 +-- .../guice/ExtensionFirstClassLoader.java | 2 +- .../apache/druid/guice/ExtensionsLoader.java | 129 +++++++++--------- .../druid/guice/ExtensionsLoaderTest.java | 6 +- .../apache/druid/cli/CliHadoopIndexer.java | 10 +- 5 files changed, 83 insertions(+), 81 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java index 201024540e16..05d45709dab3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java @@ -154,8 +154,9 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo } final List extensionURLs = new ArrayList<>(); - for (final File extension : EXTENSIONS_LOADER.getExtensionFilesToLoad()) { - final URLClassLoader extensionLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(extension, false); + final File[] extensionFiles = EXTENSIONS_LOADER.getExtensionFilesToLoad(); + for (final File extension : extensionFiles) { + final URLClassLoader extensionLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(extension, false, extensionFiles); extensionURLs.addAll(Arrays.asList(extensionLoader.getURLs())); } @@ -164,12 +165,12 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo final List localClassLoaderURLs = new ArrayList<>(jobURLs); // hadoop dependencies come before druid classes because some extensions depend on them - for (final File hadoopDependency : - Initialization.getHadoopDependencyFilesToLoad( - finalHadoopDependencyCoordinates, - EXTENSIONS_LOADER.config() - )) { - final URLClassLoader hadoopLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(hadoopDependency, false); + final File[] hadoopDependencyFiles = Initialization.getHadoopDependencyFilesToLoad( + finalHadoopDependencyCoordinates, + EXTENSIONS_LOADER.config() + ); + for (final File hadoopDependency : hadoopDependencyFiles) { + final URLClassLoader hadoopLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(hadoopDependency, false, hadoopDependencyFiles); localClassLoaderURLs.addAll(Arrays.asList(hadoopLoader.getURLs())); } diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java index 5ad4be95da1c..ebbf7cfc79bb 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java @@ -99,8 +99,8 @@ public Enumeration getResources(final String name) throws IOException { final List urls = new ArrayList<>(); Iterators.addAll(urls, Iterators.forEnumeration(super.getResources(name))); - Iterators.addAll(urls, Iterators.forEnumeration(druidLoader.getResources(name))); addExtensionResources(name, urls); + Iterators.addAll(urls, Iterators.forEnumeration(druidLoader.getResources(name))); return Iterators.asEnumeration(urls.iterator()); } } diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index b096efc8da85..5aaf1d1590e9 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -207,14 +207,43 @@ public File[] getExtensionFilesToLoad() * * @return a URLClassLoader that loads all the jars on which the extension is dependent */ - public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) + public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, File[] loadedExtensions) { return loaders.computeIfAbsent( Pair.of(extension, useExtensionClassloaderFirst), - k -> makeClassLoaderForExtension(k.lhs, k.rhs) + k -> makeClassLoaderWithExtensionDependencies(k.lhs, k.rhs, loadedExtensions) ); } + private URLClassLoader makeClassLoaderWithExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, File[] loadedExtensions) { + URLClassLoader classLoader = makeClassLoaderForExtension(extension, useExtensionClassloaderFirst); + Optional extensionDependencies = getExtensionDependencies(extension); + List extensionDependencyNames = extensionDependencies.isPresent() + ? extensionDependencies.get().getDependsOnDruidExtensions() + : ImmutableList.of(); + + List extensionDependencyClassLoaders = new ArrayList<>(); + for (String extensionDependencyName: extensionDependencyNames) { + Optional extensionDependencyFile = Arrays.stream(loadedExtensions) + .filter(file -> file.getName().equals(extensionDependencyName)) + .findAny(); + if (!extensionDependencyFile.isPresent()) { + throw new RE( + StringUtils.format( + "%s depends on %s which is not a valid extension or not loaded.", + extension.getName(), + extensionDependencyName + ) + ); + } + extensionDependencyClassLoaders.add( + getClassLoaderForExtension(extensionDependencyFile.get(), useExtensionClassloaderFirst, loadedExtensions) + ); + } + ((StandardClassLoader) classLoader).setExtensionDependencyClassLoaders(extensionDependencyClassLoaders); + return classLoader; + } + private static URLClassLoader makeClassLoaderForExtension( final File extension, final boolean useExtensionClassloaderFirst @@ -278,6 +307,35 @@ public boolean accept(File dir, String name) } } + private Optional getExtensionDependencies(File extension) + { + final Collection jars = FileUtils.listFiles(extension, new String[]{"jar"}, false); + for (File extensionFile : jars) { + if (extensionFile.getName().startsWith("druid")) { + try (JarFile jarFile = new JarFile(extensionFile.getPath())) { + Enumeration entries = jarFile.entries(); + + while (entries.hasMoreElements()) { + JarEntry entry = entries.nextElement(); + String entryName = entry.getName(); + + if (!entry.isDirectory() && EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { + log.info("Found extension dependency entry in druid jar %s", extensionFile.getPath()); + return Optional.of(objectMapper.readValue( + jarFile.getInputStream(entry), + ExtensionDependencies.class + )); + } + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } + return Optional.empty(); + } + private class ServiceLoadingFromExtensions { private final Class serviceClass; @@ -302,45 +360,15 @@ private void addAllFromCurrentClassLoader() private void addAllFromFileSystem() { - Map extensionClassLoaderMap = new HashMap<>(); - - for (File extension : getExtensionFilesToLoad()) { - extensionClassLoaderMap.put( - extension.getName(), - getClassLoaderForExtension( - extension, - extensionsConfig.isUseExtensionClassloaderFirst() - ) - ); - } - - for (File extension : getExtensionFilesToLoad()) { + File[] extensionFilesToLoad = getExtensionFilesToLoad(); + for (File extension : extensionFilesToLoad) { log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { final URLClassLoader loader = getClassLoaderForExtension( extension, - extensionsConfig.isUseExtensionClassloaderFirst() + extensionsConfig.isUseExtensionClassloaderFirst(), + extensionFilesToLoad ); - Optional extensionDependencies = getExtensionDependencies(loader); - List extensionDruidExtensionDependencies = extensionDependencies.isPresent() - ? extensionDependencies.get().getDependsOnDruidExtensions() - : ImmutableList.of(); - - List chainedClassLoadersForExtension = new ArrayList<>(); - for (String druidExtensionDependency : extensionDruidExtensionDependencies) { - if (!extensionClassLoaderMap.containsKey(druidExtensionDependency)) { - throw new RE( - StringUtils.format( - "%s depends on %s which is not a valid extension or not loaded.", - extension.getName(), - druidExtensionDependency - ) - ); - } - chainedClassLoadersForExtension.add(extensionClassLoaderMap.get(druidExtensionDependency)); - } - ((StandardClassLoader) loader).setExtensionDependencyClassLoaders(chainedClassLoadersForExtension); - log.info( "Loading extension [%s], jars: %s", extension.getName(), @@ -377,34 +405,5 @@ private void tryAdd(T serviceImpl, String extensionType) implsToLoad.add(serviceImpl); } } - - private Optional getExtensionDependencies(URLClassLoader loader) - { - for (URL url : loader.getURLs()) { - File jarFileLocation = new File(url.getPath()); - if (jarFileLocation.getName().startsWith("druid")) { - try (JarFile jarFile = new JarFile(url.getPath())) { - Enumeration entries = jarFile.entries(); - - while (entries.hasMoreElements()) { - JarEntry entry = entries.nextElement(); - String entryName = entry.getName(); - - if (!entry.isDirectory() && EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { - log.info("Found extension dependency entry in druid jar %s", url.getPath()); - return Optional.of(objectMapper.readValue( - jarFile.getInputStream(entry), - ExtensionDependencies.class - )); - } - } - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - } - return Optional.empty(); - } } } diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 3fa93066b2d4..3d4c3d228413 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -101,7 +101,7 @@ public void test06GetClassLoaderForExtension() throws IOException a_jar.createNewFile(); b_jar.createNewFile(); c_jar.createNewFile(); - final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); + final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false, new File[]{}); final URL[] expectedURLs = new URL[]{a_jar.toURI().toURL(), b_jar.toURI().toURL(), c_jar.toURI().toURL()}; final URL[] actualURLs = loader.getURLs(); Arrays.sort(actualURLs, Comparator.comparing(URL::getPath)); @@ -326,8 +326,8 @@ public void testExtensionsWithSameDirName() throws Exception Assert.assertTrue(jar2.createNewFile()); final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); - final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); - final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false); + final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false, new File[]{}); + final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false, new File[]{}); Assert.assertArrayEquals(new URL[]{jar1.toURI().toURL()}, ((URLClassLoader) classLoader1).getURLs()); Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((URLClassLoader) classLoader2).getURLs()); diff --git a/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java b/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java index 30534ccafff7..d4f6f8c6bb62 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java @@ -79,8 +79,9 @@ public void run() } final List extensionURLs = new ArrayList<>(); - for (final File extension : extnLoader.getExtensionFilesToLoad()) { - final URLClassLoader extensionLoader = extnLoader.getClassLoaderForExtension(extension, false); + final File[] extensionFiles = extnLoader.getExtensionFilesToLoad(); + for (final File extension : extensionFiles) { + final URLClassLoader extensionLoader = extnLoader.getClassLoaderForExtension(extension, false, extensionFiles); extensionURLs.addAll(Arrays.asList(extensionLoader.getURLs())); } @@ -90,8 +91,9 @@ public void run() final List driverURLs = new ArrayList<>(nonHadoopURLs); // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts - for (File hadoopDependency : Initialization.getHadoopDependencyFilesToLoad(allCoordinates, extnLoader.config())) { - final URLClassLoader hadoopLoader = extnLoader.getClassLoaderForExtension(hadoopDependency, false); + final File[] hadoopDependencyFiles = Initialization.getHadoopDependencyFilesToLoad(allCoordinates, extnLoader.config()); + for (File hadoopDependency : hadoopDependencyFiles) { + final URLClassLoader hadoopLoader = extnLoader.getClassLoaderForExtension(hadoopDependency, false, hadoopDependencyFiles); driverURLs.addAll(Arrays.asList(hadoopLoader.getURLs())); } From c670eff1f54dab4f9ffb0b9be75bad5ca48183ab Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 12 Sep 2024 10:07:05 -0400 Subject: [PATCH 11/20] generate depenencies on classloader creation --- .../indexing/common/task/HadoopTask.java | 7 +-- ...s.java => DruidExtensionDependencies.java} | 6 +- .../apache/druid/guice/ExtensionsLoader.java | 59 +++++++++++-------- .../druid/guice/ExtensionsLoaderTest.java | 6 +- .../apache/druid/cli/CliHadoopIndexer.java | 7 +-- 5 files changed, 47 insertions(+), 38 deletions(-) rename processing/src/main/java/org/apache/druid/guice/{ExtensionDependencies.java => DruidExtensionDependencies.java} (88%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java index 05d45709dab3..27a86d75ab2a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java @@ -154,9 +154,8 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo } final List extensionURLs = new ArrayList<>(); - final File[] extensionFiles = EXTENSIONS_LOADER.getExtensionFilesToLoad(); - for (final File extension : extensionFiles) { - final URLClassLoader extensionLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(extension, false, extensionFiles); + for (final File extension : EXTENSIONS_LOADER.getExtensionFilesToLoad()) { + final URLClassLoader extensionLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(extension, false); extensionURLs.addAll(Arrays.asList(extensionLoader.getURLs())); } @@ -170,7 +169,7 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo EXTENSIONS_LOADER.config() ); for (final File hadoopDependency : hadoopDependencyFiles) { - final URLClassLoader hadoopLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(hadoopDependency, false, hadoopDependencyFiles); + final URLClassLoader hadoopLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(hadoopDependency, false); localClassLoaderURLs.addAll(Arrays.asList(hadoopLoader.getURLs())); } diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java b/processing/src/main/java/org/apache/druid/guice/DruidExtensionDependencies.java similarity index 88% rename from processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java rename to processing/src/main/java/org/apache/druid/guice/DruidExtensionDependencies.java index 100cc87f08f8..87b587849c83 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionDependencies.java +++ b/processing/src/main/java/org/apache/druid/guice/DruidExtensionDependencies.java @@ -26,18 +26,18 @@ import java.util.ArrayList; import java.util.List; -public class ExtensionDependencies +public class DruidExtensionDependencies { @JsonProperty("dependsOnDruidExtensions") private List dependsOnDruidExtensions; - public ExtensionDependencies() + public DruidExtensionDependencies() { this.dependsOnDruidExtensions = new ArrayList<>(); } - public ExtensionDependencies(@Nonnull final List dependsOnDruidExtensions) + public DruidExtensionDependencies(@Nonnull final List dependsOnDruidExtensions) { this.dependsOnDruidExtensions = dependsOnDruidExtensions; } diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 5aaf1d1590e9..64d5efd4bb65 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import javax.inject.Inject; @@ -44,7 +45,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Enumeration; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; @@ -79,6 +79,8 @@ public class ExtensionsLoader */ private final ConcurrentHashMap, Collection> extensions = new ConcurrentHashMap<>(); + @MonotonicNonNull + private File[] extensionFilesToLoad; @Inject public ExtensionsLoader(ExtensionsConfig config, ObjectMapper objectMapper) { @@ -171,7 +173,7 @@ public Collection getModules() * * @return an array of druid extension files that will be loaded by druid process */ - public File[] getExtensionFilesToLoad() + public void initializeExtensionFilesToLoad() { final File rootExtensionsDir = new File(extensionsConfig.getDirectory()); if (rootExtensionsDir.exists() && !rootExtensionsDir.isDirectory()) { @@ -199,7 +201,14 @@ public File[] getExtensionFilesToLoad() extensionsToLoad[i++] = extensionDir; } } - return extensionsToLoad == null ? new File[]{} : extensionsToLoad; + extensionFilesToLoad = extensionsToLoad == null ? new File[]{} : extensionsToLoad; + } + + public File[] getExtensionFilesToLoad() { + if (extensionFilesToLoad == null) { + initializeExtensionFilesToLoad(); + } + return extensionFilesToLoad; } /** @@ -207,37 +216,41 @@ public File[] getExtensionFilesToLoad() * * @return a URLClassLoader that loads all the jars on which the extension is dependent */ - public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, File[] loadedExtensions) + public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) { - return loaders.computeIfAbsent( - Pair.of(extension, useExtensionClassloaderFirst), - k -> makeClassLoaderWithExtensionDependencies(k.lhs, k.rhs, loadedExtensions) - ); + Pair classLoaderKey = Pair.of(extension, useExtensionClassloaderFirst); + URLClassLoader classLoader = loaders.get(classLoaderKey); + if (classLoader == null) { + classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst); + loaders.put(classLoaderKey, classLoader); + } + + return classLoader; } - private URLClassLoader makeClassLoaderWithExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, File[] loadedExtensions) { + private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst) { URLClassLoader classLoader = makeClassLoaderForExtension(extension, useExtensionClassloaderFirst); - Optional extensionDependencies = getExtensionDependencies(extension); - List extensionDependencyNames = extensionDependencies.isPresent() - ? extensionDependencies.get().getDependsOnDruidExtensions() + Optional druidExtensionDependenciesOptional = getDruidExtensionDependencies(extension); + List druidExtensionDependenciesList = druidExtensionDependenciesOptional.isPresent() + ? druidExtensionDependenciesOptional.get().getDependsOnDruidExtensions() : ImmutableList.of(); List extensionDependencyClassLoaders = new ArrayList<>(); - for (String extensionDependencyName: extensionDependencyNames) { - Optional extensionDependencyFile = Arrays.stream(loadedExtensions) - .filter(file -> file.getName().equals(extensionDependencyName)) - .findAny(); + for (String druidExtensionDependencyName: druidExtensionDependenciesList) { + Optional extensionDependencyFile = Arrays.stream(getExtensionFilesToLoad()) + .filter(file -> file.getName().equals(druidExtensionDependencyName)) + .findFirst(); if (!extensionDependencyFile.isPresent()) { throw new RE( StringUtils.format( "%s depends on %s which is not a valid extension or not loaded.", extension.getName(), - extensionDependencyName + druidExtensionDependencyName ) ); } extensionDependencyClassLoaders.add( - getClassLoaderForExtension(extensionDependencyFile.get(), useExtensionClassloaderFirst, loadedExtensions) + getClassLoaderForExtension(extensionDependencyFile.get(), useExtensionClassloaderFirst) ); } ((StandardClassLoader) classLoader).setExtensionDependencyClassLoaders(extensionDependencyClassLoaders); @@ -307,7 +320,7 @@ public boolean accept(File dir, String name) } } - private Optional getExtensionDependencies(File extension) + private Optional getDruidExtensionDependencies(File extension) { final Collection jars = FileUtils.listFiles(extension, new String[]{"jar"}, false); for (File extensionFile : jars) { @@ -323,7 +336,7 @@ private Optional getExtensionDependencies(File extension) log.info("Found extension dependency entry in druid jar %s", extensionFile.getPath()); return Optional.of(objectMapper.readValue( jarFile.getInputStream(entry), - ExtensionDependencies.class + DruidExtensionDependencies.class )); } } @@ -360,14 +373,12 @@ private void addAllFromCurrentClassLoader() private void addAllFromFileSystem() { - File[] extensionFilesToLoad = getExtensionFilesToLoad(); - for (File extension : extensionFilesToLoad) { + for (File extension : getExtensionFilesToLoad()) { log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { final URLClassLoader loader = getClassLoaderForExtension( extension, - extensionsConfig.isUseExtensionClassloaderFirst(), - extensionFilesToLoad + extensionsConfig.isUseExtensionClassloaderFirst() ); log.info( "Loading extension [%s], jars: %s", diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 3d4c3d228413..3fa93066b2d4 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -101,7 +101,7 @@ public void test06GetClassLoaderForExtension() throws IOException a_jar.createNewFile(); b_jar.createNewFile(); c_jar.createNewFile(); - final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false, new File[]{}); + final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); final URL[] expectedURLs = new URL[]{a_jar.toURI().toURL(), b_jar.toURI().toURL(), c_jar.toURI().toURL()}; final URL[] actualURLs = loader.getURLs(); Arrays.sort(actualURLs, Comparator.comparing(URL::getPath)); @@ -326,8 +326,8 @@ public void testExtensionsWithSameDirName() throws Exception Assert.assertTrue(jar2.createNewFile()); final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); - final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false, new File[]{}); - final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false, new File[]{}); + final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); + final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false); Assert.assertArrayEquals(new URL[]{jar1.toURI().toURL()}, ((URLClassLoader) classLoader1).getURLs()); Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((URLClassLoader) classLoader2).getURLs()); diff --git a/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java b/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java index d4f6f8c6bb62..f1519e503349 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java @@ -79,9 +79,8 @@ public void run() } final List extensionURLs = new ArrayList<>(); - final File[] extensionFiles = extnLoader.getExtensionFilesToLoad(); - for (final File extension : extensionFiles) { - final URLClassLoader extensionLoader = extnLoader.getClassLoaderForExtension(extension, false, extensionFiles); + for (final File extension : extnLoader.getExtensionFilesToLoad()) { + final URLClassLoader extensionLoader = extnLoader.getClassLoaderForExtension(extension, false); extensionURLs.addAll(Arrays.asList(extensionLoader.getURLs())); } @@ -93,7 +92,7 @@ public void run() // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts final File[] hadoopDependencyFiles = Initialization.getHadoopDependencyFilesToLoad(allCoordinates, extnLoader.config()); for (File hadoopDependency : hadoopDependencyFiles) { - final URLClassLoader hadoopLoader = extnLoader.getClassLoaderForExtension(hadoopDependency, false, hadoopDependencyFiles); + final URLClassLoader hadoopLoader = extnLoader.getClassLoaderForExtension(hadoopDependency, false); driverURLs.addAll(Arrays.asList(hadoopLoader.getURLs())); } From 86286fdf7f460d8b022723d43ccdd14eddc92cef Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 12 Sep 2024 11:24:29 -0400 Subject: [PATCH 12/20] add check for circular dependencies --- ...json => druid-extension-dependencies.json} | 0 .../apache/druid/guice/ExtensionsLoader.java | 29 +++++++++++++++---- 2 files changed, 23 insertions(+), 6 deletions(-) rename extensions-core/kafka-extraction-namespace/src/main/resources/{extension-dependencies.json => druid-extension-dependencies.json} (100%) diff --git a/extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json b/extensions-core/kafka-extraction-namespace/src/main/resources/druid-extension-dependencies.json similarity index 100% rename from extensions-core/kafka-extraction-namespace/src/main/resources/extension-dependencies.json rename to extensions-core/kafka-extraction-namespace/src/main/resources/druid-extension-dependencies.json diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 64d5efd4bb65..927c88ff22d9 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -68,7 +68,7 @@ public class ExtensionsLoader { private static final Logger log = new Logger(ExtensionsLoader.class); - public static final String EXTENSION_DEPENDENCIES_JSON = "extension-dependencies.json"; + public static final String EXTENSION_DEPENDENCIES_JSON = "druid-extension-dependencies.json"; private final ExtensionsConfig extensionsConfig; private final ObjectMapper objectMapper; @@ -217,18 +217,23 @@ public File[] getExtensionFilesToLoad() { * @return a URLClassLoader that loads all the jars on which the extension is dependent */ public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) + { + return getClassLoaderForExtension(extension, useExtensionClassloaderFirst, new ArrayList<>()); + } + + public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { Pair classLoaderKey = Pair.of(extension, useExtensionClassloaderFirst); URLClassLoader classLoader = loaders.get(classLoaderKey); if (classLoader == null) { - classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst); + classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst, extensionDependencyStack); loaders.put(classLoaderKey, classLoader); } return classLoader; } - private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst) { + private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { URLClassLoader classLoader = makeClassLoaderForExtension(extension, useExtensionClassloaderFirst); Optional druidExtensionDependenciesOptional = getDruidExtensionDependencies(extension); List druidExtensionDependenciesList = druidExtensionDependenciesOptional.isPresent() @@ -237,10 +242,10 @@ private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extens List extensionDependencyClassLoaders = new ArrayList<>(); for (String druidExtensionDependencyName: druidExtensionDependenciesList) { - Optional extensionDependencyFile = Arrays.stream(getExtensionFilesToLoad()) + Optional extensionDependencyFileOptional = Arrays.stream(getExtensionFilesToLoad()) .filter(file -> file.getName().equals(druidExtensionDependencyName)) .findFirst(); - if (!extensionDependencyFile.isPresent()) { + if (!extensionDependencyFileOptional.isPresent()) { throw new RE( StringUtils.format( "%s depends on %s which is not a valid extension or not loaded.", @@ -249,8 +254,20 @@ private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extens ) ); } + File extensionDependencyFile = extensionDependencyFileOptional.get(); + if (extensionDependencyStack.contains(extensionDependencyFile.getName())) { + extensionDependencyStack.add(extensionDependencyFile.getName()); + throw new RE( + StringUtils.format( + "%s has a circular druid extension dependency. Dependency stack [%s].", + extensionDependencyStack.get(0), + extensionDependencyStack + ) + ); + } + extensionDependencyStack.add(extensionDependencyFile.getName()); extensionDependencyClassLoaders.add( - getClassLoaderForExtension(extensionDependencyFile.get(), useExtensionClassloaderFirst) + getClassLoaderForExtension(extensionDependencyFile, useExtensionClassloaderFirst, extensionDependencyStack) ); } ((StandardClassLoader) classLoader).setExtensionDependencyClassLoaders(extensionDependencyClassLoaders); From 60eff3188e4a9deb2ab1a8c83e7b9dbbce8abb91 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 12 Sep 2024 15:05:52 -0400 Subject: [PATCH 13/20] fix style --- .../apache/druid/guice/ExtensionsLoader.java | 55 +++++++++++-------- 1 file changed, 32 insertions(+), 23 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 927c88ff22d9..fc1cf9026745 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -81,6 +81,7 @@ public class ExtensionsLoader @MonotonicNonNull private File[] extensionFilesToLoad; + @Inject public ExtensionsLoader(ExtensionsConfig config, ObjectMapper objectMapper) { @@ -204,7 +205,8 @@ public void initializeExtensionFilesToLoad() extensionFilesToLoad = extensionsToLoad == null ? new File[]{} : extensionsToLoad; } - public File[] getExtensionFilesToLoad() { + public File[] getExtensionFilesToLoad() + { if (extensionFilesToLoad == null) { initializeExtensionFilesToLoad(); } @@ -233,7 +235,8 @@ public URLClassLoader getClassLoaderForExtension(File extension, boolean useExte return classLoader; } - private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { + private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) + { URLClassLoader classLoader = makeClassLoaderForExtension(extension, useExtensionClassloaderFirst); Optional druidExtensionDependenciesOptional = getDruidExtensionDependencies(extension); List druidExtensionDependenciesList = druidExtensionDependenciesOptional.isPresent() @@ -241,14 +244,14 @@ private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extens : ImmutableList.of(); List extensionDependencyClassLoaders = new ArrayList<>(); - for (String druidExtensionDependencyName: druidExtensionDependenciesList) { + for (String druidExtensionDependencyName : druidExtensionDependenciesList) { Optional extensionDependencyFileOptional = Arrays.stream(getExtensionFilesToLoad()) .filter(file -> file.getName().equals(druidExtensionDependencyName)) .findFirst(); if (!extensionDependencyFileOptional.isPresent()) { throw new RE( StringUtils.format( - "%s depends on %s which is not a valid extension or not loaded.", + "[%s] depends on [%s] which is not a valid extension or not loaded.", extension.getName(), druidExtensionDependencyName ) @@ -259,7 +262,7 @@ private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extens extensionDependencyStack.add(extensionDependencyFile.getName()); throw new RE( StringUtils.format( - "%s has a circular druid extension dependency. Dependency stack [%s].", + "[%s] has a circular druid extension dependency. Dependency stack [%s].", extensionDependencyStack.get(0), extensionDependencyStack ) @@ -340,30 +343,36 @@ public boolean accept(File dir, String name) private Optional getDruidExtensionDependencies(File extension) { final Collection jars = FileUtils.listFiles(extension, new String[]{"jar"}, false); + DruidExtensionDependencies druidExtensionDependencies = null; for (File extensionFile : jars) { - if (extensionFile.getName().startsWith("druid")) { - try (JarFile jarFile = new JarFile(extensionFile.getPath())) { - Enumeration entries = jarFile.entries(); - - while (entries.hasMoreElements()) { - JarEntry entry = entries.nextElement(); - String entryName = entry.getName(); - - if (!entry.isDirectory() && EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { - log.info("Found extension dependency entry in druid jar %s", extensionFile.getPath()); - return Optional.of(objectMapper.readValue( - jarFile.getInputStream(entry), - DruidExtensionDependencies.class - )); + try (JarFile jarFile = new JarFile(extensionFile.getPath())) { + Enumeration entries = jarFile.entries(); + + while (entries.hasMoreElements()) { + JarEntry entry = entries.nextElement(); + String entryName = entry.getName(); + if (EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { + log.debug("Found extension dependency entry in druid jar %s", extensionFile.getPath()); + if (druidExtensionDependencies != null) { + throw new RE( + StringUtils.format( + "The extension [%s] has multiple druid jars with dependencies in it. Each jar should be in a separate extension directory", + extension.getName() + ) + ); } + druidExtensionDependencies = objectMapper.readValue( + jarFile.getInputStream(entry), + DruidExtensionDependencies.class + ); } } - catch (IOException e) { - throw new RuntimeException(e); - } + } + catch (IOException e) { + throw new RuntimeException(e); } } - return Optional.empty(); + return druidExtensionDependencies == null ? Optional.empty() : Optional.of(druidExtensionDependencies); } private class ServiceLoadingFromExtensions From 94eea8149e09cf740fc12bc0b657bd33b874b190 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 12 Sep 2024 15:06:39 -0400 Subject: [PATCH 14/20] revert style changes --- .../apache/druid/indexing/common/task/HadoopTask.java | 10 +++++----- .../java/org/apache/druid/cli/CliHadoopIndexer.java | 3 +-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java index 27a86d75ab2a..201024540e16 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java @@ -164,11 +164,11 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo final List localClassLoaderURLs = new ArrayList<>(jobURLs); // hadoop dependencies come before druid classes because some extensions depend on them - final File[] hadoopDependencyFiles = Initialization.getHadoopDependencyFilesToLoad( - finalHadoopDependencyCoordinates, - EXTENSIONS_LOADER.config() - ); - for (final File hadoopDependency : hadoopDependencyFiles) { + for (final File hadoopDependency : + Initialization.getHadoopDependencyFilesToLoad( + finalHadoopDependencyCoordinates, + EXTENSIONS_LOADER.config() + )) { final URLClassLoader hadoopLoader = EXTENSIONS_LOADER.getClassLoaderForExtension(hadoopDependency, false); localClassLoaderURLs.addAll(Arrays.asList(hadoopLoader.getURLs())); } diff --git a/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java b/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java index f1519e503349..30534ccafff7 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliHadoopIndexer.java @@ -90,8 +90,7 @@ public void run() final List driverURLs = new ArrayList<>(nonHadoopURLs); // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts - final File[] hadoopDependencyFiles = Initialization.getHadoopDependencyFilesToLoad(allCoordinates, extnLoader.config()); - for (File hadoopDependency : hadoopDependencyFiles) { + for (File hadoopDependency : Initialization.getHadoopDependencyFilesToLoad(allCoordinates, extnLoader.config())) { final URLClassLoader hadoopLoader = extnLoader.getClassLoaderForExtension(hadoopDependency, false); driverURLs.addAll(Arrays.asList(hadoopLoader.getURLs())); } From 5cc471dbfaca3733853e4c35496d4cb35bf9d945 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 12 Sep 2024 15:29:58 -0400 Subject: [PATCH 15/20] remove mutable class loader --- .../org/apache/druid/guice/ExtensionsLoader.java | 16 ++++++++-------- .../apache/druid/guice/StandardClassLoader.java | 9 +-------- 2 files changed, 9 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index fc1cf9026745..93ac676cd6f2 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -68,7 +68,7 @@ public class ExtensionsLoader { private static final Logger log = new Logger(ExtensionsLoader.class); - public static final String EXTENSION_DEPENDENCIES_JSON = "druid-extension-dependencies.json"; + public static final String DRUID_EXTENSION_DEPENDENCIES_JSON = "druid-extension-dependencies.json"; private final ExtensionsConfig extensionsConfig; private final ObjectMapper objectMapper; @@ -237,7 +237,6 @@ public URLClassLoader getClassLoaderForExtension(File extension, boolean useExte private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { - URLClassLoader classLoader = makeClassLoaderForExtension(extension, useExtensionClassloaderFirst); Optional druidExtensionDependenciesOptional = getDruidExtensionDependencies(extension); List druidExtensionDependenciesList = druidExtensionDependenciesOptional.isPresent() ? druidExtensionDependenciesOptional.get().getDependsOnDruidExtensions() @@ -273,13 +272,14 @@ private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extens getClassLoaderForExtension(extensionDependencyFile, useExtensionClassloaderFirst, extensionDependencyStack) ); } - ((StandardClassLoader) classLoader).setExtensionDependencyClassLoaders(extensionDependencyClassLoaders); - return classLoader; + + return makeClassLoaderForExtension(extension, useExtensionClassloaderFirst, extensionDependencyClassLoaders); } private static URLClassLoader makeClassLoaderForExtension( final File extension, - final boolean useExtensionClassloaderFirst + final boolean useExtensionClassloaderFirst, + final List extensionDependencyClassLoaders ) { final Collection jars = FileUtils.listFiles(extension, new String[]{"jar"}, false); @@ -298,9 +298,9 @@ private static URLClassLoader makeClassLoaderForExtension( } if (useExtensionClassloaderFirst) { - return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader(), ImmutableList.of()); + return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders); } else { - return new StandardClassLoader(urls, ExtensionsLoader.class.getClassLoader(), ImmutableList.of()); + return new StandardClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders); } } @@ -351,7 +351,7 @@ private Optional getDruidExtensionDependencies(File while (entries.hasMoreElements()) { JarEntry entry = entries.nextElement(); String entryName = entry.getName(); - if (EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { + if (DRUID_EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { log.debug("Found extension dependency entry in druid jar %s", extensionFile.getPath()); if (druidExtensionDependencies != null) { throw new RE( diff --git a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java b/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java index f535bbde908e..f9f536416b8f 100644 --- a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java @@ -22,8 +22,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; -import javax.annotation.Nonnull; - import java.io.IOException; import java.net.URL; import java.net.URLClassLoader; @@ -37,7 +35,7 @@ */ public class StandardClassLoader extends URLClassLoader { - private List extensionDependencyClassLoaders; + private final List extensionDependencyClassLoaders; public StandardClassLoader(final URL[] urls, final ClassLoader druidLoader, final List extensionDependencyClassLoaders) { @@ -113,9 +111,4 @@ protected void addExtensionResources(final String name, List urls) throws I Iterators.addAll(urls, Iterators.forEnumeration(classLoader.getResources(name))); } } - - public void setExtensionDependencyClassLoaders(@Nonnull List extensionDependencyClassLoaders) - { - this.extensionDependencyClassLoaders = extensionDependencyClassLoaders; - } } From 4eea7210e9d37c817f747ec943abaf57d54535f3 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 12 Sep 2024 15:56:02 -0400 Subject: [PATCH 16/20] clean up class heirarchy --- .../guice/ExtensionFirstClassLoader.java | 2 +- .../apache/druid/guice/ExtensionsLoader.java | 24 +++++++++---------- ...oader.java => StandardURLClassLoader.java} | 9 +++++-- .../druid/guice/ExtensionsLoaderTest.java | 9 ++++--- 4 files changed, 24 insertions(+), 20 deletions(-) rename processing/src/main/java/org/apache/druid/guice/{StandardClassLoader.java => StandardURLClassLoader.java} (91%) diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java index ebbf7cfc79bb..4824333af015 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionFirstClassLoader.java @@ -31,7 +31,7 @@ /** * The ClassLoader that gets used when druid.extensions.useExtensionClassloaderFirst = true. */ -public class ExtensionFirstClassLoader extends StandardClassLoader +public class ExtensionFirstClassLoader extends StandardURLClassLoader { private final ClassLoader druidLoader; diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 93ac676cd6f2..89d31787b939 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -39,7 +39,6 @@ import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; -import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -72,7 +71,7 @@ public class ExtensionsLoader private final ExtensionsConfig extensionsConfig; private final ObjectMapper objectMapper; - private final ConcurrentHashMap, URLClassLoader> loaders = new ConcurrentHashMap<>(); + private final ConcurrentHashMap, StandardURLClassLoader> loaders = new ConcurrentHashMap<>(); /** * Map of loaded extensions, keyed by class (or interface). @@ -124,7 +123,7 @@ public Collection getLoadedModules() } @VisibleForTesting - public Map, URLClassLoader> getLoadersMap() + public Map, StandardURLClassLoader> getLoadersMap() { return loaders; } @@ -218,15 +217,15 @@ public File[] getExtensionFilesToLoad() * * @return a URLClassLoader that loads all the jars on which the extension is dependent */ - public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) + public StandardURLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) { return getClassLoaderForExtension(extension, useExtensionClassloaderFirst, new ArrayList<>()); } - public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) + public StandardURLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { Pair classLoaderKey = Pair.of(extension, useExtensionClassloaderFirst); - URLClassLoader classLoader = loaders.get(classLoaderKey); + StandardURLClassLoader classLoader = loaders.get(classLoaderKey); if (classLoader == null) { classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst, extensionDependencyStack); loaders.put(classLoaderKey, classLoader); @@ -235,7 +234,7 @@ public URLClassLoader getClassLoaderForExtension(File extension, boolean useExte return classLoader; } - private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) + private StandardURLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { Optional druidExtensionDependenciesOptional = getDruidExtensionDependencies(extension); List druidExtensionDependenciesList = druidExtensionDependenciesOptional.isPresent() @@ -276,7 +275,7 @@ private URLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extens return makeClassLoaderForExtension(extension, useExtensionClassloaderFirst, extensionDependencyClassLoaders); } - private static URLClassLoader makeClassLoaderForExtension( + private static StandardURLClassLoader makeClassLoaderForExtension( final File extension, final boolean useExtensionClassloaderFirst, final List extensionDependencyClassLoaders @@ -300,7 +299,7 @@ private static URLClassLoader makeClassLoaderForExtension( if (useExtensionClassloaderFirst) { return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders); } else { - return new StandardClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders); + return new StandardURLClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders); } } @@ -402,16 +401,17 @@ private void addAllFromFileSystem() for (File extension : getExtensionFilesToLoad()) { log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { - final URLClassLoader loader = getClassLoaderForExtension( + final StandardURLClassLoader loader = getClassLoaderForExtension( extension, extensionsConfig.isUseExtensionClassloaderFirst() ); log.info( - "Loading extension [%s], jars: %s", + "Loading extension [%s], jars: %s. Druid extension dependencies [%s]", extension.getName(), Arrays.stream(loader.getURLs()) .map(u -> new File(u.getPath()).getName()) - .collect(Collectors.joining(", ")) + .collect(Collectors.joining(", ")), + loader.getExtensionDependencyClassLoaders() ); ServiceLoader.load(serviceClass, loader).forEach(impl -> tryAdd(impl, "local file system")); diff --git a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java b/processing/src/main/java/org/apache/druid/guice/StandardURLClassLoader.java similarity index 91% rename from processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java rename to processing/src/main/java/org/apache/druid/guice/StandardURLClassLoader.java index f9f536416b8f..b7b7c0ed741c 100644 --- a/processing/src/main/java/org/apache/druid/guice/StandardClassLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/StandardURLClassLoader.java @@ -33,11 +33,11 @@ /** * The ClassLoader that gets used when druid.extensions.useExtensionClassloaderFirst = false. */ -public class StandardClassLoader extends URLClassLoader +public class StandardURLClassLoader extends URLClassLoader { private final List extensionDependencyClassLoaders; - public StandardClassLoader(final URL[] urls, final ClassLoader druidLoader, final List extensionDependencyClassLoaders) + public StandardURLClassLoader(final URL[] urls, final ClassLoader druidLoader, final List extensionDependencyClassLoaders) { super(urls, druidLoader); this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders"); @@ -111,4 +111,9 @@ protected void addExtensionResources(final String name, List urls) throws I Iterators.addAll(urls, Iterators.forEnumeration(classLoader.getResources(name))); } } + + public List getExtensionDependencyClassLoaders() + { + return extensionDependencyClassLoaders; + } } diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 3fa93066b2d4..b97fcb58b283 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -34,7 +34,6 @@ import java.io.File; import java.io.IOException; import java.net.URL; -import java.net.URLClassLoader; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; @@ -78,7 +77,7 @@ public void test04DuplicateClassLoaderExtensions() throws Exception Pair key = Pair.of(extensionDir, true); extnLoader.getLoadersMap() - .put(key, new URLClassLoader(new URL[]{}, ExtensionsLoader.class.getClassLoader())); + .put(key, new StandardURLClassLoader(new URL[]{}, ExtensionsLoader.class.getClassLoader(), ImmutableList.of())); Collection modules = extnLoader.getFromExtensions(DruidModule.class); @@ -101,7 +100,7 @@ public void test06GetClassLoaderForExtension() throws IOException a_jar.createNewFile(); b_jar.createNewFile(); c_jar.createNewFile(); - final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); + final StandardURLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); final URL[] expectedURLs = new URL[]{a_jar.toURI().toURL(), b_jar.toURI().toURL(), c_jar.toURI().toURL()}; final URL[] actualURLs = loader.getURLs(); Arrays.sort(actualURLs, Comparator.comparing(URL::getPath)); @@ -329,7 +328,7 @@ public void testExtensionsWithSameDirName() throws Exception final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false); - Assert.assertArrayEquals(new URL[]{jar1.toURI().toURL()}, ((URLClassLoader) classLoader1).getURLs()); - Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((URLClassLoader) classLoader2).getURLs()); + Assert.assertArrayEquals(new URL[]{jar1.toURI().toURL()}, ((StandardURLClassLoader) classLoader1).getURLs()); + Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((StandardURLClassLoader) classLoader2).getURLs()); } } From 8b77d6a792721e55e81c774d4ff45c409653285a Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 13 Sep 2024 10:42:46 -0400 Subject: [PATCH 17/20] extensions loader test working --- .../druid/guice/ExtensionsLoaderTest.java | 27 +++++++++++++++---- 1 file changed, 22 insertions(+), 5 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index b97fcb58b283..96bac7b7d6a8 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -32,6 +32,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; import java.net.URL; import java.util.Arrays; @@ -41,6 +42,8 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Set; +import java.util.jar.JarEntry; +import java.util.jar.JarOutputStream; public class ExtensionsLoaderTest { @@ -97,9 +100,11 @@ public void test06GetClassLoaderForExtension() throws IOException final File a_jar = new File(some_extension_dir, "a.jar"); final File b_jar = new File(some_extension_dir, "b.jar"); final File c_jar = new File(some_extension_dir, "c.jar"); - a_jar.createNewFile(); - b_jar.createNewFile(); - c_jar.createNewFile(); + createNewJar(a_jar); + createNewJar(b_jar); + createNewJar(c_jar); + + final StandardURLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); final URL[] expectedURLs = new URL[]{a_jar.toURI().toURL(), b_jar.toURI().toURL(), c_jar.toURI().toURL()}; final URL[] actualURLs = loader.getURLs(); @@ -321,8 +326,8 @@ public void testExtensionsWithSameDirName() throws Exception final File jar1 = new File(extension1, "jar1.jar"); final File jar2 = new File(extension2, "jar2.jar"); - Assert.assertTrue(jar1.createNewFile()); - Assert.assertTrue(jar2.createNewFile()); + createNewJar(jar1); + createNewJar(jar2); final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); @@ -331,4 +336,16 @@ public void testExtensionsWithSameDirName() throws Exception Assert.assertArrayEquals(new URL[]{jar1.toURI().toURL()}, ((StandardURLClassLoader) classLoader1).getURLs()); Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((StandardURLClassLoader) classLoader2).getURLs()); } + + private void createNewJar(File jarFileLocation) throws IOException { + Assert.assertTrue(jarFileLocation.createNewFile()); + FileOutputStream fos = new FileOutputStream(jarFileLocation.getPath()); + JarOutputStream jarOut = new JarOutputStream(fos); + JarEntry entry = new JarEntry("jar-resource.json"); + jarOut.putNextEntry(entry); + jarOut.write("jar-resource".getBytes()); + jarOut.closeEntry(); + jarOut.close(); + fos.close(); + } } From 5098bdaea59067041b5ce947284b7c0406b8e790 Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 13 Sep 2024 11:56:31 -0400 Subject: [PATCH 18/20] add unit tests --- .../apache/druid/guice/ExtensionsLoader.java | 2 +- .../druid/guice/ExtensionsLoaderTest.java | 143 ++++++++++++++++-- 2 files changed, 134 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 89d31787b939..5820dbb481b2 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -355,7 +355,7 @@ private Optional getDruidExtensionDependencies(File if (druidExtensionDependencies != null) { throw new RE( StringUtils.format( - "The extension [%s] has multiple druid jars with dependencies in it. Each jar should be in a separate extension directory", + "The extension [%s] has multiple druid jars with dependencies in it. Each jar should be in a separate extension directory.", extension.getName() ) ); diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 96bac7b7d6a8..8cf47795597a 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -21,11 +21,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.inject.Injector; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -35,12 +38,14 @@ import java.io.FileOutputStream; import java.io.IOException; import java.net.URL; +import java.nio.charset.Charset; import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.jar.JarEntry; import java.util.jar.JarOutputStream; @@ -51,6 +56,10 @@ public class ExtensionsLoaderTest public final TemporaryFolder temporaryFolder = new TemporaryFolder(); private final ObjectMapper objectMapper = new ObjectMapper(); + private final Map jarFileContents = ImmutableMap.of( + "jar-resource", + "jar-resource-contents".getBytes(Charset.defaultCharset()) + ); private Injector startupInjector() { @@ -100,9 +109,9 @@ public void test06GetClassLoaderForExtension() throws IOException final File a_jar = new File(some_extension_dir, "a.jar"); final File b_jar = new File(some_extension_dir, "b.jar"); final File c_jar = new File(some_extension_dir, "c.jar"); - createNewJar(a_jar); - createNewJar(b_jar); - createNewJar(c_jar); + createNewJar(a_jar, jarFileContents); + createNewJar(b_jar, jarFileContents); + createNewJar(c_jar, jarFileContents); final StandardURLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); @@ -326,8 +335,8 @@ public void testExtensionsWithSameDirName() throws Exception final File jar1 = new File(extension1, "jar1.jar"); final File jar2 = new File(extension2, "jar2.jar"); - createNewJar(jar1); - createNewJar(jar2); + createNewJar(jar1, jarFileContents); + createNewJar(jar2, jarFileContents); final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); @@ -337,15 +346,129 @@ public void testExtensionsWithSameDirName() throws Exception Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((StandardURLClassLoader) classLoader2).getURLs()); } - private void createNewJar(File jarFileLocation) throws IOException { + @Test + public void testGetClassLoaderForExtension_withMissingDependency() throws IOException + { + final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig(), objectMapper); + final String druidExtensionDependency = "other-druid-extension"; + final DruidExtensionDependencies druidExtensionDependencies = new DruidExtensionDependencies(ImmutableList.of(druidExtensionDependency)); + + final File extensionDir = temporaryFolder.newFolder(); + final File extensionJar = new File(extensionDir, "a.jar"); + createNewJar(extensionJar, ImmutableMap.of("druid-extension-dependencies.json", objectMapper.writeValueAsBytes(druidExtensionDependencies))); + + RE exception = Assert.assertThrows(RE.class, () -> { + extnLoader.getClassLoaderForExtension(extensionDir, false); + }); + + Assert.assertEquals( + StringUtils.format("[%s] depends on [%s] which is not a valid extension or not loaded.", extensionDir.getName(), druidExtensionDependency), + exception.getMessage() + ); + } + + @Test + public void testGetClassLoaderForExtension_dependencyLoaded() throws IOException + { + ExtensionsConfig extensionsConfig = new TestExtensionsConfig(temporaryFolder.getRoot().getPath()); + final ExtensionsLoader extnLoader = new ExtensionsLoader(extensionsConfig, objectMapper); + + final File extensionDir = temporaryFolder.newFolder(); + final File extensionJar = new File(extensionDir, "a.jar"); + createNewJar(extensionJar, jarFileContents); + + final File dependentExtensionDir = temporaryFolder.newFolder(); + final File dependentExtensionJar = new File(dependentExtensionDir, "a.jar"); + final DruidExtensionDependencies druidExtensionDependencies = new DruidExtensionDependencies(ImmutableList.of(extensionDir.getName())); + createNewJar(dependentExtensionJar, ImmutableMap.of("druid-extension-dependencies.json", objectMapper.writeValueAsBytes(druidExtensionDependencies))); + + StandardURLClassLoader classLoader = extnLoader.getClassLoaderForExtension(extensionDir, false); + StandardURLClassLoader dependendentClassLoader = extnLoader.getClassLoaderForExtension(dependentExtensionDir, false); + Assert.assertTrue(dependendentClassLoader.getExtensionDependencyClassLoaders().contains(classLoader)); + Assert.assertEquals(0, classLoader.getExtensionDependencyClassLoaders().size()); + + } + + @Test + public void testGetClassLoaderForExtension_circularDependency() throws IOException + { + ExtensionsConfig extensionsConfig = new TestExtensionsConfig(temporaryFolder.getRoot().getPath()); + final ExtensionsLoader extnLoader = new ExtensionsLoader(extensionsConfig, objectMapper); + + final File extensionDir = temporaryFolder.newFolder(); + final File dependentExtensionDir = temporaryFolder.newFolder(); + + final File extensionJar = new File(extensionDir, "a.jar"); + final DruidExtensionDependencies druidExtensionDependencies = new DruidExtensionDependencies(ImmutableList.of(dependentExtensionDir.getName())); + createNewJar(extensionJar, ImmutableMap.of("druid-extension-dependencies.json", objectMapper.writeValueAsBytes(druidExtensionDependencies))); + + final File dependentExtensionJar = new File(dependentExtensionDir, "a.jar"); + final DruidExtensionDependencies druidExtensionDependenciesCircular = new DruidExtensionDependencies(ImmutableList.of(extensionDir.getName())); + createNewJar(dependentExtensionJar, ImmutableMap.of("druid-extension-dependencies.json", objectMapper.writeValueAsBytes(druidExtensionDependenciesCircular))); + + RE exception = Assert.assertThrows(RE.class, () -> { + extnLoader.getClassLoaderForExtension(extensionDir, false); + }); + + Assert.assertTrue(exception.getMessage().contains("has a circular druid extension dependency.")); + } + + @Test + public void testGetClassLoaderForExtension_multipleDruidJars() throws IOException + { + ExtensionsConfig extensionsConfig = new TestExtensionsConfig(temporaryFolder.getRoot().getPath()); + final ExtensionsLoader extnLoader = new ExtensionsLoader(extensionsConfig, objectMapper); + + final File extensionDir = temporaryFolder.newFolder(); + + final File extensionJar = new File(extensionDir, "a.jar"); + final DruidExtensionDependencies druidExtensionDependencies = new DruidExtensionDependencies(ImmutableList.of()); + createNewJar(extensionJar, ImmutableMap.of("druid-extension-dependencies.json", objectMapper.writeValueAsBytes(druidExtensionDependencies))); + + final File extensionJar2 = new File(extensionDir, "b.jar"); + createNewJar(extensionJar2, ImmutableMap.of("druid-extension-dependencies.json", objectMapper.writeValueAsBytes(druidExtensionDependencies))); + + + RE exception = Assert.assertThrows(RE.class, () -> { + extnLoader.getClassLoaderForExtension(extensionDir, false); + }); + + Assert.assertEquals( + exception.getMessage(), + StringUtils.format("The extension [%s] has multiple druid jars with dependencies in it. Each jar should be in a separate extension directory.", extensionDir.getName()) + ); + } + + + + private void createNewJar(File jarFileLocation, Map jarFileContents) throws IOException + { Assert.assertTrue(jarFileLocation.createNewFile()); FileOutputStream fos = new FileOutputStream(jarFileLocation.getPath()); JarOutputStream jarOut = new JarOutputStream(fos); - JarEntry entry = new JarEntry("jar-resource.json"); - jarOut.putNextEntry(entry); - jarOut.write("jar-resource".getBytes()); - jarOut.closeEntry(); + for (Map.Entry fileNameToContents : jarFileContents.entrySet()) { + JarEntry entry = new JarEntry(fileNameToContents.getKey()); + jarOut.putNextEntry(entry); + jarOut.write(fileNameToContents.getValue()); + jarOut.closeEntry(); + } jarOut.close(); fos.close(); } + + private static class TestExtensionsConfig extends ExtensionsConfig + { + final String directory; + + public TestExtensionsConfig(String directory) + { + this.directory = directory; + } + + @Override + public String getDirectory() + { + return directory; + } + } } From 37b2990c641ad6614789cb4eea3d0b964b53d483 Mon Sep 17 00:00:00 2001 From: George Wu Date: Tue, 24 Sep 2024 10:40:38 -0400 Subject: [PATCH 19/20] pr comments --- .../apache/druid/guice/ExtensionsLoader.java | 94 ++++++++++++------- 1 file changed, 61 insertions(+), 33 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 5820dbb481b2..f450f4e8b10f 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Injector; import org.apache.commons.io.FileUtils; import org.apache.druid.initialization.DruidModule; @@ -44,6 +45,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Enumeration; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; @@ -51,7 +53,6 @@ import java.util.Optional; import java.util.ServiceLoader; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.jar.JarEntry; import java.util.jar.JarFile; import java.util.stream.Collectors; @@ -71,13 +72,16 @@ public class ExtensionsLoader private final ExtensionsConfig extensionsConfig; private final ObjectMapper objectMapper; - private final ConcurrentHashMap, StandardURLClassLoader> loaders = new ConcurrentHashMap<>(); + @GuardedBy("this") + private final HashMap, StandardURLClassLoader> loaders = new HashMap<>(); /** * Map of loaded extensions, keyed by class (or interface). */ - private final ConcurrentHashMap, Collection> extensions = new ConcurrentHashMap<>(); + @GuardedBy("this") + private final HashMap, Collection> extensions = new HashMap<>(); + @GuardedBy("this") @MonotonicNonNull private File[] extensionFilesToLoad; @@ -106,12 +110,14 @@ public ExtensionsConfig config() */ public Collection getLoadedImplementations(Class clazz) { - @SuppressWarnings("unchecked") - Collection retVal = (Collection) extensions.get(clazz); - if (retVal == null) { - return Collections.emptySet(); + synchronized (this) { + @SuppressWarnings("unchecked") + Collection retVal = (Collection) extensions.get(clazz); + if (retVal == null) { + return Collections.emptySet(); + } + return retVal; } - return retVal; } /** @@ -125,7 +131,9 @@ public Collection getLoadedModules() @VisibleForTesting public Map, StandardURLClassLoader> getLoadersMap() { - return loaders; + synchronized (this) { + return loaders; + } } /** @@ -149,12 +157,14 @@ public Collection getFromExtensions(Class serviceClass) // In practice, it appears the only place this matters is with DruidModule: // initialization gets the list of extensions, and two REST API calls later // ask for the same list. - Collection modules = extensions.computeIfAbsent( - serviceClass, - serviceC -> new ServiceLoadingFromExtensions<>(serviceC).implsToLoad - ); - //noinspection unchecked - return (Collection) modules; + synchronized (this) { + Collection modules = extensions.computeIfAbsent( + serviceClass, + serviceC -> new ServiceLoadingFromExtensions<>(serviceC).implsToLoad + ); + //noinspection unchecked + return (Collection) modules; + } } public Collection getModules() @@ -201,37 +211,51 @@ public void initializeExtensionFilesToLoad() extensionsToLoad[i++] = extensionDir; } } - extensionFilesToLoad = extensionsToLoad == null ? new File[]{} : extensionsToLoad; + synchronized (this) { + extensionFilesToLoad = extensionsToLoad == null ? new File[]{} : extensionsToLoad; + } } public File[] getExtensionFilesToLoad() { - if (extensionFilesToLoad == null) { - initializeExtensionFilesToLoad(); + synchronized (this) { + if (extensionFilesToLoad == null) { + initializeExtensionFilesToLoad(); + } + return extensionFilesToLoad; } - return extensionFilesToLoad; } /** * @param extension The File instance of the extension we want to load * - * @return a URLClassLoader that loads all the jars on which the extension is dependent + * @return a StandardURLClassLoader that loads all the jars on which the extension is dependent */ public StandardURLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) { return getClassLoaderForExtension(extension, useExtensionClassloaderFirst, new ArrayList<>()); } + /** + * @param extension The File instance of the extension we want to load + * @param useExtensionClassloaderFirst Whether to return a StandardURLClassLoader that checks extension classloaders first for classes + * @param extensionDependencyStack If the extension is requested as a dependency of another extension, a list containing the + * dependency stack of the dependent extension (for checking circular dependencies). Otherwise + * this is a empty list. + * @return a StandardURLClassLoader that loads all the jars on which the extension is dependent + */ public StandardURLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) { Pair classLoaderKey = Pair.of(extension, useExtensionClassloaderFirst); - StandardURLClassLoader classLoader = loaders.get(classLoaderKey); - if (classLoader == null) { - classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst, extensionDependencyStack); - loaders.put(classLoaderKey, classLoader); - } + synchronized (this) { + StandardURLClassLoader classLoader = loaders.get(classLoaderKey); + if (classLoader == null) { + classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst, extensionDependencyStack); + loaders.put(classLoaderKey, classLoader); + } - return classLoader; + return classLoader; + } } private StandardURLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List extensionDependencyStack) @@ -249,7 +273,7 @@ private StandardURLClassLoader makeClassLoaderWithDruidExtensionDependencies(Fil if (!extensionDependencyFileOptional.isPresent()) { throw new RE( StringUtils.format( - "[%s] depends on [%s] which is not a valid extension or not loaded.", + "Extension [%s] depends on [%s] which is not a valid extension or not loaded.", extension.getName(), druidExtensionDependencyName ) @@ -260,7 +284,7 @@ private StandardURLClassLoader makeClassLoaderWithDruidExtensionDependencies(Fil extensionDependencyStack.add(extensionDependencyFile.getName()); throw new RE( StringUtils.format( - "[%s] has a circular druid extension dependency. Dependency stack [%s].", + "Extension [%s] has a circular druid extension dependency. Dependency stack [%s].", extensionDependencyStack.get(0), extensionDependencyStack ) @@ -343,6 +367,7 @@ private Optional getDruidExtensionDependencies(File { final Collection jars = FileUtils.listFiles(extension, new String[]{"jar"}, false); DruidExtensionDependencies druidExtensionDependencies = null; + String druidExtensionDependenciesJarName = null; for (File extensionFile : jars) { try (JarFile jarFile = new JarFile(extensionFile.getPath())) { Enumeration entries = jarFile.entries(); @@ -351,12 +376,14 @@ private Optional getDruidExtensionDependencies(File JarEntry entry = entries.nextElement(); String entryName = entry.getName(); if (DRUID_EXTENSION_DEPENDENCIES_JSON.equals(entryName)) { - log.debug("Found extension dependency entry in druid jar %s", extensionFile.getPath()); - if (druidExtensionDependencies != null) { + log.debug("Found extension dependency entry in jar [%s]", extensionFile.getPath()); + if (druidExtensionDependenciesJarName != null) { throw new RE( StringUtils.format( - "The extension [%s] has multiple druid jars with dependencies in it. Each jar should be in a separate extension directory.", - extension.getName() + "The extension [%s] has multiple jars [%s] [%s] with dependencies in them. Each jar should be in a separate extension directory.", + extension.getName(), + druidExtensionDependenciesJarName, + jarFile.getName() ) ); } @@ -364,11 +391,12 @@ private Optional getDruidExtensionDependencies(File jarFile.getInputStream(entry), DruidExtensionDependencies.class ); + druidExtensionDependenciesJarName = jarFile.getName(); } } } catch (IOException e) { - throw new RuntimeException(e); + throw new RE(e, "Failed to get dependencies for extension [%s]", extension); } } return druidExtensionDependencies == null ? Optional.empty() : Optional.of(druidExtensionDependencies); From 10c4d80207bb1f7baf7ffb59f64bb4221b83ebc4 Mon Sep 17 00:00:00 2001 From: George Wu Date: Tue, 24 Sep 2024 12:08:22 -0400 Subject: [PATCH 20/20] fix unit tests --- .../java/org/apache/druid/guice/ExtensionsLoaderTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 8cf47795597a..2e9be1741007 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -362,7 +362,7 @@ public void testGetClassLoaderForExtension_withMissingDependency() throws IOExce }); Assert.assertEquals( - StringUtils.format("[%s] depends on [%s] which is not a valid extension or not loaded.", extensionDir.getName(), druidExtensionDependency), + StringUtils.format("Extension [%s] depends on [%s] which is not a valid extension or not loaded.", extensionDir.getName(), druidExtensionDependency), exception.getMessage() ); } @@ -433,9 +433,8 @@ public void testGetClassLoaderForExtension_multipleDruidJars() throws IOExceptio extnLoader.getClassLoaderForExtension(extensionDir, false); }); - Assert.assertEquals( - exception.getMessage(), - StringUtils.format("The extension [%s] has multiple druid jars with dependencies in it. Each jar should be in a separate extension directory.", extensionDir.getName()) + Assert.assertTrue( + exception.getMessage().contains("Each jar should be in a separate extension directory.") ); }