C0urante commented on code in PR #14064: URL: https://github.com/apache/kafka/pull/14064#discussion_r1281978798
########## tools/src/main/java/org/apache/kafka/tools/ConnectPluginPath.java: ########## @@ -0,0 +1,498 @@ +/* + * 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.kafka.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.impl.Arguments; +import net.sourceforge.argparse4j.inf.ArgumentGroup; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.isolation.ClassLoaderFactory; +import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; +import org.apache.kafka.connect.runtime.isolation.PluginDesc; +import org.apache.kafka.connect.runtime.isolation.PluginScanResult; +import org.apache.kafka.connect.runtime.isolation.PluginSource; +import org.apache.kafka.connect.runtime.isolation.PluginType; +import org.apache.kafka.connect.runtime.isolation.PluginUtils; +import org.apache.kafka.connect.runtime.isolation.ReflectionScanner; +import org.apache.kafka.connect.runtime.isolation.ServiceLoaderScanner; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ConnectPluginPath { + + private static final String MANIFEST_PREFIX = "META-INF/services/"; + private static final int LIST_TABLE_COLUMN_COUNT = 8; + + public static void main(String[] args) { + Exit.exit(mainNoExit(args, System.out, System.err)); + } + + public static int mainNoExit(String[] args, PrintStream out, PrintStream err) { + ArgumentParser parser = parser(); + try { + Namespace namespace = parser.parseArgs(args); + Config config = parseConfig(parser, namespace, out); + runCommand(config); + return 0; + } catch (ArgumentParserException e) { + parser.handleError(e); + return 1; + } catch (TerseException e) { + err.println(e.getMessage()); + return 2; + } catch (Throwable e) { + err.println(e.getMessage()); + err.println(Utils.stackTrace(e)); + return 3; + } + } + + private static ArgumentParser parser() { + ArgumentParser parser = ArgumentParsers.newArgumentParser("connect-plugin-path") + .defaultHelp(true) + .description("Manage plugins on the Connect plugin.path"); + + ArgumentParser listCommand = parser.addSubparsers() + .description("List information about plugins contained within the specified plugin locations") + .dest("subcommand") + .addParser("list"); + + ArgumentParser[] subparsers = new ArgumentParser[] { + listCommand, + }; + + for (ArgumentParser subparser : subparsers) { + ArgumentGroup pluginProviders = subparser.addArgumentGroup("plugin providers"); + pluginProviders.addArgument("--plugin-location") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A single plugin location (jar file or directory)"); + + pluginProviders.addArgument("--plugin-path") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A comma-delimited list of locations containing plugins"); + + pluginProviders.addArgument("--worker-config") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A Connect worker configuration file"); + } + + return parser; + } + + private static Config parseConfig(ArgumentParser parser, Namespace namespace, PrintStream out) throws ArgumentParserException, TerseException { + Set<Path> locations = parseLocations(parser, namespace); + String subcommand = namespace.getString("subcommand"); + if (subcommand == null) { + throw new ArgumentParserException("No subcommand specified", parser); + } + switch (subcommand) { + case "list": + return new Config(Command.LIST, locations, out); + default: + throw new ArgumentParserException("Unrecognized subcommand: '" + subcommand + "'", parser); + } + } + + private static Set<Path> parseLocations(ArgumentParser parser, Namespace namespace) throws ArgumentParserException, TerseException { + List<String> rawLocations = new ArrayList<>(namespace.getList("plugin_location")); + List<String> rawPluginPaths = new ArrayList<>(namespace.getList("plugin_path")); + List<String> rawWorkerConfigs = new ArrayList<>(namespace.getList("worker_config")); + if (rawLocations.isEmpty() && rawPluginPaths.isEmpty() && rawWorkerConfigs.isEmpty()) { + throw new ArgumentParserException("Must specify at least one --plugin-location, --plugin-path, or --worker-config", parser); + } + Set<Path> pluginLocations = new HashSet<>(); + for (String rawWorkerConfig : rawWorkerConfigs) { + Properties properties; + try { + properties = Utils.loadProps(rawWorkerConfig); + } catch (IOException e) { + throw new TerseException("Unable to read worker config at " + rawWorkerConfig); + } + String pluginPath = properties.getProperty(WorkerConfig.PLUGIN_PATH_CONFIG); + if (pluginPath != null) { + rawPluginPaths.add(pluginPath); + } + } + for (String rawPluginPath : rawPluginPaths) { + try { + pluginLocations.addAll(PluginUtils.pluginLocations(rawPluginPath, true)); + } catch (UncheckedIOException e) { + throw new TerseException("Unable to parse plugin path " + rawPluginPath + ": " + e.getMessage()); + } + } + for (String rawLocation : rawLocations) { + Path pluginLocation = Paths.get(rawLocation); + if (!pluginLocation.toFile().exists()) { + throw new TerseException("Specified location " + pluginLocation + " does not exist"); + } + pluginLocations.add(pluginLocation); + } + return pluginLocations; + } + + enum Command { + LIST + } + + private static class Config { + private final Command command; + private final Set<Path> locations; + private final PrintStream out; + + private Config(Command command, Set<Path> locations, PrintStream out) { + this.command = command; + this.locations = locations; + this.out = out; + } + + @Override + public String toString() { + return "Config{" + + "command=" + command + + ", locations=" + locations + + '}'; + } + } + + public static void runCommand(Config config) throws TerseException { + try { + ClassLoader parent = ConnectPluginPath.class.getClassLoader(); + ServiceLoaderScanner serviceLoaderScanner = new ServiceLoaderScanner(); + ReflectionScanner reflectionScanner = new ReflectionScanner(); + // Process the contents of the classpath to exclude it from later results. + PluginSource classpathSource = PluginUtils.classpathPluginSource(parent); + Map<String, Set<ManifestEntry>> classpathManifests = findManifests(classpathSource, Collections.emptyMap()); + PluginScanResult classpathPlugins = discoverPlugins(classpathSource, reflectionScanner, serviceLoaderScanner); + Map<Path, Set<Row>> rowsByLocation = new LinkedHashMap<>(); + Set<Row> classpathRows = enumerateRows(classpathSource, classpathManifests, classpathPlugins); + rowsByLocation.put(classpathSource.location(), classpathRows); + + ClassLoaderFactory factory = new ClassLoaderFactory(); + try (DelegatingClassLoader delegatingClassLoader = factory.newDelegatingClassLoader(parent)) { + beginCommand(config); + for (Path pluginLocation : config.locations) { + PluginSource source = PluginUtils.isolatedPluginSource(pluginLocation, delegatingClassLoader, factory); + Map<String, Set<ManifestEntry>> manifests = findManifests(source, classpathManifests); + PluginScanResult plugins = discoverPlugins(source, reflectionScanner, serviceLoaderScanner); + Set<Row> rows = enumerateRows(source, manifests, plugins); + rowsByLocation.put(pluginLocation, rows); + for (Row row : rows) { + handlePlugin(config, row); + } + } + endCommand(config, rowsByLocation); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * The unit of work for a command. + * <p>This is unique to the (source, class, type) tuple, and contains additional pre-computed information + * that pertains to this specific plugin. + */ + private static class Row { + private final Path pluginLocation; + private final String className; + private final PluginType type; + private final String version; + private final List<String> aliases; + private final boolean loadable; + private final boolean hasManifest; + + public Row(Path pluginLocation, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) { + this.pluginLocation = pluginLocation; + this.className = className; + this.version = version; + this.type = type; + this.aliases = aliases; + this.loadable = loadable; + this.hasManifest = hasManifest; + } + + private boolean loadable() { + return loadable; + } + + private boolean compatible() { + return loadable && hasManifest; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Row row = (Row) o; + return pluginLocation.equals(row.pluginLocation) && className.equals(row.className) && type == row.type; + } + + @Override + public int hashCode() { + return Objects.hash(pluginLocation, className, type); + } + } + + private static Set<Row> enumerateRows(PluginSource source, Map<String, Set<ManifestEntry>> manifests, PluginScanResult scanResult) { + Set<Row> rows = new HashSet<>(); + // Perform a deep copy of the manifests because we're going to be mutating our copy. + Map<String, Set<ManifestEntry>> unloadablePlugins = manifests.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new HashSet<>(e.getValue()))); + scanResult.forEach(pluginDesc -> { + // Emit a loadable row for this scan result, since it was found during plugin discovery + rows.add(newRow(source, pluginDesc.className(), pluginDesc.type(), pluginDesc.version(), true, manifests)); + // Remove the ManifestEntry if it has the same className and type as one of the loadable plugins. + unloadablePlugins.getOrDefault(pluginDesc.className(), Collections.emptySet()).removeIf(entry -> entry.type == pluginDesc.type()); + }); + unloadablePlugins.values().forEach(entries -> entries.forEach(entry -> { + // Emit a non-loadable row, since all the loadable rows showed up in the previous iteration. + // Two ManifestEntries may produce the same row if they have different URIs + rows.add(newRow(source, entry.className, entry.type, PluginDesc.UNDEFINED_VERSION, false, manifests)); + })); + return rows; + } + + private static Row newRow(PluginSource source, String className, PluginType type, String version, boolean loadable, Map<String, Set<ManifestEntry>> manifests) { + Set<String> rowAliases = new LinkedHashSet<>(); + rowAliases.add(PluginUtils.simpleName(className)); + rowAliases.add(PluginUtils.prunedName(className, type)); + boolean hasManifest = manifests.containsKey(className); + return new Row(source.location(), className, type, version, new ArrayList<>(rowAliases), loadable, hasManifest); + } + + private static void beginCommand(Config config) { + if (config.command == Command.LIST) { + listTablePrint(config, + "pluginName", + "firstAlias", + "secondAlias", + "pluginVersion", + "pluginType", + "isLoadable", + "hasManifest", + "pluginLocation" // last because it is least important and most repetitive + ); + } + } + + private static void handlePlugin(Config config, Row row) { + if (config.command == Command.LIST) { + String firstAlias = row.aliases.size() > 0 ? row.aliases.get(0) : "null"; + String secondAlias = row.aliases.size() > 1 ? row.aliases.get(1) : "null"; Review Comment: This isn't very human-readable, how about "N/A"? ```suggestion String firstAlias = row.aliases.size() > 0 ? row.aliases.get(0) : "N/A"; String secondAlias = row.aliases.size() > 1 ? row.aliases.get(1) : "N/A"; ``` ########## tools/src/main/java/org/apache/kafka/tools/ConnectPluginPath.java: ########## @@ -0,0 +1,498 @@ +/* + * 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.kafka.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.impl.Arguments; +import net.sourceforge.argparse4j.inf.ArgumentGroup; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.isolation.ClassLoaderFactory; +import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; +import org.apache.kafka.connect.runtime.isolation.PluginDesc; +import org.apache.kafka.connect.runtime.isolation.PluginScanResult; +import org.apache.kafka.connect.runtime.isolation.PluginSource; +import org.apache.kafka.connect.runtime.isolation.PluginType; +import org.apache.kafka.connect.runtime.isolation.PluginUtils; +import org.apache.kafka.connect.runtime.isolation.ReflectionScanner; +import org.apache.kafka.connect.runtime.isolation.ServiceLoaderScanner; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ConnectPluginPath { + + private static final String MANIFEST_PREFIX = "META-INF/services/"; + private static final int LIST_TABLE_COLUMN_COUNT = 8; + + public static void main(String[] args) { + Exit.exit(mainNoExit(args, System.out, System.err)); + } + + public static int mainNoExit(String[] args, PrintStream out, PrintStream err) { + ArgumentParser parser = parser(); + try { + Namespace namespace = parser.parseArgs(args); + Config config = parseConfig(parser, namespace, out); + runCommand(config); + return 0; + } catch (ArgumentParserException e) { + parser.handleError(e); + return 1; + } catch (TerseException e) { + err.println(e.getMessage()); + return 2; + } catch (Throwable e) { + err.println(e.getMessage()); + err.println(Utils.stackTrace(e)); + return 3; + } + } + + private static ArgumentParser parser() { + ArgumentParser parser = ArgumentParsers.newArgumentParser("connect-plugin-path") + .defaultHelp(true) + .description("Manage plugins on the Connect plugin.path"); + + ArgumentParser listCommand = parser.addSubparsers() + .description("List information about plugins contained within the specified plugin locations") + .dest("subcommand") + .addParser("list"); + + ArgumentParser[] subparsers = new ArgumentParser[] { + listCommand, + }; + + for (ArgumentParser subparser : subparsers) { + ArgumentGroup pluginProviders = subparser.addArgumentGroup("plugin providers"); + pluginProviders.addArgument("--plugin-location") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A single plugin location (jar file or directory)"); + + pluginProviders.addArgument("--plugin-path") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A comma-delimited list of locations containing plugins"); + + pluginProviders.addArgument("--worker-config") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A Connect worker configuration file"); + } + + return parser; + } + + private static Config parseConfig(ArgumentParser parser, Namespace namespace, PrintStream out) throws ArgumentParserException, TerseException { + Set<Path> locations = parseLocations(parser, namespace); + String subcommand = namespace.getString("subcommand"); + if (subcommand == null) { + throw new ArgumentParserException("No subcommand specified", parser); + } + switch (subcommand) { + case "list": + return new Config(Command.LIST, locations, out); + default: + throw new ArgumentParserException("Unrecognized subcommand: '" + subcommand + "'", parser); + } + } + + private static Set<Path> parseLocations(ArgumentParser parser, Namespace namespace) throws ArgumentParserException, TerseException { + List<String> rawLocations = new ArrayList<>(namespace.getList("plugin_location")); + List<String> rawPluginPaths = new ArrayList<>(namespace.getList("plugin_path")); + List<String> rawWorkerConfigs = new ArrayList<>(namespace.getList("worker_config")); + if (rawLocations.isEmpty() && rawPluginPaths.isEmpty() && rawWorkerConfigs.isEmpty()) { + throw new ArgumentParserException("Must specify at least one --plugin-location, --plugin-path, or --worker-config", parser); + } + Set<Path> pluginLocations = new HashSet<>(); + for (String rawWorkerConfig : rawWorkerConfigs) { + Properties properties; + try { + properties = Utils.loadProps(rawWorkerConfig); + } catch (IOException e) { + throw new TerseException("Unable to read worker config at " + rawWorkerConfig); + } + String pluginPath = properties.getProperty(WorkerConfig.PLUGIN_PATH_CONFIG); + if (pluginPath != null) { + rawPluginPaths.add(pluginPath); + } + } + for (String rawPluginPath : rawPluginPaths) { + try { + pluginLocations.addAll(PluginUtils.pluginLocations(rawPluginPath, true)); + } catch (UncheckedIOException e) { + throw new TerseException("Unable to parse plugin path " + rawPluginPath + ": " + e.getMessage()); + } + } + for (String rawLocation : rawLocations) { + Path pluginLocation = Paths.get(rawLocation); + if (!pluginLocation.toFile().exists()) { + throw new TerseException("Specified location " + pluginLocation + " does not exist"); + } + pluginLocations.add(pluginLocation); + } + return pluginLocations; + } + + enum Command { + LIST + } + + private static class Config { + private final Command command; + private final Set<Path> locations; + private final PrintStream out; + + private Config(Command command, Set<Path> locations, PrintStream out) { + this.command = command; + this.locations = locations; + this.out = out; + } + + @Override + public String toString() { + return "Config{" + + "command=" + command + + ", locations=" + locations + + '}'; + } + } + + public static void runCommand(Config config) throws TerseException { + try { + ClassLoader parent = ConnectPluginPath.class.getClassLoader(); + ServiceLoaderScanner serviceLoaderScanner = new ServiceLoaderScanner(); + ReflectionScanner reflectionScanner = new ReflectionScanner(); + // Process the contents of the classpath to exclude it from later results. + PluginSource classpathSource = PluginUtils.classpathPluginSource(parent); + Map<String, Set<ManifestEntry>> classpathManifests = findManifests(classpathSource, Collections.emptyMap()); + PluginScanResult classpathPlugins = discoverPlugins(classpathSource, reflectionScanner, serviceLoaderScanner); + Map<Path, Set<Row>> rowsByLocation = new LinkedHashMap<>(); + Set<Row> classpathRows = enumerateRows(classpathSource, classpathManifests, classpathPlugins); + rowsByLocation.put(classpathSource.location(), classpathRows); + + ClassLoaderFactory factory = new ClassLoaderFactory(); + try (DelegatingClassLoader delegatingClassLoader = factory.newDelegatingClassLoader(parent)) { + beginCommand(config); + for (Path pluginLocation : config.locations) { + PluginSource source = PluginUtils.isolatedPluginSource(pluginLocation, delegatingClassLoader, factory); + Map<String, Set<ManifestEntry>> manifests = findManifests(source, classpathManifests); + PluginScanResult plugins = discoverPlugins(source, reflectionScanner, serviceLoaderScanner); + Set<Row> rows = enumerateRows(source, manifests, plugins); + rowsByLocation.put(pluginLocation, rows); + for (Row row : rows) { + handlePlugin(config, row); + } + } + endCommand(config, rowsByLocation); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * The unit of work for a command. + * <p>This is unique to the (source, class, type) tuple, and contains additional pre-computed information + * that pertains to this specific plugin. + */ + private static class Row { + private final Path pluginLocation; + private final String className; + private final PluginType type; + private final String version; + private final List<String> aliases; + private final boolean loadable; + private final boolean hasManifest; + + public Row(Path pluginLocation, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) { + this.pluginLocation = pluginLocation; + this.className = className; + this.version = version; + this.type = type; + this.aliases = aliases; + this.loadable = loadable; + this.hasManifest = hasManifest; + } + + private boolean loadable() { + return loadable; + } + + private boolean compatible() { + return loadable && hasManifest; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Row row = (Row) o; + return pluginLocation.equals(row.pluginLocation) && className.equals(row.className) && type == row.type; + } + + @Override + public int hashCode() { + return Objects.hash(pluginLocation, className, type); + } + } + + private static Set<Row> enumerateRows(PluginSource source, Map<String, Set<ManifestEntry>> manifests, PluginScanResult scanResult) { + Set<Row> rows = new HashSet<>(); + // Perform a deep copy of the manifests because we're going to be mutating our copy. + Map<String, Set<ManifestEntry>> unloadablePlugins = manifests.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new HashSet<>(e.getValue()))); + scanResult.forEach(pluginDesc -> { + // Emit a loadable row for this scan result, since it was found during plugin discovery + rows.add(newRow(source, pluginDesc.className(), pluginDesc.type(), pluginDesc.version(), true, manifests)); + // Remove the ManifestEntry if it has the same className and type as one of the loadable plugins. + unloadablePlugins.getOrDefault(pluginDesc.className(), Collections.emptySet()).removeIf(entry -> entry.type == pluginDesc.type()); + }); + unloadablePlugins.values().forEach(entries -> entries.forEach(entry -> { + // Emit a non-loadable row, since all the loadable rows showed up in the previous iteration. + // Two ManifestEntries may produce the same row if they have different URIs + rows.add(newRow(source, entry.className, entry.type, PluginDesc.UNDEFINED_VERSION, false, manifests)); + })); + return rows; + } + + private static Row newRow(PluginSource source, String className, PluginType type, String version, boolean loadable, Map<String, Set<ManifestEntry>> manifests) { + Set<String> rowAliases = new LinkedHashSet<>(); + rowAliases.add(PluginUtils.simpleName(className)); + rowAliases.add(PluginUtils.prunedName(className, type)); + boolean hasManifest = manifests.containsKey(className); + return new Row(source.location(), className, type, version, new ArrayList<>(rowAliases), loadable, hasManifest); + } + + private static void beginCommand(Config config) { + if (config.command == Command.LIST) { + listTablePrint(config, + "pluginName", + "firstAlias", + "secondAlias", + "pluginVersion", + "pluginType", + "isLoadable", + "hasManifest", + "pluginLocation" // last because it is least important and most repetitive + ); + } + } + + private static void handlePlugin(Config config, Row row) { + if (config.command == Command.LIST) { + String firstAlias = row.aliases.size() > 0 ? row.aliases.get(0) : "null"; + String secondAlias = row.aliases.size() > 1 ? row.aliases.get(1) : "null"; + listTablePrint(config, + row.className, + firstAlias, + secondAlias, + row.version, + row.type, + row.loadable, + row.hasManifest, + row.pluginLocation // last because it is least important and most repetitive + ); + } + } + + private static void endCommand( + Config config, + Map<Path, Set<Row>> rowsByLocation + ) { + if (config.command == Command.LIST) { + // end the table with an empty line + config.out.println(); + Set<Row> allRows = rowsByLocation.values().stream().flatMap(Set::stream).collect(Collectors.toSet()); + Map<String, Set<String>> aliasCollisions = aliasCollisions(allRows); + for (Map.Entry<String, Set<String>> entry : aliasCollisions.entrySet()) { + String alias = entry.getKey(); + Set<String> classNames = entry.getValue(); + if (classNames.size() != 1) { + config.out.printf("Ignoring ambiguous alias '%s' since it refers to multiple distinct plugins %s%n", Review Comment: I know this is the language that workers use when warning users of colliding aliases on startup, but it doesn't really make sense here if we're not actually using these aliases in this script (besides printing them to the user). What are we actually trying to convey to users with this message? I was thinking it could be something like "NOTE: It will not be possible to use the plugin alias $ALIAS in connector configurations as it refers to multiple plugins: $PLUGIN_LIST". ########## tools/src/main/java/org/apache/kafka/tools/ConnectPluginPath.java: ########## @@ -0,0 +1,498 @@ +/* + * 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.kafka.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.impl.Arguments; +import net.sourceforge.argparse4j.inf.ArgumentGroup; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.isolation.ClassLoaderFactory; +import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; +import org.apache.kafka.connect.runtime.isolation.PluginDesc; +import org.apache.kafka.connect.runtime.isolation.PluginScanResult; +import org.apache.kafka.connect.runtime.isolation.PluginSource; +import org.apache.kafka.connect.runtime.isolation.PluginType; +import org.apache.kafka.connect.runtime.isolation.PluginUtils; +import org.apache.kafka.connect.runtime.isolation.ReflectionScanner; +import org.apache.kafka.connect.runtime.isolation.ServiceLoaderScanner; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ConnectPluginPath { + + private static final String MANIFEST_PREFIX = "META-INF/services/"; + private static final int LIST_TABLE_COLUMN_COUNT = 8; + + public static void main(String[] args) { + Exit.exit(mainNoExit(args, System.out, System.err)); + } + + public static int mainNoExit(String[] args, PrintStream out, PrintStream err) { + ArgumentParser parser = parser(); + try { + Namespace namespace = parser.parseArgs(args); + Config config = parseConfig(parser, namespace, out); + runCommand(config); + return 0; + } catch (ArgumentParserException e) { + parser.handleError(e); + return 1; + } catch (TerseException e) { + err.println(e.getMessage()); + return 2; + } catch (Throwable e) { + err.println(e.getMessage()); + err.println(Utils.stackTrace(e)); + return 3; + } + } + + private static ArgumentParser parser() { + ArgumentParser parser = ArgumentParsers.newArgumentParser("connect-plugin-path") + .defaultHelp(true) + .description("Manage plugins on the Connect plugin.path"); + + ArgumentParser listCommand = parser.addSubparsers() + .description("List information about plugins contained within the specified plugin locations") + .dest("subcommand") + .addParser("list"); + + ArgumentParser[] subparsers = new ArgumentParser[] { + listCommand, + }; + + for (ArgumentParser subparser : subparsers) { + ArgumentGroup pluginProviders = subparser.addArgumentGroup("plugin providers"); + pluginProviders.addArgument("--plugin-location") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A single plugin location (jar file or directory)"); + + pluginProviders.addArgument("--plugin-path") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A comma-delimited list of locations containing plugins"); + + pluginProviders.addArgument("--worker-config") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A Connect worker configuration file"); + } + + return parser; + } + + private static Config parseConfig(ArgumentParser parser, Namespace namespace, PrintStream out) throws ArgumentParserException, TerseException { + Set<Path> locations = parseLocations(parser, namespace); + String subcommand = namespace.getString("subcommand"); + if (subcommand == null) { + throw new ArgumentParserException("No subcommand specified", parser); + } + switch (subcommand) { + case "list": + return new Config(Command.LIST, locations, out); + default: + throw new ArgumentParserException("Unrecognized subcommand: '" + subcommand + "'", parser); + } + } + + private static Set<Path> parseLocations(ArgumentParser parser, Namespace namespace) throws ArgumentParserException, TerseException { + List<String> rawLocations = new ArrayList<>(namespace.getList("plugin_location")); + List<String> rawPluginPaths = new ArrayList<>(namespace.getList("plugin_path")); + List<String> rawWorkerConfigs = new ArrayList<>(namespace.getList("worker_config")); + if (rawLocations.isEmpty() && rawPluginPaths.isEmpty() && rawWorkerConfigs.isEmpty()) { + throw new ArgumentParserException("Must specify at least one --plugin-location, --plugin-path, or --worker-config", parser); + } + Set<Path> pluginLocations = new HashSet<>(); + for (String rawWorkerConfig : rawWorkerConfigs) { + Properties properties; + try { + properties = Utils.loadProps(rawWorkerConfig); + } catch (IOException e) { + throw new TerseException("Unable to read worker config at " + rawWorkerConfig); + } + String pluginPath = properties.getProperty(WorkerConfig.PLUGIN_PATH_CONFIG); + if (pluginPath != null) { + rawPluginPaths.add(pluginPath); + } + } + for (String rawPluginPath : rawPluginPaths) { + try { + pluginLocations.addAll(PluginUtils.pluginLocations(rawPluginPath, true)); + } catch (UncheckedIOException e) { + throw new TerseException("Unable to parse plugin path " + rawPluginPath + ": " + e.getMessage()); + } + } + for (String rawLocation : rawLocations) { + Path pluginLocation = Paths.get(rawLocation); + if (!pluginLocation.toFile().exists()) { + throw new TerseException("Specified location " + pluginLocation + " does not exist"); + } + pluginLocations.add(pluginLocation); + } + return pluginLocations; + } + + enum Command { + LIST + } + + private static class Config { + private final Command command; + private final Set<Path> locations; + private final PrintStream out; + + private Config(Command command, Set<Path> locations, PrintStream out) { + this.command = command; + this.locations = locations; + this.out = out; + } + + @Override + public String toString() { + return "Config{" + + "command=" + command + + ", locations=" + locations + + '}'; + } + } + + public static void runCommand(Config config) throws TerseException { + try { + ClassLoader parent = ConnectPluginPath.class.getClassLoader(); + ServiceLoaderScanner serviceLoaderScanner = new ServiceLoaderScanner(); + ReflectionScanner reflectionScanner = new ReflectionScanner(); + // Process the contents of the classpath to exclude it from later results. + PluginSource classpathSource = PluginUtils.classpathPluginSource(parent); + Map<String, Set<ManifestEntry>> classpathManifests = findManifests(classpathSource, Collections.emptyMap()); + PluginScanResult classpathPlugins = discoverPlugins(classpathSource, reflectionScanner, serviceLoaderScanner); + Map<Path, Set<Row>> rowsByLocation = new LinkedHashMap<>(); + Set<Row> classpathRows = enumerateRows(classpathSource, classpathManifests, classpathPlugins); + rowsByLocation.put(classpathSource.location(), classpathRows); + + ClassLoaderFactory factory = new ClassLoaderFactory(); + try (DelegatingClassLoader delegatingClassLoader = factory.newDelegatingClassLoader(parent)) { + beginCommand(config); + for (Path pluginLocation : config.locations) { + PluginSource source = PluginUtils.isolatedPluginSource(pluginLocation, delegatingClassLoader, factory); + Map<String, Set<ManifestEntry>> manifests = findManifests(source, classpathManifests); + PluginScanResult plugins = discoverPlugins(source, reflectionScanner, serviceLoaderScanner); + Set<Row> rows = enumerateRows(source, manifests, plugins); + rowsByLocation.put(pluginLocation, rows); + for (Row row : rows) { + handlePlugin(config, row); + } + } + endCommand(config, rowsByLocation); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * The unit of work for a command. + * <p>This is unique to the (source, class, type) tuple, and contains additional pre-computed information + * that pertains to this specific plugin. + */ + private static class Row { + private final Path pluginLocation; + private final String className; + private final PluginType type; + private final String version; + private final List<String> aliases; + private final boolean loadable; + private final boolean hasManifest; + + public Row(Path pluginLocation, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) { + this.pluginLocation = pluginLocation; + this.className = className; + this.version = version; + this.type = type; + this.aliases = aliases; + this.loadable = loadable; + this.hasManifest = hasManifest; + } + + private boolean loadable() { + return loadable; + } + + private boolean compatible() { + return loadable && hasManifest; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Row row = (Row) o; + return pluginLocation.equals(row.pluginLocation) && className.equals(row.className) && type == row.type; + } + + @Override + public int hashCode() { + return Objects.hash(pluginLocation, className, type); + } + } + + private static Set<Row> enumerateRows(PluginSource source, Map<String, Set<ManifestEntry>> manifests, PluginScanResult scanResult) { + Set<Row> rows = new HashSet<>(); + // Perform a deep copy of the manifests because we're going to be mutating our copy. + Map<String, Set<ManifestEntry>> unloadablePlugins = manifests.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new HashSet<>(e.getValue()))); + scanResult.forEach(pluginDesc -> { + // Emit a loadable row for this scan result, since it was found during plugin discovery + rows.add(newRow(source, pluginDesc.className(), pluginDesc.type(), pluginDesc.version(), true, manifests)); + // Remove the ManifestEntry if it has the same className and type as one of the loadable plugins. + unloadablePlugins.getOrDefault(pluginDesc.className(), Collections.emptySet()).removeIf(entry -> entry.type == pluginDesc.type()); + }); + unloadablePlugins.values().forEach(entries -> entries.forEach(entry -> { + // Emit a non-loadable row, since all the loadable rows showed up in the previous iteration. + // Two ManifestEntries may produce the same row if they have different URIs + rows.add(newRow(source, entry.className, entry.type, PluginDesc.UNDEFINED_VERSION, false, manifests)); + })); + return rows; + } + + private static Row newRow(PluginSource source, String className, PluginType type, String version, boolean loadable, Map<String, Set<ManifestEntry>> manifests) { + Set<String> rowAliases = new LinkedHashSet<>(); + rowAliases.add(PluginUtils.simpleName(className)); + rowAliases.add(PluginUtils.prunedName(className, type)); + boolean hasManifest = manifests.containsKey(className); + return new Row(source.location(), className, type, version, new ArrayList<>(rowAliases), loadable, hasManifest); + } + + private static void beginCommand(Config config) { + if (config.command == Command.LIST) { + listTablePrint(config, + "pluginName", + "firstAlias", + "secondAlias", + "pluginVersion", + "pluginType", + "isLoadable", + "hasManifest", + "pluginLocation" // last because it is least important and most repetitive + ); Review Comment: We can capture these in a class constant: ```java private static final List<String> LIST_TABLE_COLUMNS = Arrays.asList( "pluginName", "firstAlias", "secondAlias", "pluginVersion", "pluginType", "isLoadable", "hasManifest", "pluginLocation" // last because it is least important and most repetitive ); ``` and then use them here with the no-args `List::toArray` method: ```suggestion listTablePrint(config, LIST_TABLE_COLUMNS.toArray()); ``` and then compare `ConnectPluginPath.LIST_TABLE_COLUMNS.size()` instead of `ConnectPluginPath.LIST_TABLE_COLUMNS` to the number of varargs passed in to `listTablePrint`. (The advantage is that we don't have to manually maintain a constant for the number of columns in each row.) ########## tools/src/main/java/org/apache/kafka/tools/ConnectPluginPath.java: ########## @@ -0,0 +1,498 @@ +/* + * 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.kafka.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.impl.Arguments; +import net.sourceforge.argparse4j.inf.ArgumentGroup; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.isolation.ClassLoaderFactory; +import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; +import org.apache.kafka.connect.runtime.isolation.PluginDesc; +import org.apache.kafka.connect.runtime.isolation.PluginScanResult; +import org.apache.kafka.connect.runtime.isolation.PluginSource; +import org.apache.kafka.connect.runtime.isolation.PluginType; +import org.apache.kafka.connect.runtime.isolation.PluginUtils; +import org.apache.kafka.connect.runtime.isolation.ReflectionScanner; +import org.apache.kafka.connect.runtime.isolation.ServiceLoaderScanner; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ConnectPluginPath { + + private static final String MANIFEST_PREFIX = "META-INF/services/"; + private static final int LIST_TABLE_COLUMN_COUNT = 8; + + public static void main(String[] args) { + Exit.exit(mainNoExit(args, System.out, System.err)); + } + + public static int mainNoExit(String[] args, PrintStream out, PrintStream err) { + ArgumentParser parser = parser(); + try { + Namespace namespace = parser.parseArgs(args); + Config config = parseConfig(parser, namespace, out); + runCommand(config); + return 0; + } catch (ArgumentParserException e) { + parser.handleError(e); + return 1; + } catch (TerseException e) { + err.println(e.getMessage()); + return 2; + } catch (Throwable e) { + err.println(e.getMessage()); + err.println(Utils.stackTrace(e)); + return 3; + } + } + + private static ArgumentParser parser() { + ArgumentParser parser = ArgumentParsers.newArgumentParser("connect-plugin-path") + .defaultHelp(true) + .description("Manage plugins on the Connect plugin.path"); + + ArgumentParser listCommand = parser.addSubparsers() + .description("List information about plugins contained within the specified plugin locations") + .dest("subcommand") + .addParser("list"); + + ArgumentParser[] subparsers = new ArgumentParser[] { + listCommand, + }; + + for (ArgumentParser subparser : subparsers) { + ArgumentGroup pluginProviders = subparser.addArgumentGroup("plugin providers"); + pluginProviders.addArgument("--plugin-location") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A single plugin location (jar file or directory)"); + + pluginProviders.addArgument("--plugin-path") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A comma-delimited list of locations containing plugins"); + + pluginProviders.addArgument("--worker-config") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A Connect worker configuration file"); + } + + return parser; + } + + private static Config parseConfig(ArgumentParser parser, Namespace namespace, PrintStream out) throws ArgumentParserException, TerseException { + Set<Path> locations = parseLocations(parser, namespace); + String subcommand = namespace.getString("subcommand"); + if (subcommand == null) { + throw new ArgumentParserException("No subcommand specified", parser); + } + switch (subcommand) { + case "list": + return new Config(Command.LIST, locations, out); + default: + throw new ArgumentParserException("Unrecognized subcommand: '" + subcommand + "'", parser); + } + } + + private static Set<Path> parseLocations(ArgumentParser parser, Namespace namespace) throws ArgumentParserException, TerseException { + List<String> rawLocations = new ArrayList<>(namespace.getList("plugin_location")); + List<String> rawPluginPaths = new ArrayList<>(namespace.getList("plugin_path")); + List<String> rawWorkerConfigs = new ArrayList<>(namespace.getList("worker_config")); + if (rawLocations.isEmpty() && rawPluginPaths.isEmpty() && rawWorkerConfigs.isEmpty()) { + throw new ArgumentParserException("Must specify at least one --plugin-location, --plugin-path, or --worker-config", parser); + } + Set<Path> pluginLocations = new HashSet<>(); + for (String rawWorkerConfig : rawWorkerConfigs) { + Properties properties; + try { + properties = Utils.loadProps(rawWorkerConfig); + } catch (IOException e) { + throw new TerseException("Unable to read worker config at " + rawWorkerConfig); + } + String pluginPath = properties.getProperty(WorkerConfig.PLUGIN_PATH_CONFIG); + if (pluginPath != null) { + rawPluginPaths.add(pluginPath); + } + } + for (String rawPluginPath : rawPluginPaths) { + try { + pluginLocations.addAll(PluginUtils.pluginLocations(rawPluginPath, true)); + } catch (UncheckedIOException e) { + throw new TerseException("Unable to parse plugin path " + rawPluginPath + ": " + e.getMessage()); + } + } + for (String rawLocation : rawLocations) { + Path pluginLocation = Paths.get(rawLocation); + if (!pluginLocation.toFile().exists()) { + throw new TerseException("Specified location " + pluginLocation + " does not exist"); + } + pluginLocations.add(pluginLocation); + } + return pluginLocations; + } + + enum Command { + LIST + } + + private static class Config { + private final Command command; + private final Set<Path> locations; + private final PrintStream out; + + private Config(Command command, Set<Path> locations, PrintStream out) { + this.command = command; + this.locations = locations; + this.out = out; + } + + @Override + public String toString() { + return "Config{" + + "command=" + command + + ", locations=" + locations + + '}'; + } + } + + public static void runCommand(Config config) throws TerseException { + try { + ClassLoader parent = ConnectPluginPath.class.getClassLoader(); + ServiceLoaderScanner serviceLoaderScanner = new ServiceLoaderScanner(); + ReflectionScanner reflectionScanner = new ReflectionScanner(); + // Process the contents of the classpath to exclude it from later results. + PluginSource classpathSource = PluginUtils.classpathPluginSource(parent); + Map<String, Set<ManifestEntry>> classpathManifests = findManifests(classpathSource, Collections.emptyMap()); + PluginScanResult classpathPlugins = discoverPlugins(classpathSource, reflectionScanner, serviceLoaderScanner); + Map<Path, Set<Row>> rowsByLocation = new LinkedHashMap<>(); + Set<Row> classpathRows = enumerateRows(classpathSource, classpathManifests, classpathPlugins); + rowsByLocation.put(classpathSource.location(), classpathRows); + + ClassLoaderFactory factory = new ClassLoaderFactory(); + try (DelegatingClassLoader delegatingClassLoader = factory.newDelegatingClassLoader(parent)) { + beginCommand(config); + for (Path pluginLocation : config.locations) { + PluginSource source = PluginUtils.isolatedPluginSource(pluginLocation, delegatingClassLoader, factory); + Map<String, Set<ManifestEntry>> manifests = findManifests(source, classpathManifests); + PluginScanResult plugins = discoverPlugins(source, reflectionScanner, serviceLoaderScanner); + Set<Row> rows = enumerateRows(source, manifests, plugins); + rowsByLocation.put(pluginLocation, rows); + for (Row row : rows) { + handlePlugin(config, row); + } + } + endCommand(config, rowsByLocation); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * The unit of work for a command. + * <p>This is unique to the (source, class, type) tuple, and contains additional pre-computed information + * that pertains to this specific plugin. + */ + private static class Row { + private final Path pluginLocation; + private final String className; + private final PluginType type; + private final String version; + private final List<String> aliases; + private final boolean loadable; + private final boolean hasManifest; + + public Row(Path pluginLocation, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) { + this.pluginLocation = pluginLocation; + this.className = className; + this.version = version; + this.type = type; + this.aliases = aliases; + this.loadable = loadable; + this.hasManifest = hasManifest; + } + + private boolean loadable() { + return loadable; + } + + private boolean compatible() { + return loadable && hasManifest; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Row row = (Row) o; + return pluginLocation.equals(row.pluginLocation) && className.equals(row.className) && type == row.type; + } + + @Override + public int hashCode() { + return Objects.hash(pluginLocation, className, type); + } + } + + private static Set<Row> enumerateRows(PluginSource source, Map<String, Set<ManifestEntry>> manifests, PluginScanResult scanResult) { + Set<Row> rows = new HashSet<>(); + // Perform a deep copy of the manifests because we're going to be mutating our copy. + Map<String, Set<ManifestEntry>> unloadablePlugins = manifests.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new HashSet<>(e.getValue()))); + scanResult.forEach(pluginDesc -> { + // Emit a loadable row for this scan result, since it was found during plugin discovery + rows.add(newRow(source, pluginDesc.className(), pluginDesc.type(), pluginDesc.version(), true, manifests)); + // Remove the ManifestEntry if it has the same className and type as one of the loadable plugins. + unloadablePlugins.getOrDefault(pluginDesc.className(), Collections.emptySet()).removeIf(entry -> entry.type == pluginDesc.type()); + }); + unloadablePlugins.values().forEach(entries -> entries.forEach(entry -> { + // Emit a non-loadable row, since all the loadable rows showed up in the previous iteration. + // Two ManifestEntries may produce the same row if they have different URIs + rows.add(newRow(source, entry.className, entry.type, PluginDesc.UNDEFINED_VERSION, false, manifests)); + })); + return rows; + } + + private static Row newRow(PluginSource source, String className, PluginType type, String version, boolean loadable, Map<String, Set<ManifestEntry>> manifests) { + Set<String> rowAliases = new LinkedHashSet<>(); + rowAliases.add(PluginUtils.simpleName(className)); + rowAliases.add(PluginUtils.prunedName(className, type)); + boolean hasManifest = manifests.containsKey(className); + return new Row(source.location(), className, type, version, new ArrayList<>(rowAliases), loadable, hasManifest); + } + + private static void beginCommand(Config config) { + if (config.command == Command.LIST) { + listTablePrint(config, + "pluginName", + "firstAlias", + "secondAlias", + "pluginVersion", + "pluginType", + "isLoadable", + "hasManifest", + "pluginLocation" // last because it is least important and most repetitive + ); + } + } + + private static void handlePlugin(Config config, Row row) { + if (config.command == Command.LIST) { + String firstAlias = row.aliases.size() > 0 ? row.aliases.get(0) : "null"; + String secondAlias = row.aliases.size() > 1 ? row.aliases.get(1) : "null"; + listTablePrint(config, + row.className, + firstAlias, + secondAlias, + row.version, + row.type, + row.loadable, + row.hasManifest, + row.pluginLocation // last because it is least important and most repetitive + ); + } + } + + private static void endCommand( + Config config, + Map<Path, Set<Row>> rowsByLocation + ) { + if (config.command == Command.LIST) { + // end the table with an empty line Review Comment: This explains what we're doing, but not why. Someone might read the comment and then remove both it and the empty line without understanding the impact on users. Can we mention that the idea here is to be friendly with tools that may want to parse the rows we've printed above? ########## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java: ########## @@ -197,15 +202,23 @@ public static boolean isClassFile(Path path) { return path.toString().toLowerCase(Locale.ROOT).endsWith(".class"); } - public static Set<Path> pluginLocations(String pluginPath) { + public static Set<Path> pluginLocations(String pluginPath, boolean failFast) { if (pluginPath == null) { return Collections.emptySet(); } String[] pluginPathElements = COMMA_WITH_WHITESPACE.split(pluginPath.trim(), -1); Set<Path> pluginLocations = new LinkedHashSet<>(); for (String path : pluginPathElements) { try { - Path pluginPathElement = Paths.get(path).toAbsolutePath(); + Path specifiedPath = Paths.get(path); + Path pluginPathElement = specifiedPath.toAbsolutePath(); + if (!specifiedPath.isAbsolute()) { + log.warn("Plugin path element '{}' is relative, evaluating to {}.", + specifiedPath, pluginPathElement); + } Review Comment: I don't think we need to warn people when any relative path is specified. The only case I think we need to warn about is when an empty (but non-null) path is given. The way relative paths work is fairly intuitive; the way Connect workers and this CLI work with an empty path is less so (and is more likely to come from user error). ```suggestion Path pluginPathElement = Paths.get(pluginPath).toAbsolutePath(); if (pluginPath.isEmpty()) { log.warn("Plugin path element is empty, evaluating to current directory {}", pluginPathElement); } ``` ########## tools/src/main/java/org/apache/kafka/tools/ConnectPluginPath.java: ########## @@ -0,0 +1,399 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * 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.kafka.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.impl.Arguments; +import net.sourceforge.argparse4j.inf.ArgumentGroup; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.isolation.ClassLoaderFactory; +import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; +import org.apache.kafka.connect.runtime.isolation.PluginDesc; +import org.apache.kafka.connect.runtime.isolation.PluginScanResult; +import org.apache.kafka.connect.runtime.isolation.PluginSource; +import org.apache.kafka.connect.runtime.isolation.PluginType; +import org.apache.kafka.connect.runtime.isolation.PluginUtils; +import org.apache.kafka.connect.runtime.isolation.ReflectionScanner; +import org.apache.kafka.connect.runtime.isolation.ServiceLoaderScanner; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +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.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +public class ConnectPluginPath { + + private static final String MANIFEST_PREFIX = "META-INF/services/"; + + public static void main(String[] args) { + Exit.exit(mainNoExit(args, System.out, System.err)); + } + + public static int mainNoExit(String[] args, PrintStream out, PrintStream err) { + ArgumentParser parser = parser(); + try { + Namespace namespace = parser.parseArgs(args); + Config config = parseConfig(parser, namespace, out, err); + runCommand(config); + return 0; + } catch (ArgumentParserException e) { + parser.handleError(e); + return 1; + } catch (TerseException e) { + err.println(e.getMessage()); + return 2; + } catch (Throwable e) { + err.println(e.getMessage()); + err.println(Utils.stackTrace(e)); + return 3; + } + } + + private static ArgumentParser parser() { + ArgumentParser parser = ArgumentParsers.newArgumentParser("connect-plugin-path") + .defaultHelp(true) + .description("Manage plugins on the Connect plugin.path"); + + ArgumentParser listCommand = parser.addSubparsers() + .description("List information about plugins contained within the specified plugin locations") + .dest("subcommand") + .addParser("list"); + + ArgumentParser[] subparsers = new ArgumentParser[] { + listCommand, + }; + + for (ArgumentParser subparser : subparsers) { + ArgumentGroup pluginProviders = subparser.addArgumentGroup("plugin providers"); + pluginProviders.addArgument("--plugin-location") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A single plugin location (jar file or directory)"); + + pluginProviders.addArgument("--plugin-path") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A list of locations containing plugins"); + + pluginProviders.addArgument("--worker-config") + .setDefault(new ArrayList<>()) + .action(Arguments.append()) + .help("A Connect worker configuration file"); + } + + return parser; + } + + private static Config parseConfig(ArgumentParser parser, Namespace namespace, PrintStream out, PrintStream err) throws ArgumentParserException, TerseException { + Set<Path> locations = parseLocations(parser, namespace); + switch (namespace.getString("subcommand")) { + case "list": + return new Config(Command.LIST, locations, out, err); + default: + throw new ArgumentParserException("No subcommand specified", parser); + } + } + + private static Set<Path> parseLocations(ArgumentParser parser, Namespace namespace) throws ArgumentParserException, TerseException { + List<String> rawLocations = new ArrayList<>(namespace.getList("plugin_location")); + List<String> rawPluginPaths = new ArrayList<>(namespace.getList("plugin_path")); + List<String> rawWorkerConfigs = new ArrayList<>(namespace.getList("worker_config")); + if (rawLocations.isEmpty() && rawPluginPaths.isEmpty() && rawWorkerConfigs.isEmpty()) { + throw new ArgumentParserException("Must specify at least one --plugin-location, --plugin-path, or --worker-config", parser); + } + Set<Path> pluginLocations = new HashSet<>(); + for (String rawWorkerConfig : rawWorkerConfigs) { + Properties properties; + try { + properties = Utils.loadProps(rawWorkerConfig); + } catch (IOException e) { + throw new TerseException("Unable to read worker config at " + rawWorkerConfig); + } + String pluginPath = properties.getProperty(WorkerConfig.PLUGIN_PATH_CONFIG); + if (pluginPath != null) { + rawPluginPaths.add(pluginPath); + } + } + for (String rawPluginPath : rawPluginPaths) { + try { + pluginLocations.addAll(PluginUtils.pluginLocations(rawPluginPath, true)); + } catch (UncheckedIOException e) { + throw new TerseException("Unable to parse plugin path " + rawPluginPath + ": " + e.getMessage()); + } + } + for (String rawLocation : rawLocations) { + pluginLocations.add(Paths.get(rawLocation)); + } + return pluginLocations; + } + + enum Command { + LIST + } + + private static class Config { + private final Command command; + private final Set<Path> locations; + private final PrintStream out; + private final PrintStream err; + + private Config(Command command, Set<Path> locations, PrintStream out, PrintStream err) { + this.command = command; + this.locations = locations; + this.out = out; + this.err = err; + } + + @Override + public String toString() { + return "Config{" + + "command=" + command + + ", locations=" + locations + + '}'; + } + } + + public static void runCommand(Config config) throws TerseException { + try { + for (Path pluginLocation : config.locations) { + ClassLoaderFactory factory = new ClassLoaderFactory(); + try (DelegatingClassLoader delegatingClassLoader = factory.newDelegatingClassLoader(ConnectPluginPath.class.getClassLoader())) { + Set<PluginSource> pluginSources = PluginUtils.pluginSources(Collections.singletonList(pluginLocation), delegatingClassLoader, factory); + Predicate<PluginSource> isolated = PluginSource::isolated; + PluginSource isolatedSource = pluginSources.stream() + .filter(isolated) + .findFirst() + .orElseThrow(() -> new IllegalStateException("DelegatingClassLoader should have a PluginSource corresponding to" + pluginLocation)); + PluginSource classpathSource = pluginSources.stream() + .filter(isolated.negate()) + .findFirst() + .orElseThrow(() -> new IllegalStateException("DelegatingClassLoader should have a PluginSource corresponding to the classpath")); + PluginScanResult scanResult = new ReflectionScanner().discoverPlugins(Collections.singleton(isolatedSource)); + PluginScanResult serviceLoadResult = new ServiceLoaderScanner().discoverPlugins(Collections.singleton(isolatedSource)); + PluginScanResult merged = new PluginScanResult(Arrays.asList(scanResult, serviceLoadResult)); + Set<ManifestEntry> isolatedManifests = findManifests(isolatedSource); + Set<ManifestEntry> classpathManifests = findManifests(classpathSource); + classpathManifests.forEach(isolatedManifests::remove); + Map<String, Set<ManifestEntry>> indexedManifests = indexManifests(isolatedManifests); + Map<String, List<String>> aliases = invertAliases(PluginUtils.computeAliases(merged)); + Map<String, Set<ManifestEntry>> unloadablePlugins = new HashMap<>(indexedManifests); + merged.forEach(pluginDesc -> unloadablePlugins.remove(pluginDesc.className())); + merged.forEach(pluginDesc -> operation(config, isolatedSource, aliases, indexedManifests, pluginDesc.className(), pluginDesc.version(), pluginDesc.type(), true)); + for (Set<ManifestEntry> manifestEntries : unloadablePlugins.values()) { + manifestEntries.forEach(manifestEntry -> operation(config, isolatedSource, aliases, indexedManifests, manifestEntry.className, PluginDesc.UNDEFINED_VERSION, manifestEntry.type, false)); + } + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static <T> void operation( + // fixed + Config config, + // each location + PluginSource source, + Map<String, List<String>> aliases, + Map<String, Set<ManifestEntry>> manifests, + // each plugin + String pluginName, + String pluginVersion, + PluginType pluginType, + boolean loadable + ) { + operation(config, source, pluginName, pluginVersion, aliases.getOrDefault(pluginName, Collections.emptyList()), pluginType, loadable, manifests.containsKey(pluginName)); + } + + private static <T> void operation( + // fixed + Config config, + // each location + PluginSource source, + // each plugin + String pluginName, + String pluginVersion, + List<String> aliases, + PluginType pluginType, + boolean loadable, + boolean hasManifest + ) { + Path pluginLocation = source.location(); + if (config.command == Command.LIST) { + String firstAlias = aliases.size() > 0 ? aliases.get(0) : "null"; + String secondAlias = aliases.size() > 1 ? aliases.get(1) : "null"; + // there should only be one location, otherwise the plugin source column will be ambiguous. + config.out.printf( + "%s\t%s\t%s\t%s\t%s\t%s\t%b\t%b%n", + pluginLocation, + pluginName, + firstAlias, + secondAlias, + pluginVersion, + pluginType, + loadable, + hasManifest + ); + } + } + + private static class ManifestEntry { + private final URI manifestURI; + private final String className; + private final PluginType type; + + private ManifestEntry(URI manifestURI, String className, PluginType type) { + this.manifestURI = manifestURI; + this.className = className; + this.type = type; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ManifestEntry that = (ManifestEntry) o; + return manifestURI.equals(that.manifestURI) && className.equals(that.className) && type == that.type; + } + + @Override + public int hashCode() { + return Objects.hash(manifestURI, className, type); + } + } + + private static Set<ManifestEntry> findManifests(PluginSource source) throws IOException { + Set<ManifestEntry> manifests = new HashSet<>(); + for (PluginType type : PluginType.values()) { + if (type == PluginType.UNKNOWN) { + continue; + } + try { + Enumeration<URL> resources = source.loader().getResources(MANIFEST_PREFIX + type.typeName()); + while (resources.hasMoreElements()) { + URL url = resources.nextElement(); + for (String className : parse(url)) { + manifests.add(new ManifestEntry(url.toURI(), className, type)); + } + } + } catch (URISyntaxException e) { + throw new IOException(e); + } + } + return manifests; + } + + // Based on implementation from ServiceLoader.LazyClassPathLookupIterator Review Comment: Can we add that info to the comment? If bugs are reported for the implementation that we're adapting, and they also affect our modified version, it may make it easier to connect the dots in the future. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org