diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index c4c44a8ed3ffe..08c782b8be5a1 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -43,6 +43,16 @@ metadataStoreOperationTimeoutSeconds: 30 # Metadata store cache expiry time in seconds metadataStoreCacheExpirySeconds: 300 +# Specifies if the function worker should use classloading for validating submissions for built-in +# connectors and functions. This is required for validateConnectorConfig to take effect. +# Default is false. +enableClassloadingOfBuiltinFiles: false + +# Specifies if the function worker should use classloading for validating submissions for external +# connectors and functions. This is required for validateConnectorConfig to take effect. +# Default is false. +enableClassloadingOfExternalFiles: false + ################################ # Function package management ################################ @@ -393,7 +403,10 @@ saslJaasServerRoleTokenSignerSecretPath: connectorsDirectory: ./connectors functionsDirectory: ./functions -# Should connector config be validated during submission +# Enables extended validation for connector config with fine-grain annotation based validation +# during submission. Classloading with either enableClassloadingOfExternalFiles or +# enableClassloadingOfBuiltinFiles must be enabled on the worker for this to take effect. +# Default is false. validateConnectorConfig: false # Whether to initialize distributed log metadata by runtime. diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 85d08f2fd38de..bd8ec10ef7a0f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -485,6 +485,8 @@ The Apache Software License, Version 2.0 - io.grpc-grpc-services-1.55.3.jar - io.grpc-grpc-xds-1.55.3.jar - io.grpc-grpc-rls-1.55.3.jar + - io.grpc-grpc-servlet-1.55.3.jar + - io.grpc-grpc-servlet-jakarta-1.55.3.jar - com.google.auto.service-auto-service-annotations-1.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar @@ -495,6 +497,10 @@ The Apache Software License, Version 2.0 * Jodah - net.jodah-typetools-0.5.0.jar - net.jodah-failsafe-2.4.4.jar + * Byte Buddy + - net.bytebuddy-byte-buddy-1.14.12.jar + * zt-zip + - org.zeroturnaround-zt-zip-1.17.jar * Apache Avro - org.apache.avro-avro-1.11.3.jar - org.apache.avro-avro-protobuf-1.11.3.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 36aa60f91224c..064b0499f99c2 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -345,22 +345,22 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.93.Final.jar - - netty-codec-4.1.93.Final.jar - - netty-codec-dns-4.1.93.Final.jar - - netty-codec-http-4.1.93.Final.jar - - netty-codec-socks-4.1.93.Final.jar - - netty-codec-haproxy-4.1.93.Final.jar - - netty-common-4.1.93.Final.jar - - netty-handler-4.1.93.Final.jar - - netty-handler-proxy-4.1.93.Final.jar - - netty-resolver-4.1.93.Final.jar - - netty-resolver-dns-4.1.93.Final.jar - - netty-transport-4.1.93.Final.jar - - netty-transport-classes-epoll-4.1.93.Final.jar - - netty-transport-native-epoll-4.1.93.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.93.Final.jar - - netty-transport-native-unix-common-4.1.93.Final-linux-x86_64.jar + - netty-buffer-4.1.100.Final.jar + - netty-codec-4.1.100.Final.jar + - netty-codec-dns-4.1.100.Final.jar + - netty-codec-http-4.1.100.Final.jar + - netty-codec-socks-4.1.100.Final.jar + - netty-codec-haproxy-4.1.100.Final.jar + - netty-common-4.1.100.Final.jar + - netty-handler-4.1.100.Final.jar + - netty-handler-proxy-4.1.100.Final.jar + - netty-resolver-4.1.100.Final.jar + - netty-resolver-dns-4.1.100.Final.jar + - netty-transport-4.1.100.Final.jar + - netty-transport-classes-epoll-4.1.100.Final.jar + - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.100.Final.jar + - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -402,8 +402,8 @@ The Apache Software License, Version 2.0 * Google Error Prone Annotations - error_prone_annotations-2.5.1.jar * Javassist -- javassist-3.25.0-GA.jar * Apache Avro - - avro-1.10.2.jar - - avro-protobuf-1.10.2.jar + - avro-1.11.3.jar + - avro-protobuf-1.11.3.jar BSD 3-clause "New" or "Revised" License * JSR305 -- jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt diff --git a/pom.xml b/pom.xml index c58d4ea90b28b..f037c9ef0e520 100644 --- a/pom.xml +++ b/pom.xml @@ -143,6 +143,8 @@ flexible messaging model and an intuitive client API. 8.37 1.4.13 0.5.0 + 1.14.12 + 1.17 3.19.6 ${protobuf3.version} 1.55.3 @@ -978,6 +980,18 @@ flexible messaging model and an intuitive client API. ${typetools.version} + + net.bytebuddy + byte-buddy + ${byte-buddy.version} + + + + org.zeroturnaround + zt-zip + ${zt-zip.version} + + io.grpc grpc-bom diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java index c4864883788d9..0705e76bf583c 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java @@ -155,6 +155,11 @@ public NarClassLoader run() { }); } + public static List getClasspathFromArchive(File narPath, String narExtractionDirectory) throws IOException { + File unpacked = NarUnpacker.unpackNar(narPath, getNarExtractionDirectory(narExtractionDirectory)); + return getClassPathEntries(unpacked); + } + private static File getNarExtractionDirectory(String configuredDirectory) { return new File(configuredDirectory + "/" + TMP_DIR_PREFIX); } @@ -165,16 +170,11 @@ private static File getNarExtractionDirectory(String configuredDirectory) { * @param narWorkingDirectory * directory to explode nar contents to * @param parent - * @throws IllegalArgumentException - * if the NAR is missing the Java Services API file for FlowFileProcessor implementations. - * @throws ClassNotFoundException - * if any of the FlowFileProcessor implementations defined by the Java Services API cannot be - * loaded. * @throws IOException * if an error occurs while loading the NAR. */ private NarClassLoader(final File narWorkingDirectory, Set additionalJars, ClassLoader parent) - throws ClassNotFoundException, IOException { + throws IOException { super(new URL[0], parent); this.narWorkingDirectory = narWorkingDirectory; @@ -239,22 +239,31 @@ public List getServiceImplementation(String serviceName) throws IOExcept * if the URL list could not be updated. */ private void updateClasspath(File root) throws IOException { - addURL(root.toURI().toURL()); // for compiled classes, META-INF/, etc. + getClassPathEntries(root).forEach(f -> { + try { + addURL(f.toURI().toURL()); + } catch (IOException e) { + log.error("Failed to add entry to classpath: {}", f, e); + } + }); + } + static List getClassPathEntries(File root) { + List classPathEntries = new ArrayList<>(); + classPathEntries.add(root); File dependencies = new File(root, "META-INF/bundled-dependencies"); if (!dependencies.isDirectory()) { - log.warn("{} does not contain META-INF/bundled-dependencies!", narWorkingDirectory); + log.warn("{} does not contain META-INF/bundled-dependencies!", root); } - addURL(dependencies.toURI().toURL()); + classPathEntries.add(dependencies); if (dependencies.isDirectory()) { final File[] jarFiles = dependencies.listFiles(JAR_FILTER); if (jarFiles != null) { Arrays.sort(jarFiles, Comparator.comparing(File::getName)); - for (File libJar : jarFiles) { - addURL(libJar.toURI().toURL()); - } + classPathEntries.addAll(Arrays.asList(jarFiles)); } } + return classPathEntries; } @Override diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java index b2c0a37a65036..1554db5693fd8 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java @@ -33,13 +33,14 @@ import java.io.RandomAccessFile; import java.nio.channels.FileChannel; import java.nio.channels.FileLock; +import java.nio.file.Path; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Base64; import java.util.Enumeration; import java.util.concurrent.ConcurrentHashMap; -import java.util.jar.JarEntry; -import java.util.jar.JarFile; +import java.util.zip.ZipEntry; +import java.util.zip.ZipFile; import lombok.extern.slf4j.Slf4j; /** @@ -114,16 +115,22 @@ static File doUnpackNar(final File nar, final File baseWorkingDirectory, Runnabl * if the NAR could not be unpacked. */ private static void unpack(final File nar, final File workingDirectory) throws IOException { - try (JarFile jarFile = new JarFile(nar)) { - Enumeration jarEntries = jarFile.entries(); - while (jarEntries.hasMoreElements()) { - JarEntry jarEntry = jarEntries.nextElement(); - String name = jarEntry.getName(); - File f = new File(workingDirectory, name); - if (jarEntry.isDirectory()) { + Path workingDirectoryPath = workingDirectory.toPath().normalize(); + try (ZipFile zipFile = new ZipFile(nar)) { + Enumeration zipEntries = zipFile.entries(); + while (zipEntries.hasMoreElements()) { + ZipEntry zipEntry = zipEntries.nextElement(); + String name = zipEntry.getName(); + Path targetFilePath = workingDirectoryPath.resolve(name).normalize(); + if (!targetFilePath.startsWith(workingDirectoryPath)) { + log.error("Invalid zip file with entry '{}'", name); + throw new IOException("Invalid zip file. Aborting unpacking."); + } + File f = targetFilePath.toFile(); + if (zipEntry.isDirectory()) { FileUtils.ensureDirectoryExistAndCanReadAndWrite(f); } else { - makeFile(jarFile.getInputStream(jarEntry), f); + makeFile(zipFile.getInputStream(zipEntry), f); } } } diff --git a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java index 2b9b25ed74349..2b97627875948 100644 --- a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java +++ b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java @@ -31,8 +31,9 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.net.InetSocketAddress; +import java.net.URISyntaxException; import java.nio.file.Files; -import java.nio.file.Paths; +import java.nio.file.Path; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -47,9 +48,12 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Builder; +import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.Utils; +import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.nar.FileUtils; @@ -59,6 +63,7 @@ import org.apache.pulsar.functions.instance.InstanceConfig; import org.apache.pulsar.functions.instance.stats.FunctionCollectorRegistry; import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType; import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.runtime.RuntimeSpawner; import org.apache.pulsar.functions.runtime.RuntimeUtils; @@ -71,8 +76,11 @@ import org.apache.pulsar.functions.secretsproviderconfigurator.SecretsProviderConfigurator; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; +import org.apache.pulsar.functions.utils.FunctionRuntimeCommon; +import org.apache.pulsar.functions.utils.LoadedFunctionPackage; import org.apache.pulsar.functions.utils.SinkConfigUtils; import org.apache.pulsar.functions.utils.SourceConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.functioncache.FunctionCacheEntry; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.utils.functions.FunctionUtils; @@ -90,8 +98,7 @@ public class LocalRunner implements AutoCloseable { private final String functionsDir; private final Thread shutdownHook; private final int instanceLivenessCheck; - private ClassLoader userCodeClassLoader; - private boolean userCodeClassLoaderCreated; + private UserCodeClassLoader userCodeClassLoader; private RuntimeFactory runtimeFactory; private HTTPServer metricsServer; @@ -100,6 +107,12 @@ public enum RuntimeEnv { PROCESS } + @Value + private static class UserCodeClassLoader { + ClassLoader classLoader; + boolean classLoaderCreated; + } + public static class FunctionConfigConverter implements IStringConverter { @Override public FunctionConfig convert(String value) { @@ -257,11 +270,13 @@ public LocalRunner(FunctionConfig functionConfig, SourceConfig sourceConfig, Sin } private static String getPulsarDirectory(String directory) { - String pulsarHome = System.getenv("PULSAR_HOME"); - if (pulsarHome == null) { - pulsarHome = Paths.get("").toAbsolutePath().toString(); + final Path directoryPath; + if (System.getenv("PULSAR_HOME") != null) { + directoryPath = Path.of(System.getenv("PULSAR_HOME"), directory); + } else { + directoryPath = Path.of(directory); } - return Paths.get(pulsarHome, directory).toString(); + return directoryPath.toAbsolutePath().toString(); } private static File createNarExtractionTempDirectory() { @@ -308,16 +323,19 @@ public synchronized void stop() { runtimeFactory = null; } - if (userCodeClassLoaderCreated) { - if (userCodeClassLoader instanceof Closeable) { - try { - ((Closeable) userCodeClassLoader).close(); - } catch (IOException e) { - log.warn("Error closing classloader", e); - } + closeClassLoaderIfneeded(userCodeClassLoader); + userCodeClassLoader = null; + } + } + + private static void closeClassLoaderIfneeded(UserCodeClassLoader userCodeClassLoader) { + if (userCodeClassLoader != null && userCodeClassLoader.isClassLoaderCreated()) { + if (userCodeClassLoader.getClassLoader() instanceof Closeable) { + try { + ((Closeable) userCodeClassLoader.getClassLoader()).close(); + } catch (IOException e) { + log.warn("Error closing classloader", e); } - userCodeClassLoaderCreated = false; - userCodeClassLoader = null; } } } @@ -337,47 +355,14 @@ public void start(boolean blocking) throws Exception { parallelism = functionConfig.getParallelism(); if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { userCodeFile = functionConfig.getJar(); - ClassLoader builtInFunctionClassLoader = userCodeFile != null - ? isBuiltInFunction(userCodeFile) - : null; - if (builtInFunctionClassLoader != null) { - userCodeClassLoader = builtInFunctionClassLoader; - functionDetails = FunctionConfigUtils.convert( - functionConfig, - FunctionConfigUtils.validateJavaFunction(functionConfig, builtInFunctionClassLoader)); - } else if (userCodeFile != null && Utils.isFunctionPackageUrlSupported(userCodeFile)) { - File file = FunctionCommon.extractFileFromPkgURL(userCodeFile); - ClassLoader functionClassLoader = FunctionCommon.getClassLoaderFromPackage( - Function.FunctionDetails.ComponentType.FUNCTION, - functionConfig.getClassName(), file, narExtractionDirectory); - functionDetails = FunctionConfigUtils.convert( - functionConfig, - FunctionConfigUtils.validateJavaFunction(functionConfig, functionClassLoader)); - userCodeClassLoader = functionClassLoader; - userCodeClassLoaderCreated = true; - } else if (userCodeFile != null) { - File file = new File(userCodeFile); - if (!file.exists()) { - throw new RuntimeException("User jar does not exist"); - } - ClassLoader functionClassLoader = FunctionCommon.getClassLoaderFromPackage( - Function.FunctionDetails.ComponentType.FUNCTION, - functionConfig.getClassName(), file, narExtractionDirectory); - functionDetails = FunctionConfigUtils.convert( - functionConfig, - FunctionConfigUtils.validateJavaFunction(functionConfig, functionClassLoader)); - userCodeClassLoader = functionClassLoader; - userCodeClassLoaderCreated = true; - } else { - if (!(runtimeEnv == null || runtimeEnv == RuntimeEnv.THREAD)) { - throw new IllegalStateException("The jar property must be specified in FunctionConfig."); - } - functionDetails = FunctionConfigUtils.convert( - functionConfig, - FunctionConfigUtils.validateJavaFunction( - functionConfig, - Thread.currentThread().getContextClassLoader())); - } + userCodeClassLoader = extractClassLoader( + userCodeFile, ComponentType.FUNCTION, functionConfig.getClassName()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), + FunctionDefinition.class); + functionDetails = FunctionConfigUtils.convert( + functionConfig, + FunctionConfigUtils.validateJavaFunction(functionConfig, validatableFunctionPackage)); } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.GO) { userCodeFile = functionConfig.getGo(); } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.PYTHON) { @@ -387,91 +372,34 @@ public void start(boolean blocking) throws Exception { } if (functionDetails == null) { - functionDetails = FunctionConfigUtils.convert(functionConfig, - userCodeClassLoader != null ? userCodeClassLoader : - Thread.currentThread().getContextClassLoader()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), + FunctionDefinition.class); + functionDetails = FunctionConfigUtils.convert(functionConfig, validatableFunctionPackage); } } else if (sourceConfig != null) { inferMissingArguments(sourceConfig); userCodeFile = sourceConfig.getArchive(); parallelism = sourceConfig.getParallelism(); - - ClassLoader builtInSourceClassLoader = userCodeFile != null ? isBuiltInSource(userCodeFile) : null; - if (builtInSourceClassLoader != null) { - functionDetails = SourceConfigUtils.convert( - sourceConfig, SourceConfigUtils.validateAndExtractDetails( - sourceConfig, builtInSourceClassLoader, true)); - userCodeClassLoader = builtInSourceClassLoader; - } else if (userCodeFile != null && Utils.isFunctionPackageUrlSupported(userCodeFile)) { - File file = FunctionCommon.extractFileFromPkgURL(userCodeFile); - ClassLoader sourceClassLoader = FunctionCommon.getClassLoaderFromPackage( - Function.FunctionDetails.ComponentType.SOURCE, - sourceConfig.getClassName(), file, narExtractionDirectory); - functionDetails = SourceConfigUtils.convert( - sourceConfig, - SourceConfigUtils.validateAndExtractDetails(sourceConfig, sourceClassLoader, true)); - userCodeClassLoader = sourceClassLoader; - userCodeClassLoaderCreated = true; - } else if (userCodeFile != null) { - File file = new File(userCodeFile); - if (!file.exists()) { - throw new RuntimeException("Source archive (" + userCodeFile + ") does not exist"); - } - ClassLoader sourceClassLoader = FunctionCommon.getClassLoaderFromPackage( - Function.FunctionDetails.ComponentType.SOURCE, - sourceConfig.getClassName(), file, narExtractionDirectory); - functionDetails = SourceConfigUtils.convert(sourceConfig, - SourceConfigUtils.validateAndExtractDetails(sourceConfig, sourceClassLoader, true)); - userCodeClassLoader = sourceClassLoader; - userCodeClassLoaderCreated = true; - } else { - if (!(runtimeEnv == null || runtimeEnv == RuntimeEnv.THREAD)) { - throw new IllegalStateException("The archive property must be specified in SourceConfig."); - } - functionDetails = SourceConfigUtils.convert( - sourceConfig, SourceConfigUtils.validateAndExtractDetails( - sourceConfig, Thread.currentThread().getContextClassLoader(), true)); - } + userCodeClassLoader = extractClassLoader( + userCodeFile, ComponentType.SOURCE, sourceConfig.getClassName()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), ConnectorDefinition.class); + functionDetails = SourceConfigUtils.convert(sourceConfig, + SourceConfigUtils.validateAndExtractDetails(sourceConfig, validatableFunctionPackage, true)); } else if (sinkConfig != null) { inferMissingArguments(sinkConfig); userCodeFile = sinkConfig.getArchive(); parallelism = sinkConfig.getParallelism(); - - ClassLoader builtInSinkClassLoader = userCodeFile != null ? isBuiltInSink(userCodeFile) : null; - if (builtInSinkClassLoader != null) { - functionDetails = SinkConfigUtils.convert( - sinkConfig, SinkConfigUtils.validateAndExtractDetails( - sinkConfig, builtInSinkClassLoader, true)); - userCodeClassLoader = builtInSinkClassLoader; - } else if (Utils.isFunctionPackageUrlSupported(userCodeFile)) { - File file = FunctionCommon.extractFileFromPkgURL(userCodeFile); - ClassLoader sinkClassLoader = FunctionCommon.getClassLoaderFromPackage( - Function.FunctionDetails.ComponentType.SINK, - sinkConfig.getClassName(), file, narExtractionDirectory); - functionDetails = SinkConfigUtils.convert( - sinkConfig, SinkConfigUtils.validateAndExtractDetails(sinkConfig, sinkClassLoader, true)); - userCodeClassLoader = sinkClassLoader; - userCodeClassLoaderCreated = true; - } else if (userCodeFile != null) { - File file = new File(userCodeFile); - if (!file.exists()) { - throw new RuntimeException("Sink archive does not exist"); - } - ClassLoader sinkClassLoader = FunctionCommon.getClassLoaderFromPackage( - Function.FunctionDetails.ComponentType.SINK, - sinkConfig.getClassName(), file, narExtractionDirectory); - functionDetails = SinkConfigUtils.convert( - sinkConfig, SinkConfigUtils.validateAndExtractDetails(sinkConfig, sinkClassLoader, true)); - userCodeClassLoader = sinkClassLoader; - userCodeClassLoaderCreated = true; - } else { - if (!(runtimeEnv == null || runtimeEnv == RuntimeEnv.THREAD)) { - throw new IllegalStateException("The archive property must be specified in SourceConfig."); - } - functionDetails = SinkConfigUtils.convert( - sinkConfig, SinkConfigUtils.validateAndExtractDetails( - sinkConfig, Thread.currentThread().getContextClassLoader(), true)); - } + userCodeClassLoader = extractClassLoader( + userCodeFile, ComponentType.SINK, sinkConfig.getClassName()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), ConnectorDefinition.class); + + functionDetails = SinkConfigUtils.convert( + sinkConfig, + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunctionPackage, + true)); } else { throw new IllegalArgumentException("Must specify Function, Source or Sink config"); } @@ -526,6 +454,67 @@ public void start(boolean blocking) throws Exception { } } + private ClassLoader getCurrentOrUserCodeClassLoader() { + return userCodeClassLoader == null || userCodeClassLoader.getClassLoader() == null + ? Thread.currentThread().getContextClassLoader() + : userCodeClassLoader.getClassLoader(); + } + + private UserCodeClassLoader extractClassLoader(String userCodeFile, ComponentType componentType, String className) + throws IOException, URISyntaxException { + ClassLoader classLoader = userCodeFile != null ? isBuiltIn(userCodeFile, componentType) : null; + boolean classLoaderCreated = false; + if (classLoader == null) { + if (userCodeFile != null && Utils.isFunctionPackageUrlSupported(userCodeFile)) { + File file = FunctionCommon.extractFileFromPkgURL(userCodeFile); + classLoader = FunctionRuntimeCommon.getClassLoaderFromPackage( + componentType, className, file, narExtractionDirectory); + classLoaderCreated = true; + } else if (userCodeFile != null) { + File file = new File(userCodeFile); + if (!file.exists()) { + String errorMsg; + switch (componentType) { + case FUNCTION: + errorMsg = "User jar"; + break; + case SOURCE: + errorMsg = "Source archive"; + break; + case SINK: + errorMsg = "Sink archive"; + break; + default: + throw new IllegalStateException("Unexpected value: " + componentType); + } + throw new RuntimeException(errorMsg + " (" + userCodeFile + ") does not exist"); + } + classLoader = FunctionRuntimeCommon.getClassLoaderFromPackage( + componentType, className, file, narExtractionDirectory); + classLoaderCreated = true; + } else { + if (!(runtimeEnv == null || runtimeEnv == RuntimeEnv.THREAD)) { + String errorMsg; + switch (componentType) { + case FUNCTION: + errorMsg = "The jar property must be specified in FunctionConfig."; + break; + case SOURCE: + errorMsg = "The archive property must be specified in SourceConfig."; + break; + case SINK: + errorMsg = "The archive property must be specified in SinkConfig."; + break; + default: + throw new IllegalStateException("Unexpected ComponentType: " + componentType); + } + throw new IllegalStateException(errorMsg); + } + } + } + return new UserCodeClassLoader(classLoader, classLoaderCreated); + } + private void startProcessMode(org.apache.pulsar.functions.proto.Function.FunctionDetails functionDetails, int parallelism, int instanceIdOffset, String serviceUrl, String stateStorageServiceUrl, AuthenticationConfig authConfig, @@ -641,8 +630,8 @@ private void startThreadedMode(org.apache.pulsar.functions.proto.Function.Functi ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader(); try { - if (userCodeClassLoader != null) { - Thread.currentThread().setContextClassLoader(userCodeClassLoader); + if (userCodeClassLoader != null && userCodeClassLoader.getClassLoader() != null) { + Thread.currentThread().setContextClassLoader(userCodeClassLoader.getClassLoader()); } runtimeFactory = new ThreadRuntimeFactory("LocalRunnerThreadGroup", serviceUrl, @@ -692,6 +681,20 @@ private void startThreadedMode(org.apache.pulsar.functions.proto.Function.Functi } } + private ClassLoader isBuiltIn(String component, ComponentType componentType) + throws IOException { + switch (componentType) { + case FUNCTION: + return isBuiltInFunction(component); + case SOURCE: + return isBuiltInSource(component); + case SINK: + return isBuiltInSink(component); + default: + throw new IllegalStateException("Unexpected ComponentType: " + componentType); + } + } + private ClassLoader isBuiltInFunction(String functionType) throws IOException { // Validate the connector type from the locally available connectors TreeMap functions = getFunctions(); @@ -700,7 +703,7 @@ private ClassLoader isBuiltInFunction(String functionType) throws IOException { FunctionArchive function = functions.get(functionName); if (function != null && function.getFunctionDefinition().getFunctionClass() != null) { // Function type is a valid built-in type. - return function.getClassLoader(); + return function.getFunctionPackage().getClassLoader(); } else { return null; } @@ -714,7 +717,7 @@ private ClassLoader isBuiltInSource(String sourceType) throws IOException { Connector connector = connectors.get(source); if (connector != null && connector.getConnectorDefinition().getSourceClass() != null) { // Source type is a valid built-in connector type. - return connector.getClassLoader(); + return connector.getConnectorFunctionPackage().getClassLoader(); } else { return null; } @@ -728,18 +731,18 @@ private ClassLoader isBuiltInSink(String sinkType) throws IOException { Connector connector = connectors.get(sink); if (connector != null && connector.getConnectorDefinition().getSinkClass() != null) { // Sink type is a valid built-in connector type - return connector.getClassLoader(); + return connector.getConnectorFunctionPackage().getClassLoader(); } else { return null; } } private TreeMap getFunctions() throws IOException { - return FunctionUtils.searchForFunctions(functionsDir); + return FunctionUtils.searchForFunctions(functionsDir, narExtractionDirectory, true); } private TreeMap getConnectors() throws IOException { - return ConnectorUtils.searchForConnectors(connectorsDir, narExtractionDirectory); + return ConnectorUtils.searchForConnectors(connectorsDir, narExtractionDirectory, true); } private SecretsProviderConfigurator getSecretsProviderConfigurator() { diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java index 5743341483f14..f3ec46dcf2809 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java @@ -38,6 +38,9 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.dynamic.ClassFileLocator; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.functions.WindowConfig; import org.apache.pulsar.common.nar.NarClassLoader; @@ -306,7 +309,8 @@ public void close() { } private void inferringMissingTypeClassName(Function.FunctionDetails.Builder functionDetailsBuilder, - ClassLoader classLoader) throws ClassNotFoundException { + ClassLoader classLoader) { + TypePool typePool = TypePool.Default.of(ClassFileLocator.ForClassLoader.of(classLoader)); switch (functionDetailsBuilder.getComponentType()) { case FUNCTION: if ((functionDetailsBuilder.hasSource() @@ -325,14 +329,13 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func WindowConfig.class); className = windowConfig.getActualWindowFunctionClassName(); } - - Class[] typeArgs = FunctionCommon.getFunctionTypes(classLoader.loadClass(className), + TypeDefinition[] typeArgs = FunctionCommon.getFunctionTypes(typePool.describe(className).resolve(), isWindowConfigPresent); if (functionDetailsBuilder.hasSource() && functionDetailsBuilder.getSource().getTypeClassName().isEmpty() && typeArgs[0] != null) { Function.SourceSpec.Builder sourceBuilder = functionDetailsBuilder.getSource().toBuilder(); - sourceBuilder.setTypeClassName(typeArgs[0].getName()); + sourceBuilder.setTypeClassName(typeArgs[0].asErasure().getTypeName()); functionDetailsBuilder.setSource(sourceBuilder.build()); } @@ -340,7 +343,7 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func && functionDetailsBuilder.getSink().getTypeClassName().isEmpty() && typeArgs[1] != null) { Function.SinkSpec.Builder sinkBuilder = functionDetailsBuilder.getSink().toBuilder(); - sinkBuilder.setTypeClassName(typeArgs[1].getName()); + sinkBuilder.setTypeClassName(typeArgs[1].asErasure().getTypeName()); functionDetailsBuilder.setSink(sinkBuilder.build()); } } @@ -349,7 +352,8 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func if ((functionDetailsBuilder.hasSink() && functionDetailsBuilder.getSink().getTypeClassName().isEmpty())) { String typeArg = - getSinkType(functionDetailsBuilder.getSink().getClassName(), classLoader).getName(); + getSinkType(functionDetailsBuilder.getSink().getClassName(), typePool).asErasure() + .getTypeName(); Function.SinkSpec.Builder sinkBuilder = Function.SinkSpec.newBuilder(functionDetailsBuilder.getSink()); @@ -368,7 +372,8 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func if ((functionDetailsBuilder.hasSource() && functionDetailsBuilder.getSource().getTypeClassName().isEmpty())) { String typeArg = - getSourceType(functionDetailsBuilder.getSource().getClassName(), classLoader).getName(); + getSourceType(functionDetailsBuilder.getSource().getClassName(), typePool).asErasure() + .getTypeName(); Function.SourceSpec.Builder sourceBuilder = Function.SourceSpec.newBuilder(functionDetailsBuilder.getSource()); diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java index 26651010fefc4..47bd3ba0472b8 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java @@ -121,17 +121,17 @@ private static ClassLoader getFunctionClassLoader(InstanceConfig instanceConfig, if (componentType == Function.FunctionDetails.ComponentType.FUNCTION && functionsManager.isPresent()) { return functionsManager.get() .getFunction(instanceConfig.getFunctionDetails().getBuiltin()) - .getClassLoader(); + .getFunctionPackage().getClassLoader(); } if (componentType == Function.FunctionDetails.ComponentType.SOURCE && connectorsManager.isPresent()) { return connectorsManager.get() .getConnector(instanceConfig.getFunctionDetails().getSource().getBuiltin()) - .getClassLoader(); + .getConnectorFunctionPackage().getClassLoader(); } if (componentType == Function.FunctionDetails.ComponentType.SINK && connectorsManager.isPresent()) { return connectorsManager.get() .getConnector(instanceConfig.getFunctionDetails().getSink().getBuiltin()) - .getClassLoader(); + .getConnectorFunctionPackage().getClassLoader(); } } return loadJars(jarFile, instanceConfig, instanceConfig.getFunctionDetails().getName(), diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java index 643afa4f26c3c..c78bb73428548 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.functions.worker; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.nio.file.Path; import java.util.List; @@ -27,18 +28,35 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.utils.io.ConnectorUtils; @Slf4j -public class ConnectorsManager { +public class ConnectorsManager implements AutoCloseable { @Getter private volatile TreeMap connectors; + @VisibleForTesting + public ConnectorsManager() { + this.connectors = new TreeMap<>(); + } + public ConnectorsManager(WorkerConfig workerConfig) throws IOException { - this.connectors = ConnectorUtils - .searchForConnectors(workerConfig.getConnectorsDirectory(), workerConfig.getNarExtractionDirectory()); + this.connectors = createConnectors(workerConfig); + } + + private static TreeMap createConnectors(WorkerConfig workerConfig) throws IOException { + boolean enableClassloading = workerConfig.getEnableClassloadingOfBuiltinFiles() + || ThreadRuntimeFactory.class.getName().equals(workerConfig.getFunctionRuntimeFactoryClassName()); + return ConnectorUtils.searchForConnectors(workerConfig.getConnectorsDirectory(), + workerConfig.getNarExtractionDirectory(), enableClassloading); + } + + @VisibleForTesting + public void addConnector(String connectorType, Connector connector) { + connectors.put(connectorType, connector); } public Connector getConnector(String connectorType) { @@ -71,7 +89,25 @@ public Path getSinkArchive(String sinkType) { } public void reloadConnectors(WorkerConfig workerConfig) throws IOException { - connectors = ConnectorUtils - .searchForConnectors(workerConfig.getConnectorsDirectory(), workerConfig.getNarExtractionDirectory()); + TreeMap oldConnectors = connectors; + this.connectors = createConnectors(workerConfig); + closeConnectors(oldConnectors); } + + @Override + public void close() { + closeConnectors(connectors); + } + + private void closeConnectors(TreeMap connectorMap) { + connectorMap.values().forEach(connector -> { + try { + connector.close(); + } catch (Exception e) { + log.warn("Failed to close connector", e); + } + }); + connectorMap.clear(); + } + } diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java index 0f1c0fcc8356e..753d41f0fa772 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java @@ -18,20 +18,30 @@ */ package org.apache.pulsar.functions.worker; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.nio.file.Path; import java.util.TreeMap; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.utils.functions.FunctionUtils; @Slf4j -public class FunctionsManager { - +public class FunctionsManager implements AutoCloseable { private TreeMap functions; + @VisibleForTesting + public FunctionsManager() { + this.functions = new TreeMap<>(); + } + public FunctionsManager(WorkerConfig workerConfig) throws IOException { - this.functions = FunctionUtils.searchForFunctions(workerConfig.getFunctionsDirectory()); + this.functions = createFunctions(workerConfig); + } + + public void addFunction(String functionType, FunctionArchive functionArchive) { + functions.put(functionType, functionArchive); } public FunctionArchive getFunction(String functionType) { @@ -43,6 +53,32 @@ public Path getFunctionArchive(String functionType) { } public void reloadFunctions(WorkerConfig workerConfig) throws IOException { - this.functions = FunctionUtils.searchForFunctions(workerConfig.getFunctionsDirectory()); + TreeMap oldFunctions = functions; + this.functions = createFunctions(workerConfig); + closeFunctions(oldFunctions); + } + + private static TreeMap createFunctions(WorkerConfig workerConfig) throws IOException { + boolean enableClassloading = workerConfig.getEnableClassloadingOfBuiltinFiles() + || ThreadRuntimeFactory.class.getName().equals(workerConfig.getFunctionRuntimeFactoryClassName()); + return FunctionUtils.searchForFunctions(workerConfig.getFunctionsDirectory(), + workerConfig.getNarExtractionDirectory(), + enableClassloading); + } + + @Override + public void close() { + closeFunctions(functions); + } + + private void closeFunctions(TreeMap functionMap) { + functionMap.values().forEach(functionArchive -> { + try { + functionArchive.close(); + } catch (Exception e) { + log.warn("Failed to close function archive", e); + } + }); + functionMap.clear(); } } diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index 76078a46e7979..5432d5c46106a 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -225,6 +225,22 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { ) private int zooKeeperCacheExpirySeconds = -1; + @FieldContext( + category = CATEGORY_WORKER, + doc = "Specifies if the function worker should use classloading for validating submissions for built-in " + + "connectors and functions. This is required for validateConnectorConfig to take effect. " + + "Default is false." + ) + private Boolean enableClassloadingOfBuiltinFiles = false; + + @FieldContext( + category = CATEGORY_WORKER, + doc = "Specifies if the function worker should use classloading for validating submissions for external " + + "connectors and functions. This is required for validateConnectorConfig to take effect. " + + "Default is false." + ) + private Boolean enableClassloadingOfExternalFiles = false; + @FieldContext( category = CATEGORY_CONNECTORS, doc = "The path to the location to locate builtin connectors" @@ -237,7 +253,10 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { private String narExtractionDirectory = NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR; @FieldContext( category = CATEGORY_CONNECTORS, - doc = "Should we validate connector config during submission" + doc = "Enables extended validation for connector config with fine-grain annotation based validation " + + "during submission. Classloading with either enableClassloadingOfExternalFiles or " + + "enableClassloadingOfBuiltinFiles must be enabled on the worker for this to take effect. " + + "Default is false." ) private Boolean validateConnectorConfig = false; @FieldContext( diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 27274258a183c..befc887f67c60 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -87,6 +87,17 @@ typetools + + net.bytebuddy + byte-buddy + + + + org.zeroturnaround + zt-zip + 1.17 + + ${project.groupId} pulsar-client-original diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java index 49b71afcad795..0e17907fb3b32 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java @@ -22,27 +22,25 @@ import com.google.protobuf.AbstractMessage.Builder; import com.google.protobuf.MessageOrBuilder; import com.google.protobuf.util.JsonFormat; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.ObjectOutputStream; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.net.MalformedURLException; import java.net.ServerSocket; import java.net.URISyntaxException; import java.net.URL; +import java.net.URLConnection; import java.nio.file.Files; import java.nio.file.StandardCopyOption; import java.util.Collection; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import lombok.AccessLevel; import lombok.NoArgsConstructor; import lombok.extern.slf4j.Slf4j; -import net.jodah.typetools.TypeResolver; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.description.type.TypeList; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionInitialPosition; @@ -50,15 +48,11 @@ import org.apache.pulsar.client.impl.TopicMessageIdImpl; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.Utils; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.WindowFunction; +import org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType; import org.apache.pulsar.functions.proto.Function.FunctionDetails.Runtime; -import org.apache.pulsar.functions.utils.functions.FunctionUtils; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; import org.apache.pulsar.io.core.BatchSource; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.Source; @@ -92,82 +86,79 @@ public static int findAvailablePort() { } } - public static Class[] getFunctionTypes(FunctionConfig functionConfig, ClassLoader classLoader) + public static TypeDefinition[] getFunctionTypes(FunctionConfig functionConfig, TypePool typePool) throws ClassNotFoundException { - return getFunctionTypes(functionConfig, classLoader.loadClass(functionConfig.getClassName())); + return getFunctionTypes(functionConfig, typePool.describe(functionConfig.getClassName()).resolve()); } - public static Class[] getFunctionTypes(FunctionConfig functionConfig, Class functionClass) - throws ClassNotFoundException { + public static TypeDefinition[] getFunctionTypes(FunctionConfig functionConfig, TypeDefinition functionClass) { boolean isWindowConfigPresent = functionConfig.getWindowConfig() != null; return getFunctionTypes(functionClass, isWindowConfigPresent); } - public static Class[] getFunctionTypes(Class userClass, boolean isWindowConfigPresent) { - Class[] typeArgs; + public static TypeDefinition[] getFunctionTypes(TypeDefinition userClass, boolean isWindowConfigPresent) { + Class classParent = getFunctionClassParent(userClass, isWindowConfigPresent); + TypeList.Generic typeArgsList = resolveInterfaceTypeArguments(userClass, classParent); + TypeDescription.Generic[] typeArgs = new TypeDescription.Generic[2]; + typeArgs[0] = typeArgsList.get(0); + typeArgs[1] = typeArgsList.get(1); // if window function if (isWindowConfigPresent) { - if (WindowFunction.class.isAssignableFrom(userClass)) { - typeArgs = getFunctionTypesUnwrappingRecordIfNeeded(WindowFunction.class, userClass); - } else { - typeArgs = getFunctionTypesUnwrappingRecordIfNeeded(java.util.function.Function.class, userClass); - if (!typeArgs[0].equals(Collection.class)) { + if (classParent.equals(java.util.function.Function.class)) { + if (!typeArgs[0].asErasure().isAssignableTo(Collection.class)) { throw new IllegalArgumentException("Window function must take a collection as input"); } - Type type = TypeResolver.resolveGenericType(java.util.function.Function.class, userClass); - Type collectionType = ((ParameterizedType) type).getActualTypeArguments()[0]; - Type actualInputType = ((ParameterizedType) collectionType).getActualTypeArguments()[0]; - typeArgs[0] = (Class) actualInputType; - } - } else { - if (Function.class.isAssignableFrom(userClass)) { - typeArgs = getFunctionTypesUnwrappingRecordIfNeeded(Function.class, userClass); - } else { - typeArgs = getFunctionTypesUnwrappingRecordIfNeeded(java.util.function.Function.class, userClass); + typeArgs[0] = typeArgs[0].getTypeArguments().get(0); } } - + if (typeArgs[1].asErasure().isAssignableTo(Record.class)) { + typeArgs[1] = typeArgs[1].getTypeArguments().get(0); + } + if (typeArgs[1].asErasure().isAssignableTo(CompletableFuture.class)) { + typeArgs[1] = typeArgs[1].getTypeArguments().get(0); + } return typeArgs; } - private static Class[] getFunctionTypesUnwrappingRecordIfNeeded(Class type, Class subType) { - Class[] typeArgs = TypeResolver.resolveRawArguments(type, subType); - if (typeArgs[1].equals(Record.class)) { - Type genericType = TypeResolver.resolveGenericType(type, subType); - Type recordType = ((ParameterizedType) genericType).getActualTypeArguments()[1]; - Type actualInputType = ((ParameterizedType) recordType).getActualTypeArguments()[0]; - typeArgs[1] = (Class) actualInputType; + private static TypeList.Generic resolveInterfaceTypeArguments(TypeDefinition userClass, Class interfaceClass) { + if (!interfaceClass.isInterface()) { + throw new IllegalArgumentException("interfaceClass must be an interface"); + } + for (TypeDescription.Generic interfaze : userClass.getInterfaces()) { + if (interfaze.asErasure().isAssignableTo(interfaceClass)) { + return interfaze.getTypeArguments(); + } + } + if (userClass.getSuperClass() != null) { + return resolveInterfaceTypeArguments(userClass.getSuperClass(), interfaceClass); } + return null; + } + + public static TypeDescription.Generic[] getRawFunctionTypes(TypeDefinition userClass, + boolean isWindowConfigPresent) { + Class classParent = getFunctionClassParent(userClass, isWindowConfigPresent); + TypeList.Generic typeArgsList = resolveInterfaceTypeArguments(userClass, classParent); + TypeDescription.Generic[] typeArgs = new TypeDescription.Generic[2]; + typeArgs[0] = typeArgsList.get(0); + typeArgs[1] = typeArgsList.get(1); return typeArgs; } - public static Object createInstance(String userClassName, ClassLoader classLoader) { - Class theCls; - try { - theCls = Class.forName(userClassName); - } catch (ClassNotFoundException | NoClassDefFoundError cnfe) { - try { - theCls = Class.forName(userClassName, true, classLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new RuntimeException("User class must be in class path", cnfe); + public static Class getFunctionClassParent(TypeDefinition userClass, boolean isWindowConfigPresent) { + if (isWindowConfigPresent) { + if (userClass.asErasure().isAssignableTo(WindowFunction.class)) { + return WindowFunction.class; + } else { + return java.util.function.Function.class; + } + } else { + if (userClass.asErasure().isAssignableTo(Function.class)) { + return Function.class; + } else { + return java.util.function.Function.class; } } - Object result; - try { - Constructor meth = theCls.getDeclaredConstructor(); - meth.setAccessible(true); - result = meth.newInstance(); - } catch (InstantiationException ie) { - throw new RuntimeException("User class must be concrete", ie); - } catch (NoSuchMethodException e) { - throw new RuntimeException("User class doesn't have such method", e); - } catch (IllegalAccessException e) { - throw new RuntimeException("User class must have a no-arg constructor", e); - } catch (InvocationTargetException e) { - throw new RuntimeException("User class constructor throws exception", e); - } - return result; - } public static Runtime convertRuntime(FunctionConfig.Runtime runtime) { @@ -210,50 +201,46 @@ public static FunctionConfig.ProcessingGuarantees convertProcessingGuarantee( throw new RuntimeException("Unrecognized processing guarantee: " + processingGuarantees.name()); } - public static Class getSourceType(String className, ClassLoader classLoader) throws ClassNotFoundException { - return getSourceType(classLoader.loadClass(className)); + public static TypeDefinition getSourceType(String className, TypePool typePool) { + return getSourceType(typePool.describe(className).resolve()); } - public static Class getSourceType(Class sourceClass) { - - if (Source.class.isAssignableFrom(sourceClass)) { - return TypeResolver.resolveRawArgument(Source.class, sourceClass); - } else if (BatchSource.class.isAssignableFrom(sourceClass)) { - return TypeResolver.resolveRawArgument(BatchSource.class, sourceClass); + public static TypeDefinition getSourceType(TypeDefinition sourceClass) { + if (sourceClass.asErasure().isAssignableTo(Source.class)) { + return resolveInterfaceTypeArguments(sourceClass, Source.class).get(0); + } else if (sourceClass.asErasure().isAssignableTo(BatchSource.class)) { + return resolveInterfaceTypeArguments(sourceClass, BatchSource.class).get(0); } else { throw new IllegalArgumentException( String.format("Source class %s does not implement the correct interface", - sourceClass.getName())); + sourceClass.getActualName())); } } - public static Class getSinkType(String className, ClassLoader classLoader) throws ClassNotFoundException { - return getSinkType(classLoader.loadClass(className)); + public static TypeDefinition getSinkType(String className, TypePool typePool) { + return getSinkType(typePool.describe(className).resolve()); } - public static Class getSinkType(Class sinkClass) { - return TypeResolver.resolveRawArgument(Sink.class, sinkClass); + public static TypeDefinition getSinkType(TypeDefinition sinkClass) { + if (sinkClass.asErasure().isAssignableTo(Sink.class)) { + return resolveInterfaceTypeArguments(sinkClass, Sink.class).get(0); + } else { + throw new IllegalArgumentException( + String.format("Sink class %s does not implement the correct interface", + sinkClass.getActualName())); + } } public static void downloadFromHttpUrl(String destPkgUrl, File targetFile) throws IOException { - URL website = new URL(destPkgUrl); - try (InputStream in = website.openStream()) { + final URL url = new URL(destPkgUrl); + final URLConnection connection = url.openConnection(); + try (InputStream in = connection.getInputStream()) { log.info("Downloading function package from {} to {} ...", destPkgUrl, targetFile.getAbsoluteFile()); Files.copy(in, targetFile.toPath(), StandardCopyOption.REPLACE_EXISTING); } log.info("Downloading function package from {} to {} completed!", destPkgUrl, targetFile.getAbsoluteFile()); } - public static ClassLoader extractClassLoader(String destPkgUrl) throws IOException, URISyntaxException { - File file = extractFileFromPkgURL(destPkgUrl); - try { - return ClassLoaderUtils.loadJar(file); - } catch (MalformedURLException e) { - throw new IllegalArgumentException( - "Corrupt User PackageFile " + file + " with error " + e.getMessage()); - } - } - public static File createPkgTempFile() throws IOException { return File.createTempFile("functions", ".tmp"); } @@ -277,21 +264,6 @@ public static File extractFileFromPkgURL(String destPkgUrl) throws IOException, } } - public static NarClassLoader extractNarClassLoader(File packageFile, - String narExtractionDirectory) { - if (packageFile != null) { - try { - return NarClassLoaderBuilder.builder() - .narFile(packageFile) - .extractionDirectory(narExtractionDirectory) - .build(); - } catch (IOException e) { - throw new IllegalArgumentException(e.getMessage()); - } - } - return null; - } - public static String getFullyQualifiedInstanceId(org.apache.pulsar.functions.proto.Function.Instance instance) { return getFullyQualifiedInstanceId( instance.getFunctionMetaData().getFunctionDetails().getTenant(), @@ -327,17 +299,6 @@ public static final MessageId getMessageId(long sequenceId) { return new MessageIdImpl(ledgerId, entryId, -1); } - public static byte[] toByteArray(Object obj) throws IOException { - byte[] bytes = null; - try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(bos)) { - oos.writeObject(obj); - oos.flush(); - bytes = bos.toByteArray(); - } - return bytes; - } - public static String getUniquePackageName(String packageName) { return String.format("%s-%s", UUID.randomUUID().toString(), packageName); } @@ -385,173 +346,38 @@ private static String extractFromFullyQualifiedName(String fqfn, int index) { throw new RuntimeException("Invalid Fully Qualified Function Name " + fqfn); } - public static Class getTypeArg(String className, Class funClass, ClassLoader classLoader) - throws ClassNotFoundException { - Class loadedClass = classLoader.loadClass(className); - if (!funClass.isAssignableFrom(loadedClass)) { - throw new IllegalArgumentException( - String.format("class %s is not type of %s", className, funClass.getName())); - } - return TypeResolver.resolveRawArgument(funClass, loadedClass); - } - public static double roundDecimal(double value, int places) { double scale = Math.pow(10, places); return Math.round(value * scale) / scale; } - public static ClassLoader getClassLoaderFromPackage( - org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType componentType, - String className, - File packageFile, - String narExtractionDirectory) { - String connectorClassName = className; - ClassLoader jarClassLoader = null; - boolean keepJarClassLoader = false; - ClassLoader narClassLoader = null; - boolean keepNarClassLoader = false; - - Exception jarClassLoaderException = null; - Exception narClassLoaderException = null; - - try { - try { - jarClassLoader = ClassLoaderUtils.extractClassLoader(packageFile); - } catch (Exception e) { - jarClassLoaderException = e; - } - try { - narClassLoader = FunctionCommon.extractNarClassLoader(packageFile, narExtractionDirectory); - } catch (Exception e) { - narClassLoaderException = e; - } - - // if connector class name is not provided, we can only try to load archive as a NAR - if (isEmpty(connectorClassName)) { - if (narClassLoader == null) { - throw new IllegalArgumentException(String.format("%s package does not have the correct format. " - + "Pulsar cannot determine if the package is a NAR package or JAR package. " - + "%s classname is not provided and attempts to load it as a NAR package produced " - + "the following error.", - capFirstLetter(componentType), capFirstLetter(componentType)), - narClassLoaderException); - } - try { - if (componentType - == org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType.FUNCTION) { - connectorClassName = FunctionUtils.getFunctionClass((NarClassLoader) narClassLoader); - } else if (componentType - == org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType.SOURCE) { - connectorClassName = ConnectorUtils.getIOSourceClass((NarClassLoader) narClassLoader); - } else { - connectorClassName = ConnectorUtils.getIOSinkClass((NarClassLoader) narClassLoader); - } - } catch (IOException e) { - throw new IllegalArgumentException(String.format("Failed to extract %s class from archive", - componentType.toString().toLowerCase()), e); - } - - try { - narClassLoader.loadClass(connectorClassName); - keepNarClassLoader = true; - return narClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", capFirstLetter(componentType), - connectorClassName), e); - } - - } else { - // if connector class name is provided, we need to try to load it as a JAR and as a NAR. - if (jarClassLoader != null) { - try { - jarClassLoader.loadClass(connectorClassName); - keepJarClassLoader = true; - return jarClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e) { - // class not found in JAR try loading as a NAR and searching for the class - if (narClassLoader != null) { - - try { - narClassLoader.loadClass(connectorClassName); - keepNarClassLoader = true; - return narClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e1) { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", - capFirstLetter(componentType), connectorClassName), e1); - } - } else { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", capFirstLetter(componentType), - connectorClassName), e); - } - } - } else if (narClassLoader != null) { - try { - narClassLoader.loadClass(connectorClassName); - keepNarClassLoader = true; - return narClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e1) { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", - capFirstLetter(componentType), connectorClassName), e1); - } - } else { - StringBuilder errorMsg = new StringBuilder(capFirstLetter(componentType) - + " package does not have the correct format." - + " Pulsar cannot determine if the package is a NAR package or JAR package."); - - if (jarClassLoaderException != null) { - errorMsg.append( - " Attempts to load it as a JAR package produced error: " + jarClassLoaderException - .getMessage()); - } - - if (narClassLoaderException != null) { - errorMsg.append( - " Attempts to load it as a NAR package produced error: " + narClassLoaderException - .getMessage()); - } - - throw new IllegalArgumentException(errorMsg.toString()); - } - } - } finally { - if (!keepJarClassLoader) { - ClassLoaderUtils.closeClassLoader(jarClassLoader); - } - if (!keepNarClassLoader) { - ClassLoaderUtils.closeClassLoader(narClassLoader); - } - } - } - public static String capFirstLetter(Enum en) { return StringUtils.capitalize(en.toString().toLowerCase()); } public static boolean isFunctionCodeBuiltin( - org.apache.pulsar.functions.proto.Function.FunctionDetailsOrBuilder functionDetails) { - if (functionDetails.hasSource()) { + org.apache.pulsar.functions.proto.Function.FunctionDetailsOrBuilder functionDetail) { + return isFunctionCodeBuiltin(functionDetail, functionDetail.getComponentType()); + } + + public static boolean isFunctionCodeBuiltin( + org.apache.pulsar.functions.proto.Function.FunctionDetailsOrBuilder functionDetails, + ComponentType componentType) { + if (componentType == ComponentType.SOURCE && functionDetails.hasSource()) { org.apache.pulsar.functions.proto.Function.SourceSpec sourceSpec = functionDetails.getSource(); if (!isEmpty(sourceSpec.getBuiltin())) { return true; } } - if (functionDetails.hasSink()) { + if (componentType == ComponentType.SINK && functionDetails.hasSink()) { org.apache.pulsar.functions.proto.Function.SinkSpec sinkSpec = functionDetails.getSink(); if (!isEmpty(sinkSpec.getBuiltin())) { return true; } } - if (!isEmpty(functionDetails.getBuiltin())) { - return true; - } - - return false; + return componentType == ComponentType.FUNCTION && !isEmpty(functionDetails.getBuiltin()); } public static SubscriptionInitialPosition convertFromFunctionDetailsSubscriptionPosition( diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index a8630dd3b34d8..dcd6982315e9a 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -18,21 +18,18 @@ */ package org.apache.pulsar.functions.utils; -import static org.apache.commons.lang.StringUtils.isBlank; -import static org.apache.commons.lang.StringUtils.isNotBlank; -import static org.apache.commons.lang.StringUtils.isNotEmpty; +import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isEmpty; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.pulsar.common.functions.Utils.BUILTIN; -import static org.apache.pulsar.common.util.ClassLoaderUtils.loadJar; import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromFunctionDetailsSubscriptionPosition; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import java.io.File; -import java.io.IOException; import java.lang.reflect.Type; -import java.net.MalformedURLException; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; @@ -42,10 +39,13 @@ import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.pool.TypePool; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.functions.ConsumerConfig; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.functions.WindowConfig; @@ -53,7 +53,6 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.utils.functions.FunctionUtils; @Slf4j public class FunctionConfigUtils { @@ -72,26 +71,21 @@ public static class ExtractedFunctionDetails { private static final ObjectMapper OBJECT_MAPPER = ObjectMapperFactory.create(); - public static FunctionDetails convert(FunctionConfig functionConfig, ClassLoader classLoader) - throws IllegalArgumentException { + public static FunctionDetails convert(FunctionConfig functionConfig) { + return convert(functionConfig, (ValidatableFunctionPackage) null); + } - if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { - if (classLoader != null) { - try { - Class[] typeArgs = FunctionCommon.getFunctionTypes(functionConfig, classLoader); - return convert( - functionConfig, - new ExtractedFunctionDetails( - functionConfig.getClassName(), - typeArgs[0].getName(), - typeArgs[1].getName())); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionConfig.getClassName()), e); - } - } + public static FunctionDetails convert(FunctionConfig functionConfig, + ValidatableFunctionPackage validatableFunctionPackage) + throws IllegalArgumentException { + if (functionConfig == null) { + throw new IllegalArgumentException("Function config is not provided"); + } + if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA && validatableFunctionPackage != null) { + return convert(functionConfig, doJavaChecks(functionConfig, validatableFunctionPackage)); + } else { + return convert(functionConfig, new ExtractedFunctionDetails(functionConfig.getClassName(), null, null)); } - return convert(functionConfig, new ExtractedFunctionDetails(functionConfig.getClassName(), null, null)); } public static FunctionDetails convert(FunctionConfig functionConfig, ExtractedFunctionDetails extractedDetails) @@ -568,7 +562,6 @@ public static void inferMissingArguments(FunctionConfig functionConfig, } else if (functionConfig.getGo() != null) { functionConfig.setRuntime(FunctionConfig.Runtime.GO); } - WindowConfig windowConfig = functionConfig.getWindowConfig(); if (windowConfig != null) { WindowConfigUtils.inferMissingArguments(windowConfig); @@ -576,48 +569,49 @@ public static void inferMissingArguments(FunctionConfig functionConfig, } } - private static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfig, ClassLoader clsLoader) { + public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfig, + ValidatableFunctionPackage validatableFunctionPackage) { - String functionClassName = functionConfig.getClassName(); - Class functionClass; + String functionClassName = StringUtils.trimToNull(functionConfig.getClassName()); + TypeDefinition functionClass; try { // if class name in function config is not set, this should be a built-in function // thus we should try to find its class name in the NAR service definition if (functionClassName == null) { - try { - functionClassName = FunctionUtils.getFunctionClass(clsLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract source class from archive", e); + FunctionDefinition functionDefinition = + validatableFunctionPackage.getFunctionMetaData(FunctionDefinition.class); + if (functionDefinition == null) { + throw new IllegalArgumentException("Function class name is not provided."); + } + functionClassName = functionDefinition.getFunctionClass(); + if (functionClassName == null) { + throw new IllegalArgumentException("Function class name is not provided."); } } - functionClass = clsLoader.loadClass(functionClassName); + functionClass = validatableFunctionPackage.resolveType(functionClassName); - if (!org.apache.pulsar.functions.api.Function.class.isAssignableFrom(functionClass) - && !java.util.function.Function.class.isAssignableFrom(functionClass) - && !org.apache.pulsar.functions.api.WindowFunction.class.isAssignableFrom(functionClass)) { + if (!functionClass.asErasure().isAssignableTo(org.apache.pulsar.functions.api.Function.class) + && !functionClass.asErasure().isAssignableTo(java.util.function.Function.class) + && !functionClass.asErasure() + .isAssignableTo(org.apache.pulsar.functions.api.WindowFunction.class)) { throw new IllegalArgumentException( String.format("Function class %s does not implement the correct interface", - functionClass.getName())); + functionClassName)); } - } catch (ClassNotFoundException | NoClassDefFoundError e) { + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionConfig.getClassName()), e); + String.format("Function class %s must be in class path", functionClassName), e); } - Class[] typeArgs; - try { - typeArgs = FunctionCommon.getFunctionTypes(functionConfig, functionClass); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionConfig.getClassName()), e); - } + TypeDefinition[] typeArgs = FunctionCommon.getFunctionTypes(functionConfig, functionClass); // inputs use default schema, so there is no check needed there // Check if the Input serialization/deserialization class exists in jar or already loaded and that it // implements SerDe class if (functionConfig.getCustomSerdeInputs() != null) { functionConfig.getCustomSerdeInputs().forEach((topicName, inputSerializer) -> { - ValidatorUtils.validateSerde(inputSerializer, typeArgs[0], clsLoader, true); + ValidatorUtils.validateSerde(inputSerializer, typeArgs[0], validatableFunctionPackage.getTypePool(), + true); }); } @@ -632,8 +626,8 @@ private static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConf throw new IllegalArgumentException( String.format("Topic %s has an incorrect schema Info", topicName)); } - ValidatorUtils.validateSchema(consumerConfig.getSchemaType(), typeArgs[0], clsLoader, true); - + ValidatorUtils.validateSchema(consumerConfig.getSchemaType(), typeArgs[0], + validatableFunctionPackage.getTypePool(), true); }); } @@ -648,13 +642,16 @@ private static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConf "Only one of schemaType or serdeClassName should be set in inputSpec"); } if (!isEmpty(conf.getSerdeClassName())) { - ValidatorUtils.validateSerde(conf.getSerdeClassName(), typeArgs[0], clsLoader, true); + ValidatorUtils.validateSerde(conf.getSerdeClassName(), typeArgs[0], + validatableFunctionPackage.getTypePool(), true); } if (!isEmpty(conf.getSchemaType())) { - ValidatorUtils.validateSchema(conf.getSchemaType(), typeArgs[0], clsLoader, true); + ValidatorUtils.validateSchema(conf.getSchemaType(), typeArgs[0], + validatableFunctionPackage.getTypePool(), true); } if (conf.getCryptoConfig() != null) { - ValidatorUtils.validateCryptoKeyReader(conf.getCryptoConfig(), clsLoader, false); + ValidatorUtils.validateCryptoKeyReader(conf.getCryptoConfig(), + validatableFunctionPackage.getTypePool(), false); } }); } @@ -662,8 +659,8 @@ private static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConf if (Void.class.equals(typeArgs[1])) { return new FunctionConfigUtils.ExtractedFunctionDetails( functionClassName, - typeArgs[0].getName(), - typeArgs[1].getName()); + typeArgs[0].asErasure().getTypeName(), + typeArgs[1].asErasure().getTypeName()); } // One and only one of outputSchemaType and outputSerdeClassName should be set @@ -673,22 +670,25 @@ private static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConf } if (!isEmpty(functionConfig.getOutputSchemaType())) { - ValidatorUtils.validateSchema(functionConfig.getOutputSchemaType(), typeArgs[1], clsLoader, false); + ValidatorUtils.validateSchema(functionConfig.getOutputSchemaType(), typeArgs[1], + validatableFunctionPackage.getTypePool(), false); } if (!isEmpty(functionConfig.getOutputSerdeClassName())) { - ValidatorUtils.validateSerde(functionConfig.getOutputSerdeClassName(), typeArgs[1], clsLoader, false); + ValidatorUtils.validateSerde(functionConfig.getOutputSerdeClassName(), typeArgs[1], + validatableFunctionPackage.getTypePool(), false); } if (functionConfig.getProducerConfig() != null && functionConfig.getProducerConfig().getCryptoConfig() != null) { ValidatorUtils - .validateCryptoKeyReader(functionConfig.getProducerConfig().getCryptoConfig(), clsLoader, true); + .validateCryptoKeyReader(functionConfig.getProducerConfig().getCryptoConfig(), + validatableFunctionPackage.getTypePool(), true); } return new FunctionConfigUtils.ExtractedFunctionDetails( functionClassName, - typeArgs[0].getName(), - typeArgs[1].getName()); + typeArgs[0].asErasure().getTypeName(), + typeArgs[1].asErasure().getTypeName()); } private static void doPythonChecks(FunctionConfig functionConfig) { @@ -703,10 +703,6 @@ private static void doPythonChecks(FunctionConfig functionConfig) { if (functionConfig.getMaxMessageRetries() != null && functionConfig.getMaxMessageRetries() >= 0) { throw new IllegalArgumentException("Message retries not yet supported in python"); } - - if (functionConfig.getRetainKeyOrdering() != null && functionConfig.getRetainKeyOrdering()) { - throw new IllegalArgumentException("Retain Key Orderering not yet supported in python"); - } } private static void doGolangChecks(FunctionConfig functionConfig) { @@ -737,7 +733,7 @@ private static void verifyNoTopicClash(Collection inputTopics, String ou } } - private static void doCommonChecks(FunctionConfig functionConfig) { + public static void doCommonChecks(FunctionConfig functionConfig) { if (isEmpty(functionConfig.getTenant())) { throw new IllegalArgumentException("Function tenant cannot be null"); } @@ -879,7 +875,7 @@ private static void doCommonChecks(FunctionConfig functionConfig) { } } - private static Collection collectAllInputTopics(FunctionConfig functionConfig) { + public static Collection collectAllInputTopics(FunctionConfig functionConfig) { List retval = new LinkedList<>(); if (functionConfig.getInputs() != null) { retval.addAll(functionConfig.getInputs()); @@ -899,47 +895,21 @@ private static Collection collectAllInputTopics(FunctionConfig functionC return retval; } - public static ClassLoader validate(FunctionConfig functionConfig, File functionPackageFile) { + public static void validateNonJavaFunction(FunctionConfig functionConfig) { doCommonChecks(functionConfig); - if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { - ClassLoader classLoader; - if (functionPackageFile != null) { - try { - classLoader = loadJar(functionPackageFile); - } catch (MalformedURLException e) { - throw new IllegalArgumentException("Corrupted Jar File", e); - } - } else if (!isEmpty(functionConfig.getJar())) { - File jarFile = new File(functionConfig.getJar()); - if (!jarFile.exists()) { - throw new IllegalArgumentException("Jar file does not exist"); - } - try { - classLoader = loadJar(jarFile); - } catch (Exception e) { - throw new IllegalArgumentException("Corrupted Jar File", e); - } - } else { - throw new IllegalArgumentException("Function Package is not provided"); - } - - doJavaChecks(functionConfig, classLoader); - return classLoader; - } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.GO) { + if (functionConfig.getRuntime() == FunctionConfig.Runtime.GO) { doGolangChecks(functionConfig); - return null; } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.PYTHON) { doPythonChecks(functionConfig); - return null; } else { throw new IllegalArgumentException("Function language runtime is either not set or cannot be determined"); } } public static ExtractedFunctionDetails validateJavaFunction(FunctionConfig functionConfig, - ClassLoader classLoader) { + ValidatableFunctionPackage validatableFunctionPackage) { doCommonChecks(functionConfig); - return doJavaChecks(functionConfig, classLoader); + return doJavaChecks(functionConfig, validatableFunctionPackage); } public static FunctionConfig validateUpdate(FunctionConfig existingConfig, FunctionConfig newConfig) { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java new file mode 100644 index 0000000000000..80b0bc05b3a1f --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java @@ -0,0 +1,179 @@ +/** + * 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.pulsar.functions.utils; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.MalformedURLException; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.List; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.ClassFileLocator; +import net.bytebuddy.pool.TypePool; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.nar.NarClassLoaderBuilder; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; +import org.zeroturnaround.zip.ZipUtil; + +/** + * FunctionFilePackage is a class that represents a function package and + * implements the ValidatableFunctionPackage interface which decouples the + * function package from classloading. + */ +public class FunctionFilePackage implements AutoCloseable, ValidatableFunctionPackage { + private final File file; + private final ClassFileLocator.Compound classFileLocator; + private final TypePool typePool; + private final boolean isNar; + private final String narExtractionDirectory; + private final boolean enableClassloading; + + private ClassLoader classLoader; + + private final Object configMetadata; + + public FunctionFilePackage(File file, String narExtractionDirectory, boolean enableClassloading, + Class configClass) { + this.file = file; + boolean nonZeroFile = file.isFile() && file.length() > 0; + this.isNar = nonZeroFile ? ZipUtil.containsAnyEntry(file, + new String[] {"META-INF/services/pulsar-io.yaml", "META-INF/bundled-dependencies"}) : false; + this.narExtractionDirectory = narExtractionDirectory; + this.enableClassloading = enableClassloading; + if (isNar) { + List classpathFromArchive = null; + try { + classpathFromArchive = NarClassLoader.getClasspathFromArchive(file, narExtractionDirectory); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + List classFileLocators = new ArrayList<>(); + classFileLocators.add(ClassFileLocator.ForClassLoader.ofSystemLoader()); + for (File classpath : classpathFromArchive) { + if (classpath.exists()) { + try { + ClassFileLocator locator; + if (classpath.isDirectory()) { + locator = new ClassFileLocator.ForFolder(classpath); + } else { + locator = ClassFileLocator.ForJarFile.of(classpath); + } + classFileLocators.add(locator); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + this.classFileLocator = new ClassFileLocator.Compound(classFileLocators); + this.typePool = TypePool.Default.of(classFileLocator); + try { + this.configMetadata = FunctionUtils.getPulsarIOServiceConfig(file, configClass); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + try { + this.classFileLocator = nonZeroFile + ? new ClassFileLocator.Compound(ClassFileLocator.ForClassLoader.ofSystemLoader(), + ClassFileLocator.ForJarFile.of(file)) : + new ClassFileLocator.Compound(ClassFileLocator.ForClassLoader.ofSystemLoader()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + this.typePool = + TypePool.Default.of(classFileLocator); + this.configMetadata = null; + } + } + + public TypeDescription resolveType(String className) { + return typePool.describe(className).resolve(); + } + + public boolean isNar() { + return isNar; + } + + public File getFile() { + return file; + } + + public TypePool getTypePool() { + return typePool; + } + + @Override + public T getFunctionMetaData(Class clazz) { + return configMetadata != null ? clazz.cast(configMetadata) : null; + } + + @Override + public synchronized void close() throws IOException { + classFileLocator.close(); + if (classLoader instanceof Closeable) { + ((Closeable) classLoader).close(); + } + } + + public boolean isEnableClassloading() { + return enableClassloading; + } + + public synchronized ClassLoader getClassLoader() { + if (classLoader == null) { + classLoader = createClassLoader(); + } + return classLoader; + } + + private ClassLoader createClassLoader() { + if (enableClassloading) { + if (isNar) { + try { + return NarClassLoaderBuilder.builder() + .narFile(file) + .extractionDirectory(narExtractionDirectory) + .build(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + try { + return new URLClassLoader(new java.net.URL[] {file.toURI().toURL()}, + NarClassLoader.class.getClassLoader()); + } catch (MalformedURLException e) { + throw new UncheckedIOException(e); + } + } + } else { + throw new IllegalStateException("Classloading is not enabled"); + } + } + + @Override + public String toString() { + return "FunctionFilePackage{" + + "file=" + file + + ", isNar=" + isNar + + '}'; + } +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java new file mode 100644 index 0000000000000..948b1f1905a00 --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java @@ -0,0 +1,170 @@ +/** + * 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.pulsar.functions.utils; + +import static org.apache.commons.lang3.StringUtils.isEmpty; +import java.io.File; +import java.io.IOException; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.nar.NarClassLoaderBuilder; +import org.apache.pulsar.common.util.ClassLoaderUtils; +import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; +import org.apache.pulsar.functions.utils.io.ConnectorUtils; + +public class FunctionRuntimeCommon { + public static NarClassLoader extractNarClassLoader(File packageFile, + String narExtractionDirectory) { + if (packageFile != null) { + try { + return NarClassLoaderBuilder.builder() + .narFile(packageFile) + .extractionDirectory(narExtractionDirectory) + .build(); + } catch (IOException e) { + throw new IllegalArgumentException(e.getMessage()); + } + } + return null; + } + + public static ClassLoader getClassLoaderFromPackage( + Function.FunctionDetails.ComponentType componentType, + String className, + File packageFile, + String narExtractionDirectory) { + String connectorClassName = className; + ClassLoader jarClassLoader = null; + boolean keepJarClassLoader = false; + NarClassLoader narClassLoader = null; + boolean keepNarClassLoader = false; + + Exception jarClassLoaderException = null; + Exception narClassLoaderException = null; + + try { + try { + jarClassLoader = ClassLoaderUtils.extractClassLoader(packageFile); + } catch (Exception e) { + jarClassLoaderException = e; + } + try { + narClassLoader = extractNarClassLoader(packageFile, narExtractionDirectory); + } catch (Exception e) { + narClassLoaderException = e; + } + + // if connector class name is not provided, we can only try to load archive as a NAR + if (isEmpty(connectorClassName)) { + if (narClassLoader == null) { + throw new IllegalArgumentException(String.format("%s package does not have the correct format. " + + "Pulsar cannot determine if the package is a NAR package or JAR package. " + + "%s classname is not provided and attempts to load it as a NAR package produced " + + "the following error.", + FunctionCommon.capFirstLetter(componentType), FunctionCommon.capFirstLetter(componentType)), + narClassLoaderException); + } + try { + if (componentType == Function.FunctionDetails.ComponentType.FUNCTION) { + connectorClassName = FunctionUtils.getFunctionClass(narClassLoader); + } else if (componentType == Function.FunctionDetails.ComponentType.SOURCE) { + connectorClassName = ConnectorUtils.getIOSourceClass(narClassLoader); + } else { + connectorClassName = ConnectorUtils.getIOSinkClass(narClassLoader); + } + } catch (IOException e) { + throw new IllegalArgumentException(String.format("Failed to extract %s class from archive", + componentType.toString().toLowerCase()), e); + } + + try { + narClassLoader.loadClass(connectorClassName); + keepNarClassLoader = true; + return narClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e) { + throw new IllegalArgumentException(String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e); + } + + } else { + // if connector class name is provided, we need to try to load it as a JAR and as a NAR. + if (jarClassLoader != null) { + try { + jarClassLoader.loadClass(connectorClassName); + keepJarClassLoader = true; + return jarClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e) { + // class not found in JAR try loading as a NAR and searching for the class + if (narClassLoader != null) { + + try { + narClassLoader.loadClass(connectorClassName); + keepNarClassLoader = true; + return narClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e1) { + throw new IllegalArgumentException( + String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e1); + } + } else { + throw new IllegalArgumentException(String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e); + } + } + } else if (narClassLoader != null) { + try { + narClassLoader.loadClass(connectorClassName); + keepNarClassLoader = true; + return narClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e1) { + throw new IllegalArgumentException( + String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e1); + } + } else { + StringBuilder errorMsg = new StringBuilder(FunctionCommon.capFirstLetter(componentType) + + " package does not have the correct format." + + " Pulsar cannot determine if the package is a NAR package or JAR package."); + + if (jarClassLoaderException != null) { + errorMsg.append( + " Attempts to load it as a JAR package produced error: " + jarClassLoaderException + .getMessage()); + } + + if (narClassLoaderException != null) { + errorMsg.append( + " Attempts to load it as a NAR package produced error: " + narClassLoaderException + .getMessage()); + } + + throw new IllegalArgumentException(errorMsg.toString()); + } + } + } finally { + if (!keepJarClassLoader) { + ClassLoaderUtils.closeClassLoader(jarClassLoader); + } + if (!keepNarClassLoader) { + ClassLoaderUtils.closeClassLoader(narClassLoader); + } + } + } + +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java new file mode 100644 index 0000000000000..3f885a7cd2a3d --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java @@ -0,0 +1,89 @@ +/** + * 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.pulsar.functions.utils; + +import java.io.IOException; +import java.io.UncheckedIOException; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.ClassFileLocator; +import net.bytebuddy.pool.TypePool; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; + +/** + * LoadedFunctionPackage is a class that represents a function package and + * implements the ValidatableFunctionPackage interface which decouples the + * function package from classloading. This implementation is backed by + * a ClassLoader, and it is used when the function package is already loaded + * by a ClassLoader. This is the case in the LocalRunner and in some of + * the unit tests. + */ +public class LoadedFunctionPackage implements ValidatableFunctionPackage { + private final ClassLoader classLoader; + private final Object configMetadata; + private final TypePool typePool; + + public LoadedFunctionPackage(ClassLoader classLoader, Class configMetadataClass, T configMetadata) { + this.classLoader = classLoader; + this.configMetadata = configMetadata; + typePool = TypePool.Default.of( + ClassFileLocator.ForClassLoader.of(classLoader)); + } + + public LoadedFunctionPackage(ClassLoader classLoader, Class configMetadataClass) { + this.classLoader = classLoader; + if (classLoader instanceof NarClassLoader) { + try { + configMetadata = FunctionUtils.getPulsarIOServiceConfig((NarClassLoader) classLoader, + configMetadataClass); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + configMetadata = null; + } + typePool = TypePool.Default.of( + ClassFileLocator.ForClassLoader.of(classLoader)); + } + + @Override + public TypeDescription resolveType(String className) { + return typePool.describe(className).resolve(); + } + + @Override + public TypePool getTypePool() { + return typePool; + } + + @Override + public T getFunctionMetaData(Class clazz) { + return configMetadata != null ? clazz.cast(configMetadata) : null; + } + + @Override + public boolean isEnableClassloading() { + return true; + } + + @Override + public ClassLoader getClassLoader() { + return classLoader; + } +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java index 34574cc4d44d7..f4b7bf2513401 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SinkConfigUtils.java @@ -39,6 +39,8 @@ import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang.StringUtils; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.functions.ConsumerConfig; @@ -47,13 +49,11 @@ import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.config.validation.ConfigValidation; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; @Slf4j public class SinkConfigUtils { @@ -190,12 +190,6 @@ public static FunctionDetails convert(SinkConfig sinkConfig, ExtractedSinkDetail sourceSpecBuilder.setNegativeAckRedeliveryDelayMs(sinkConfig.getNegativeAckRedeliveryDelayMs()); } - if (sinkConfig.getCleanupSubscription() != null) { - sourceSpecBuilder.setCleanupSubscription(sinkConfig.getCleanupSubscription()); - } else { - sourceSpecBuilder.setCleanupSubscription(true); - } - if (sinkConfig.getSourceSubscriptionPosition() == SubscriptionInitialPosition.Earliest) { sourceSpecBuilder.setSubscriptionPosition(Function.SubscriptionPosition.EARLIEST); } else { @@ -319,7 +313,6 @@ public static SinkConfig convertFromDetails(FunctionDetails functionDetails) { // Set subscription position sinkConfig.setSourceSubscriptionPosition( convertFromFunctionDetailsSubscriptionPosition(functionDetails.getSource().getSubscriptionPosition())); - sinkConfig.setCleanupSubscription(functionDetails.getSource().getCleanupSubscription()); if (functionDetails.getSource().getTimeoutMs() != 0) { sinkConfig.setTimeoutMs(functionDetails.getSource().getTimeoutMs()); @@ -380,7 +373,7 @@ public static SinkConfig convertFromDetails(FunctionDetails functionDetails) { } public static ExtractedSinkDetails validateAndExtractDetails(SinkConfig sinkConfig, - ClassLoader sinkClassLoader, + ValidatableFunctionPackage sinkFunction, boolean validateConnectorConfig) { if (isEmpty(sinkConfig.getTenant())) { throw new IllegalArgumentException("Sink tenant cannot be null"); @@ -420,34 +413,38 @@ public static ExtractedSinkDetails validateAndExtractDetails(SinkConfig sinkConf // if class name in sink config is not set, this should be a built-in sink // thus we should try to find it class name in the NAR service definition if (sinkClassName == null) { - try { - sinkClassName = ConnectorUtils.getIOSinkClass((NarClassLoader) sinkClassLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract sink class from archive", e); + ConnectorDefinition connectorDefinition = sinkFunction.getFunctionMetaData(ConnectorDefinition.class); + if (connectorDefinition == null) { + throw new IllegalArgumentException( + "Sink package doesn't contain the META-INF/services/pulsar-io.yaml file."); + } + sinkClassName = connectorDefinition.getSinkClass(); + if (sinkClassName == null) { + throw new IllegalArgumentException("Failed to extract sink class from archive"); } } // check if sink implements the correct interfaces - Class sinkClass; + TypeDefinition sinkClass; try { - sinkClass = sinkClassLoader.loadClass(sinkClassName); - } catch (ClassNotFoundException e) { + sinkClass = sinkFunction.resolveType(sinkClassName); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("Sink class %s not found in class loader", sinkClassName), e); + String.format("Sink class %s not found", sinkClassName), e); } - // extract type from sink class - Class typeArg = getSinkType(sinkClass); + TypeDefinition typeArg = getSinkType(sinkClass); + ValidatableFunctionPackage inputFunction = sinkFunction; if (sinkConfig.getTopicToSerdeClassName() != null) { for (String serdeClassName : sinkConfig.getTopicToSerdeClassName().values()) { - ValidatorUtils.validateSerde(serdeClassName, typeArg, sinkClassLoader, true); + ValidatorUtils.validateSerde(serdeClassName, typeArg, inputFunction.getTypePool(), true); } } if (sinkConfig.getTopicToSchemaType() != null) { for (String schemaType : sinkConfig.getTopicToSchemaType().values()) { - ValidatorUtils.validateSchema(schemaType, typeArg, sinkClassLoader, true); + ValidatorUtils.validateSchema(schemaType, typeArg, inputFunction.getTypePool(), true); } } @@ -460,26 +457,46 @@ public static ExtractedSinkDetails validateAndExtractDetails(SinkConfig sinkConf throw new IllegalArgumentException("Only one of serdeClassName or schemaType should be set"); } if (!isEmpty(consumerSpec.getSerdeClassName())) { - ValidatorUtils.validateSerde(consumerSpec.getSerdeClassName(), typeArg, sinkClassLoader, true); + ValidatorUtils.validateSerde(consumerSpec.getSerdeClassName(), typeArg, + inputFunction.getTypePool(), true); } if (!isEmpty(consumerSpec.getSchemaType())) { - ValidatorUtils.validateSchema(consumerSpec.getSchemaType(), typeArg, sinkClassLoader, true); + ValidatorUtils.validateSchema(consumerSpec.getSchemaType(), typeArg, + inputFunction.getTypePool(), true); } if (consumerSpec.getCryptoConfig() != null) { - ValidatorUtils.validateCryptoKeyReader(consumerSpec.getCryptoConfig(), sinkClassLoader, false); + ValidatorUtils.validateCryptoKeyReader(consumerSpec.getCryptoConfig(), + inputFunction.getTypePool(), false); } } } - // validate user defined config if enabled and sink is loaded from NAR - if (validateConnectorConfig && sinkClassLoader instanceof NarClassLoader) { - validateSinkConfig(sinkConfig, (NarClassLoader) sinkClassLoader); + if (sinkConfig.getRetainKeyOrdering() != null + && sinkConfig.getRetainKeyOrdering() + && sinkConfig.getProcessingGuarantees() != null + && sinkConfig.getProcessingGuarantees() == FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE) { + throw new IllegalArgumentException( + "When effectively once processing guarantee is specified, retain Key ordering cannot be set"); + } + + if (sinkConfig.getRetainKeyOrdering() != null && sinkConfig.getRetainKeyOrdering() + && sinkConfig.getRetainOrdering() != null && sinkConfig.getRetainOrdering()) { + throw new IllegalArgumentException("Only one of retain ordering or retain key ordering can be set"); } - return new ExtractedSinkDetails(sinkClassName, typeArg.getName()); + // validate user defined config if enabled and classloading is enabled + if (validateConnectorConfig) { + if (sinkFunction.isEnableClassloading()) { + validateSinkConfig(sinkConfig, sinkFunction); + } else { + log.warn("Skipping annotation based validation of sink config as classloading is disabled"); + } + } + + return new ExtractedSinkDetails(sinkClassName, typeArg.asErasure().getTypeName()); } - private static Collection collectAllInputTopics(SinkConfig sinkConfig) { + public static Collection collectAllInputTopics(SinkConfig sinkConfig) { List retval = new LinkedList<>(); if (sinkConfig.getInputs() != null) { retval.addAll(sinkConfig.getInputs()); @@ -501,8 +518,8 @@ private static Collection collectAllInputTopics(SinkConfig sinkConfig) { @SneakyThrows public static SinkConfig clone(SinkConfig sinkConfig) { - return ObjectMapperFactory.getThreadLocal().readValue( - ObjectMapperFactory.getThreadLocal().writeValueAsBytes(sinkConfig), SinkConfig.class); + return ObjectMapperFactory.getThreadLocal().reader().readValue( + ObjectMapperFactory.getThreadLocal().writer().writeValueAsBytes(sinkConfig), SinkConfig.class); } public static SinkConfig validateUpdate(SinkConfig existingConfig, SinkConfig newConfig) { @@ -535,7 +552,7 @@ public static SinkConfig validateUpdate(SinkConfig existingConfig, SinkConfig ne if (newConfig.getInputs() != null) { newConfig.getInputs().forEach((topicName -> { - newConfig.getInputSpecs().put(topicName, + newConfig.getInputSpecs().putIfAbsent(topicName, ConsumerConfig.builder().isRegexPattern(false).build()); })); } @@ -619,36 +636,17 @@ public static SinkConfig validateUpdate(SinkConfig existingConfig, SinkConfig ne if (!StringUtils.isEmpty(newConfig.getCustomRuntimeOptions())) { mergedConfig.setCustomRuntimeOptions(newConfig.getCustomRuntimeOptions()); } - if (newConfig.getCleanupSubscription() != null) { - mergedConfig.setCleanupSubscription(newConfig.getCleanupSubscription()); - } return mergedConfig; } - public static void validateSinkConfig(SinkConfig sinkConfig, NarClassLoader narClassLoader) { - - if (sinkConfig.getRetainKeyOrdering() != null - && sinkConfig.getRetainKeyOrdering() - && sinkConfig.getProcessingGuarantees() != null - && sinkConfig.getProcessingGuarantees() == FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE) { - throw new IllegalArgumentException( - "When effectively once processing guarantee is specified, retain Key ordering cannot be set"); - } - - if (sinkConfig.getRetainKeyOrdering() != null && sinkConfig.getRetainKeyOrdering() - && sinkConfig.getRetainOrdering() != null && sinkConfig.getRetainOrdering()) { - throw new IllegalArgumentException("Only one of retain ordering or retain key ordering can be set"); - } - + public static void validateSinkConfig(SinkConfig sinkConfig, ValidatableFunctionPackage sinkFunction) { try { - ConnectorDefinition defn = ConnectorUtils.getConnectorDefinition(narClassLoader); - if (defn.getSinkConfigClass() != null) { - Class configClass = Class.forName(defn.getSinkConfigClass(), true, narClassLoader); + ConnectorDefinition defn = sinkFunction.getFunctionMetaData(ConnectorDefinition.class); + if (defn != null && defn.getSinkConfigClass() != null) { + Class configClass = Class.forName(defn.getSinkConfigClass(), true, sinkFunction.getClassLoader()); validateSinkConfig(sinkConfig, configClass); } - } catch (IOException e) { - throw new IllegalArgumentException("Error validating sink config", e); } catch (ClassNotFoundException e) { throw new IllegalArgumentException("Could not find sink config class", e); } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java index dd32903f4e44c..207d426ab540e 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java @@ -34,7 +34,9 @@ import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import net.jodah.typetools.TypeResolver; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; @@ -43,13 +45,11 @@ import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.config.validation.ConfigValidation; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; import org.apache.pulsar.io.core.BatchSource; import org.apache.pulsar.io.core.Source; @@ -285,7 +285,7 @@ public static SourceConfig convertFromDetails(FunctionDetails functionDetails) { } public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sourceConfig, - ClassLoader sourceClassLoader, + ValidatableFunctionPackage sourceFunction, boolean validateConnectorConfig) { if (isEmpty(sourceConfig.getTenant())) { throw new IllegalArgumentException("Source tenant cannot be null"); @@ -313,29 +313,34 @@ public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sour // if class name in source config is not set, this should be a built-in source // thus we should try to find it class name in the NAR service definition if (sourceClassName == null) { - try { - sourceClassName = ConnectorUtils.getIOSourceClass((NarClassLoader) sourceClassLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract source class from archive", e); + ConnectorDefinition connectorDefinition = sourceFunction.getFunctionMetaData(ConnectorDefinition.class); + if (connectorDefinition == null) { + throw new IllegalArgumentException( + "Source package doesn't contain the META-INF/services/pulsar-io.yaml file."); + } + sourceClassName = connectorDefinition.getSourceClass(); + if (sourceClassName == null) { + throw new IllegalArgumentException("Failed to extract source class from archive"); } } // check if source implements the correct interfaces - Class sourceClass; + TypeDescription sourceClass; try { - sourceClass = sourceClassLoader.loadClass(sourceClassName); - } catch (ClassNotFoundException e) { + sourceClass = sourceFunction.resolveType(sourceClassName); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( String.format("Source class %s not found in class loader", sourceClassName), e); } - if (!Source.class.isAssignableFrom(sourceClass) && !BatchSource.class.isAssignableFrom(sourceClass)) { + if (!(sourceClass.asErasure().isAssignableTo(Source.class) || sourceClass.asErasure() + .isAssignableTo(BatchSource.class))) { throw new IllegalArgumentException( - String.format("Source class %s does not implement the correct interface", - sourceClass.getName())); + String.format("Source class %s does not implement the correct interface", + sourceClass.getName())); } - if (BatchSource.class.isAssignableFrom(sourceClass)) { + if (sourceClass.asErasure().isAssignableTo(BatchSource.class)) { if (sourceConfig.getBatchSourceConfig() != null) { validateBatchSourceConfig(sourceConfig.getBatchSourceConfig()); } else { @@ -346,7 +351,14 @@ public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sour } // extract type from source class - Class typeArg = getSourceType(sourceClass); + TypeDefinition typeArg; + + try { + typeArg = getSourceType(sourceClass); + } catch (Exception e) { + throw new IllegalArgumentException( + String.format("Failed to resolve type for Source class %s", sourceClassName), e); + } // Only one of serdeClassName or schemaType should be set if (!StringUtils.isEmpty(sourceConfig.getSerdeClassName()) && !StringUtils @@ -355,29 +367,30 @@ public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sour } if (!StringUtils.isEmpty(sourceConfig.getSerdeClassName())) { - ValidatorUtils.validateSerde(sourceConfig.getSerdeClassName(), typeArg, sourceClassLoader, false); + ValidatorUtils.validateSerde(sourceConfig.getSerdeClassName(), typeArg, sourceFunction.getTypePool(), + false); } if (!StringUtils.isEmpty(sourceConfig.getSchemaType())) { - ValidatorUtils.validateSchema(sourceConfig.getSchemaType(), typeArg, sourceClassLoader, false); + ValidatorUtils.validateSchema(sourceConfig.getSchemaType(), typeArg, sourceFunction.getTypePool(), + false); } if (sourceConfig.getProducerConfig() != null && sourceConfig.getProducerConfig().getCryptoConfig() != null) { ValidatorUtils - .validateCryptoKeyReader(sourceConfig.getProducerConfig().getCryptoConfig(), sourceClassLoader, - true); + .validateCryptoKeyReader(sourceConfig.getProducerConfig().getCryptoConfig(), + sourceFunction.getTypePool(), true); } - if (typeArg.equals(TypeResolver.Unknown.class)) { - throw new IllegalArgumentException( - String.format("Failed to resolve type for Source class %s", sourceClassName)); - } - - // validate user defined config if enabled and source is loaded from NAR - if (validateConnectorConfig && sourceClassLoader instanceof NarClassLoader) { - validateSourceConfig(sourceConfig, (NarClassLoader) sourceClassLoader); + // validate user defined config if enabled and classloading is enabled + if (validateConnectorConfig) { + if (sourceFunction.isEnableClassloading()) { + validateSourceConfig(sourceConfig, sourceFunction); + } else { + log.warn("Skipping annotation based validation of sink config as classloading is disabled"); + } } - return new ExtractedSourceDetails(sourceClassName, typeArg.getName()); + return new ExtractedSourceDetails(sourceClassName, typeArg.asErasure().getTypeName()); } @SneakyThrows @@ -518,15 +531,14 @@ public static void validateBatchSourceConfigUpdate(BatchSourceConfig existingCon } } - public static void validateSourceConfig(SourceConfig sourceConfig, NarClassLoader narClassLoader) { + public static void validateSourceConfig(SourceConfig sourceConfig, ValidatableFunctionPackage sourceFunction) { try { - ConnectorDefinition defn = ConnectorUtils.getConnectorDefinition(narClassLoader); - if (defn.getSourceConfigClass() != null) { - Class configClass = Class.forName(defn.getSourceConfigClass(), true, narClassLoader); + ConnectorDefinition defn = sourceFunction.getFunctionMetaData(ConnectorDefinition.class); + if (defn != null && defn.getSourceConfigClass() != null) { + Class configClass = + Class.forName(defn.getSourceConfigClass(), true, sourceFunction.getClassLoader()); validateSourceConfig(sourceConfig, configClass); } - } catch (IOException e) { - throw new IllegalArgumentException("Error validating source config", e); } catch (ClassNotFoundException e) { throw new IllegalArgumentException("Could not find source config class"); } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.java new file mode 100644 index 0000000000000..bd71bbfb5a584 --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.java @@ -0,0 +1,59 @@ +/** + * 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.pulsar.functions.utils; + +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.pool.TypePool; + +/** + * This abstraction separates the function and connector definition from classloading, + * enabling validation without the need for classloading. It utilizes Byte Buddy for + * type and annotation resolution. + * + * The function or connector definition is directly extracted from the archive file, + * eliminating the need for classloader initialization. + * + * The getClassLoader method should only be invoked when classloading is enabled. + * Classloading is required in the LocalRunner and in the Functions worker when the + * worker is configured with the 'validateConnectorConfig' set to true. + */ +public interface ValidatableFunctionPackage { + /** + * Resolves the type description for the given class name within the function package. + */ + TypeDescription resolveType(String className); + /** + * Returns the Byte Buddy TypePool instance for the function package. + */ + TypePool getTypePool(); + /** + * Returns the function or connector definition metadata. + * Supports FunctionDefinition and ConnectorDefinition as the metadata type. + */ + T getFunctionMetaData(Class clazz); + /** + * Returns if classloading is enabled for the function package. + */ + boolean isEnableClassloading(); + /** + * Returns the classloader for the function package. The classloader is + * lazily initialized when classloading is enabled. + */ + ClassLoader getClassLoader(); +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java index 9e9ee27dfebad..1d12deaaea59d 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java @@ -18,35 +18,40 @@ */ package org.apache.pulsar.functions.utils; -import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isEmpty; -import static org.apache.commons.lang3.StringUtils.isNotBlank; import java.util.Map; import lombok.extern.slf4j.Slf4j; -import net.jodah.typetools.TypeResolver; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.pool.TypePool; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.functions.CryptoConfig; import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.common.util.ClassLoaderUtils; -import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.functions.api.SerDe; -import org.apache.pulsar.functions.proto.Function; -import org.apache.pulsar.io.core.Sink; -import org.apache.pulsar.io.core.Source; @Slf4j public class ValidatorUtils { private static final String DEFAULT_SERDE = "org.apache.pulsar.functions.api.utils.DefaultSerDe"; - public static void validateSchema(String schemaType, Class typeArg, ClassLoader clsLoader, + public static void validateSchema(String schemaType, TypeDefinition typeArg, TypePool typePool, boolean input) { if (isEmpty(schemaType) || getBuiltinSchemaType(schemaType) != null) { // If it's empty, we use the default schema and no need to validate // If it's built-in, no need to validate } else { - ClassLoaderUtils.implementsClass(schemaType, Schema.class, clsLoader); - validateSchemaType(schemaType, typeArg, clsLoader, input); + TypeDescription schemaClass = null; + try { + schemaClass = typePool.describe(schemaType).resolve(); + } catch (TypePool.Resolution.NoSuchTypeException e) { + throw new IllegalArgumentException( + String.format("The schema class %s does not exist", schemaType)); + } + if (!schemaClass.asErasure().isAssignableTo(Schema.class)) { + throw new IllegalArgumentException( + String.format("%s does not implement %s", schemaType, Schema.class.getName())); + } + validateSchemaType(schemaClass, typeArg, typePool, input); } } @@ -60,29 +65,32 @@ private static SchemaType getBuiltinSchemaType(String schemaTypeOrClassName) { } - public static void validateCryptoKeyReader(CryptoConfig conf, ClassLoader classLoader, boolean isProducer) { + public static void validateCryptoKeyReader(CryptoConfig conf, TypePool typePool, boolean isProducer) { if (isEmpty(conf.getCryptoKeyReaderClassName())) { return; } - Class cryptoClass; + String cryptoClassName = conf.getCryptoKeyReaderClassName(); + TypeDescription cryptoClass = null; try { - cryptoClass = ClassLoaderUtils.loadClass(conf.getCryptoKeyReaderClassName(), classLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { + cryptoClass = typePool.describe(cryptoClassName).resolve(); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("The crypto key reader class %s does not exist", conf.getCryptoKeyReaderClassName())); + String.format("The crypto key reader class %s does not exist", cryptoClassName)); + } + if (!cryptoClass.asErasure().isAssignableTo(CryptoKeyReader.class)) { + throw new IllegalArgumentException( + String.format("%s does not implement %s", cryptoClassName, CryptoKeyReader.class.getName())); } - ClassLoaderUtils.implementsClass(conf.getCryptoKeyReaderClassName(), CryptoKeyReader.class, classLoader); - try { - cryptoClass.getConstructor(Map.class); - } catch (NoSuchMethodException ex) { + boolean hasConstructor = cryptoClass.getDeclaredMethods().stream() + .anyMatch(method -> method.isConstructor() && method.getParameters().size() == 1 + && method.getParameters().get(0).getType().asErasure().represents(Map.class)); + + if (!hasConstructor) { throw new IllegalArgumentException( String.format("The crypto key reader class %s does not implement the desired constructor.", conf.getCryptoKeyReaderClassName())); - - } catch (SecurityException e) { - throw new IllegalArgumentException("Failed to access crypto key reader class", e); } if (isProducer && (conf.getEncryptionKeys() == null || conf.getEncryptionKeys().length == 0)) { @@ -90,7 +98,7 @@ public static void validateCryptoKeyReader(CryptoConfig conf, ClassLoader classL } } - public static void validateSerde(String inputSerializer, Class typeArg, ClassLoader clsLoader, + public static void validateSerde(String inputSerializer, TypeDefinition typeArg, TypePool typePool, boolean deser) { if (isEmpty(inputSerializer)) { return; @@ -98,154 +106,53 @@ public static void validateSerde(String inputSerializer, Class typeArg, Class if (inputSerializer.equals(DEFAULT_SERDE)) { return; } + TypeDescription serdeClass; try { - Class serdeClass = ClassLoaderUtils.loadClass(inputSerializer, clsLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { + serdeClass = typePool.describe(inputSerializer).resolve(); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( String.format("The input serialization/deserialization class %s does not exist", inputSerializer)); } - ClassLoaderUtils.implementsClass(inputSerializer, SerDe.class, clsLoader); - - SerDe serDe = (SerDe) Reflections.createInstance(inputSerializer, clsLoader); - if (serDe == null) { - throw new IllegalArgumentException(String.format("The SerDe class %s does not exist", - inputSerializer)); - } - Class[] serDeTypes = TypeResolver.resolveRawArguments(SerDe.class, serDe.getClass()); - - // type inheritance information seems to be lost in generic type - // load the actual type class for verification - Class fnInputClass; - Class serdeInputClass; - try { - fnInputClass = Class.forName(typeArg.getName(), true, clsLoader); - serdeInputClass = Class.forName(serDeTypes[0].getName(), true, clsLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException("Failed to load type class", e); - } + TypeDescription.Generic serDeTypeArg = serdeClass.getInterfaces().stream() + .filter(i -> i.asErasure().isAssignableTo(SerDe.class)) + .findFirst() + .map(i -> i.getTypeArguments().get(0)) + .orElseThrow(() -> new IllegalArgumentException( + String.format("%s does not implement %s", inputSerializer, SerDe.class.getName()))); if (deser) { - if (!fnInputClass.isAssignableFrom(serdeInputClass)) { - throw new IllegalArgumentException("Serializer type mismatch " + typeArg + " vs " + serDeTypes[0]); + if (!serDeTypeArg.asErasure().isAssignableTo(typeArg.asErasure())) { + throw new IllegalArgumentException("Serializer type mismatch " + typeArg.getActualName() + " vs " + + serDeTypeArg.getActualName()); } } else { - if (!serdeInputClass.isAssignableFrom(fnInputClass)) { - throw new IllegalArgumentException("Serializer type mismatch " + typeArg + " vs " + serDeTypes[0]); + if (!serDeTypeArg.asErasure().isAssignableFrom(typeArg.asErasure())) { + throw new IllegalArgumentException("Serializer type mismatch " + typeArg.getActualName() + " vs " + + serDeTypeArg.getActualName()); } } } - private static void validateSchemaType(String schemaClassName, Class typeArg, ClassLoader clsLoader, + private static void validateSchemaType(TypeDefinition schema, TypeDefinition typeArg, TypePool typePool, boolean input) { - Schema schema = (Schema) Reflections.createInstance(schemaClassName, clsLoader); - if (schema == null) { - throw new IllegalArgumentException(String.format("The Schema class %s does not exist", - schemaClassName)); - } - Class[] schemaTypes = TypeResolver.resolveRawArguments(Schema.class, schema.getClass()); - // type inheritance information seems to be lost in generic type - // load the actual type class for verification - Class fnInputClass; - Class schemaInputClass; - try { - fnInputClass = Class.forName(typeArg.getName(), true, clsLoader); - schemaInputClass = Class.forName(schemaTypes[0].getName(), true, clsLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException("Failed to load type class", e); - } + TypeDescription.Generic schemaTypeArg = schema.getInterfaces().stream() + .filter(i -> i.asErasure().isAssignableTo(Schema.class)) + .findFirst() + .map(i -> i.getTypeArguments().get(0)) + .orElse(null); if (input) { - if (!fnInputClass.isAssignableFrom(schemaInputClass)) { + if (!schemaTypeArg.asErasure().isAssignableTo(typeArg.asErasure())) { throw new IllegalArgumentException( - "Schema type mismatch " + typeArg + " vs " + schemaTypes[0]); + "Schema type mismatch " + typeArg.getActualName() + " vs " + schemaTypeArg.getActualName()); } } else { - if (!schemaInputClass.isAssignableFrom(fnInputClass)) { + if (!schemaTypeArg.asErasure().isAssignableFrom(typeArg.asErasure())) { throw new IllegalArgumentException( - "Schema type mismatch " + typeArg + " vs " + schemaTypes[0]); - } - } - } - - - public static void validateFunctionClassTypes(ClassLoader classLoader, - Function.FunctionDetails.Builder functionDetailsBuilder) { - - // validate only if classLoader is provided - if (classLoader == null) { - return; - } - - if (isBlank(functionDetailsBuilder.getClassName())) { - throw new IllegalArgumentException("Function class-name can't be empty"); - } - - // validate function class-type - Class functionClass; - try { - functionClass = classLoader.loadClass(functionDetailsBuilder.getClassName()); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionDetailsBuilder.getClassName()), e); - } - Class[] typeArgs = FunctionCommon.getFunctionTypes(functionClass, false); - - if (!(org.apache.pulsar.functions.api.Function.class.isAssignableFrom(functionClass)) - && !(java.util.function.Function.class.isAssignableFrom(functionClass))) { - throw new RuntimeException("User class must either be Function or java.util.Function"); - } - - if (functionDetailsBuilder.hasSource() && functionDetailsBuilder.getSource() != null - && isNotBlank(functionDetailsBuilder.getSource().getClassName())) { - try { - String sourceClassName = functionDetailsBuilder.getSource().getClassName(); - String argClassName = FunctionCommon.getTypeArg(sourceClassName, Source.class, classLoader).getName(); - functionDetailsBuilder - .setSource(functionDetailsBuilder.getSourceBuilder().setTypeClassName(argClassName)); - - // if sink-class not present then set same arg as source - if (!functionDetailsBuilder.hasSink() || isBlank(functionDetailsBuilder.getSink().getClassName())) { - functionDetailsBuilder - .setSink(functionDetailsBuilder.getSinkBuilder().setTypeClassName(argClassName)); - } - - } catch (IllegalArgumentException ie) { - throw ie; - } catch (Exception e) { - log.error("Failed to validate source class", e); - throw new IllegalArgumentException("Failed to validate source class-name", e); - } - } else if (isBlank(functionDetailsBuilder.getSourceBuilder().getTypeClassName())) { - // if function-src-class is not present then set function-src type-class according to function class - functionDetailsBuilder - .setSource(functionDetailsBuilder.getSourceBuilder().setTypeClassName(typeArgs[0].getName())); - } - - if (functionDetailsBuilder.hasSink() && functionDetailsBuilder.getSink() != null - && isNotBlank(functionDetailsBuilder.getSink().getClassName())) { - try { - String sinkClassName = functionDetailsBuilder.getSink().getClassName(); - String argClassName = FunctionCommon.getTypeArg(sinkClassName, Sink.class, classLoader).getName(); - functionDetailsBuilder.setSink(functionDetailsBuilder.getSinkBuilder().setTypeClassName(argClassName)); - - // if source-class not present then set same arg as sink - if (!functionDetailsBuilder.hasSource() || isBlank(functionDetailsBuilder.getSource().getClassName())) { - functionDetailsBuilder - .setSource(functionDetailsBuilder.getSourceBuilder().setTypeClassName(argClassName)); - } - - } catch (IllegalArgumentException ie) { - throw ie; - } catch (Exception e) { - log.error("Failed to validate sink class", e); - throw new IllegalArgumentException("Failed to validate sink class-name", e); + "Schema type mismatch " + typeArg.getActualName() + " vs " + schemaTypeArg.getActualName()); } - } else if (isBlank(functionDetailsBuilder.getSinkBuilder().getTypeClassName())) { - // if function-sink-class is not present then set function-sink type-class according to function class - functionDetailsBuilder - .setSink(functionDetailsBuilder.getSinkBuilder().setTypeClassName(typeArgs[1].getName())); } } } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java index 8d621cc965f80..54f3da7a448c5 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java @@ -19,14 +19,50 @@ package org.apache.pulsar.functions.utils.functions; import java.nio.file.Path; -import lombok.Builder; -import lombok.Data; import org.apache.pulsar.common.functions.FunctionDefinition; +import org.apache.pulsar.functions.utils.FunctionFilePackage; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; -@Builder -@Data -public class FunctionArchive { - private Path archivePath; - private ClassLoader classLoader; - private FunctionDefinition functionDefinition; +public class FunctionArchive implements AutoCloseable { + private final Path archivePath; + private final FunctionDefinition functionDefinition; + private final String narExtractionDirectory; + private final boolean enableClassloading; + private ValidatableFunctionPackage functionPackage; + private boolean closed; + + public FunctionArchive(Path archivePath, FunctionDefinition functionDefinition, String narExtractionDirectory, + boolean enableClassloading) { + this.archivePath = archivePath; + this.functionDefinition = functionDefinition; + this.narExtractionDirectory = narExtractionDirectory; + this.enableClassloading = enableClassloading; + } + + public Path getArchivePath() { + return archivePath; + } + + public synchronized ValidatableFunctionPackage getFunctionPackage() { + if (closed) { + throw new IllegalStateException("FunctionArchive is already closed"); + } + if (functionPackage == null) { + functionPackage = new FunctionFilePackage(archivePath.toFile(), narExtractionDirectory, enableClassloading, + FunctionDefinition.class); + } + return functionPackage; + } + + public FunctionDefinition getFunctionDefinition() { + return functionDefinition; + } + + @Override + public synchronized void close() throws Exception { + closed = true; + if (functionPackage instanceof AutoCloseable) { + ((AutoCloseable) functionPackage).close(); + } + } } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java index 8b70a4b650814..e09f3f1f12f7d 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionUtils.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.pulsar.functions.utils.functions; import java.io.File; @@ -30,10 +31,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.functions.api.Function; -import org.apache.pulsar.functions.utils.Exceptions; +import org.zeroturnaround.zip.ZipUtil; @UtilityClass @@ -45,43 +44,40 @@ public class FunctionUtils { /** * Extract the Pulsar Function class from a function or archive. */ - public static String getFunctionClass(ClassLoader classLoader) throws IOException { - NarClassLoader ncl = (NarClassLoader) classLoader; - String configStr = ncl.getServiceDefinition(PULSAR_IO_SERVICE_NAME); - - FunctionDefinition conf = ObjectMapperFactory.getThreadLocalYaml().readValue(configStr, - FunctionDefinition.class); - if (StringUtils.isEmpty(conf.getFunctionClass())) { - throw new IOException( - String.format("The '%s' functionctor does not provide a function implementation", conf.getName())); - } + public static String getFunctionClass(File narFile) throws IOException { + return getFunctionDefinition(narFile).getFunctionClass(); + } - try { - // Try to load source class and check it implements Function interface - Class functionClass = ncl.loadClass(conf.getFunctionClass()); - if (!(Function.class.isAssignableFrom(functionClass))) { - throw new IOException( - "Class " + conf.getFunctionClass() + " does not implement interface " + Function.class - .getName()); - } - } catch (Throwable t) { - Exceptions.rethrowIOException(t); + public static FunctionDefinition getFunctionDefinition(File narFile) throws IOException { + return getPulsarIOServiceConfig(narFile, FunctionDefinition.class); + } + + public static T getPulsarIOServiceConfig(File narFile, Class valueType) throws IOException { + String filename = "META-INF/services/" + PULSAR_IO_SERVICE_NAME; + byte[] configEntry = ZipUtil.unpackEntry(narFile, filename); + if (configEntry != null) { + return ObjectMapperFactory.getThreadLocalYaml().reader().readValue(configEntry, valueType); + } else { + return null; } + } - return conf.getFunctionClass(); + public static String getFunctionClass(NarClassLoader narClassLoader) throws IOException { + return getFunctionDefinition(narClassLoader).getFunctionClass(); } public static FunctionDefinition getFunctionDefinition(NarClassLoader narClassLoader) throws IOException { - String configStr = narClassLoader.getServiceDefinition(PULSAR_IO_SERVICE_NAME); - return ObjectMapperFactory.getThreadLocalYaml().readValue(configStr, FunctionDefinition.class); + return getPulsarIOServiceConfig(narClassLoader, FunctionDefinition.class); } - public static TreeMap searchForFunctions(String functionsDirectory) throws IOException { - return searchForFunctions(functionsDirectory, false); + public static T getPulsarIOServiceConfig(NarClassLoader narClassLoader, Class valueType) throws IOException { + return ObjectMapperFactory.getThreadLocalYaml().reader() + .readValue(narClassLoader.getServiceDefinition(PULSAR_IO_SERVICE_NAME), valueType); } public static TreeMap searchForFunctions(String functionsDirectory, - boolean alwaysPopulatePath) throws IOException { + String narExtractionDirectory, + boolean enableClassloading) throws IOException { Path path = Paths.get(functionsDirectory).toAbsolutePath(); log.info("Searching for functions in {}", path); @@ -95,22 +91,12 @@ public static TreeMap searchForFunctions(String functio try (DirectoryStream stream = Files.newDirectoryStream(path, "*.nar")) { for (Path archive : stream) { try { - - NarClassLoader ncl = NarClassLoaderBuilder.builder() - .narFile(new File(archive.toString())) - .build(); - - FunctionArchive.FunctionArchiveBuilder functionArchiveBuilder = FunctionArchive.builder(); - FunctionDefinition cntDef = FunctionUtils.getFunctionDefinition(ncl); + FunctionDefinition cntDef = FunctionUtils.getFunctionDefinition(archive.toFile()); log.info("Found function {} from {}", cntDef, archive); - - functionArchiveBuilder.archivePath(archive); - - functionArchiveBuilder.classLoader(ncl); - functionArchiveBuilder.functionDefinition(cntDef); - - if (alwaysPopulatePath || !StringUtils.isEmpty(cntDef.getFunctionClass())) { - functions.put(cntDef.getName(), functionArchiveBuilder.build()); + if (!StringUtils.isEmpty(cntDef.getFunctionClass())) { + FunctionArchive functionArchive = + new FunctionArchive(archive, cntDef, narExtractionDirectory, enableClassloading); + functions.put(cntDef.getName(), functionArchive); } } catch (Throwable t) { log.warn("Failed to load function from {}", archive, t); diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java index e2e6e39e273ff..b78c5c45fbd8b 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java @@ -20,17 +20,79 @@ import java.nio.file.Path; import java.util.List; -import lombok.Builder; -import lombok.Data; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.functions.utils.FunctionFilePackage; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; -@Builder -@Data -public class Connector { - private Path archivePath; +public class Connector implements AutoCloseable { + private final Path archivePath; + private final String narExtractionDirectory; + private final boolean enableClassloading; + private ValidatableFunctionPackage connectorFunctionPackage; private List sourceConfigFieldDefinitions; private List sinkConfigFieldDefinitions; - private ClassLoader classLoader; private ConnectorDefinition connectorDefinition; + private boolean closed; + + public Connector(Path archivePath, ConnectorDefinition connectorDefinition, String narExtractionDirectory, + boolean enableClassloading) { + this.archivePath = archivePath; + this.connectorDefinition = connectorDefinition; + this.narExtractionDirectory = narExtractionDirectory; + this.enableClassloading = enableClassloading; + } + + public Path getArchivePath() { + return archivePath; + } + + public synchronized ValidatableFunctionPackage getConnectorFunctionPackage() { + checkState(); + if (connectorFunctionPackage == null) { + connectorFunctionPackage = + new FunctionFilePackage(archivePath.toFile(), narExtractionDirectory, enableClassloading, + ConnectorDefinition.class); + } + return connectorFunctionPackage; + } + + private void checkState() { + if (closed) { + throw new IllegalStateException("Connector is already closed"); + } + } + + public synchronized List getSourceConfigFieldDefinitions() { + checkState(); + if (sourceConfigFieldDefinitions == null && !StringUtils.isEmpty(connectorDefinition.getSourceClass()) + && !StringUtils.isEmpty(connectorDefinition.getSourceConfigClass())) { + sourceConfigFieldDefinitions = ConnectorUtils.getConnectorConfigDefinition(getConnectorFunctionPackage(), + connectorDefinition.getSourceConfigClass()); + } + return sourceConfigFieldDefinitions; + } + + public synchronized List getSinkConfigFieldDefinitions() { + checkState(); + if (sinkConfigFieldDefinitions == null && !StringUtils.isEmpty(connectorDefinition.getSinkClass()) + && !StringUtils.isEmpty(connectorDefinition.getSinkConfigClass())) { + sinkConfigFieldDefinitions = ConnectorUtils.getConnectorConfigDefinition(getConnectorFunctionPackage(), + connectorDefinition.getSinkConfigClass()); + } + return sinkConfigFieldDefinitions; + } + + public ConnectorDefinition getConnectorDefinition() { + return connectorDefinition; + } + + @Override + public synchronized void close() throws Exception { + closed = true; + if (connectorFunctionPackage instanceof AutoCloseable) { + ((AutoCloseable) connectorFunctionPackage).close(); + } + } } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java index a4c74d0cdfdd0..24d4f8fbf68de 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/ConnectorUtils.java @@ -20,9 +20,6 @@ import java.io.File; import java.io.IOException; -import java.lang.annotation.Annotation; -import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; @@ -34,14 +31,18 @@ import java.util.TreeMap; import lombok.experimental.UtilityClass; import lombok.extern.slf4j.Slf4j; +import net.bytebuddy.description.annotation.AnnotationDescription; +import net.bytebuddy.description.annotation.AnnotationValue; +import net.bytebuddy.description.field.FieldDescription; +import net.bytebuddy.description.method.MethodDescription; +import net.bytebuddy.description.type.TypeDefinition; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; -import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.functions.utils.Exceptions; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; import org.apache.pulsar.io.core.BatchSource; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.Source; @@ -68,7 +69,7 @@ public static String getIOSourceClass(NarClassLoader narClassLoader) throws IOEx Class sourceClass = narClassLoader.loadClass(conf.getSourceClass()); if (!(Source.class.isAssignableFrom(sourceClass) || BatchSource.class.isAssignableFrom(sourceClass))) { throw new IOException(String.format("Class %s does not implement interface %s or %s", - conf.getSourceClass(), Source.class.getName(), BatchSource.class.getName())); + conf.getSourceClass(), Source.class.getName(), BatchSource.class.getName())); } } catch (Throwable t) { Exceptions.rethrowIOException(t); @@ -101,32 +102,36 @@ public static String getIOSinkClass(NarClassLoader narClassLoader) throws IOExce return conf.getSinkClass(); } - public static ConnectorDefinition getConnectorDefinition(NarClassLoader narClassLoader) throws IOException { - String configStr = narClassLoader.getServiceDefinition(PULSAR_IO_SERVICE_NAME); + public static ConnectorDefinition getConnectorDefinition(File narFile) throws IOException { + return FunctionUtils.getPulsarIOServiceConfig(narFile, ConnectorDefinition.class); + } - return ObjectMapperFactory.getThreadLocalYaml().readValue(configStr, ConnectorDefinition.class); + public static ConnectorDefinition getConnectorDefinition(NarClassLoader narClassLoader) throws IOException { + return FunctionUtils.getPulsarIOServiceConfig(narClassLoader, ConnectorDefinition.class); } - public static List getConnectorConfigDefinition(ClassLoader classLoader, - String configClassName) throws Exception { + public static List getConnectorConfigDefinition( + ValidatableFunctionPackage connectorFunctionPackage, + String configClassName) { List retval = new LinkedList<>(); - Class configClass = classLoader.loadClass(configClassName); - for (Field field : Reflections.getAllFields(configClass)) { - if (java.lang.reflect.Modifier.isStatic(field.getModifiers())) { - // We dont want static fields + TypeDefinition configClass = connectorFunctionPackage.resolveType(configClassName); + + for (FieldDescription field : getAllFields(configClass)) { + if (field.isStatic()) { + // We don't want static fields continue; } - field.setAccessible(true); ConfigFieldDefinition configFieldDefinition = new ConfigFieldDefinition(); configFieldDefinition.setFieldName(field.getName()); - configFieldDefinition.setTypeName(field.getType().getName()); + configFieldDefinition.setTypeName(field.getType().getActualName()); Map attributes = new HashMap<>(); - for (Annotation annotation : field.getAnnotations()) { - if (annotation.annotationType().equals(FieldDoc.class)) { - FieldDoc fieldDoc = (FieldDoc) annotation; - for (Method method : FieldDoc.class.getDeclaredMethods()) { - Object value = method.invoke(fieldDoc); - attributes.put(method.getName(), value == null ? "" : value.toString()); + for (AnnotationDescription annotation : field.getDeclaredAnnotations()) { + if (annotation.getAnnotationType().represents(FieldDoc.class)) { + for (MethodDescription.InDefinedShape method : annotation.getAnnotationType() + .getDeclaredMethods()) { + AnnotationValue value = annotation.getValue(method.getName()); + attributes.put(method.getName(), + value == null || value.resolve() == null ? "" : value.resolve().toString()); } } } @@ -137,12 +142,25 @@ public static List getConnectorConfigDefinition(ClassLoad return retval; } + private static List getAllFields(TypeDefinition type) { + List fields = new LinkedList<>(); + fields.addAll(type.getDeclaredFields()); + + if (type.getSuperClass() != null) { + fields.addAll(getAllFields(type.getSuperClass())); + } + + return fields; + } + public static TreeMap searchForConnectors(String connectorsDirectory, - String narExtractionDirectory) throws IOException { + String narExtractionDirectory, + boolean enableClassloading) throws IOException { Path path = Paths.get(connectorsDirectory).toAbsolutePath(); log.info("Searching for connectors in {}", path); TreeMap connectors = new TreeMap<>(); + if (!path.toFile().exists()) { log.warn("Connectors archive directory not found"); return connectors; @@ -151,40 +169,15 @@ public static TreeMap searchForConnectors(String connectorsDi try (DirectoryStream stream = Files.newDirectoryStream(path, "*.nar")) { for (Path archive : stream) { try { - - NarClassLoader ncl = NarClassLoaderBuilder.builder() - .narFile(new File(archive.toString())) - .extractionDirectory(narExtractionDirectory) - .build(); - - Connector.ConnectorBuilder connectorBuilder = Connector.builder(); - ConnectorDefinition cntDef = ConnectorUtils.getConnectorDefinition(ncl); + ConnectorDefinition cntDef = ConnectorUtils.getConnectorDefinition(archive.toFile()); log.info("Found connector {} from {}", cntDef, archive); - - connectorBuilder.archivePath(archive); - if (!StringUtils.isEmpty(cntDef.getSourceClass())) { - if (!StringUtils.isEmpty(cntDef.getSourceConfigClass())) { - connectorBuilder.sourceConfigFieldDefinitions(ConnectorUtils - .getConnectorConfigDefinition(ncl, cntDef.getSourceConfigClass())); - } - } - - if (!StringUtils.isEmpty(cntDef.getSinkClass())) { - if (!StringUtils.isEmpty(cntDef.getSinkConfigClass())) { - connectorBuilder.sinkConfigFieldDefinitions( - ConnectorUtils.getConnectorConfigDefinition(ncl, cntDef.getSinkConfigClass())); - } - } - - connectorBuilder.classLoader(ncl); - connectorBuilder.connectorDefinition(cntDef); - connectors.put(cntDef.getName(), connectorBuilder.build()); + Connector connector = new Connector(archive, cntDef, narExtractionDirectory, enableClassloading); + connectors.put(cntDef.getName(), connector); } catch (Throwable t) { log.warn("Failed to load connector from {}", archive, t); } } - - return connectors; } + return connectors; } } diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java index b991f8b4f473c..98c8dacb25c12 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java @@ -19,9 +19,16 @@ package org.apache.pulsar.functions.utils; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import java.io.File; import java.util.Collection; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.pool.TypePool; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.api.Record; @@ -31,52 +38,10 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.io.File; -import java.util.UUID; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; - /** * Unit test of {@link Exceptions}. */ public class FunctionCommonTest { - - @Test - public void testValidateLocalFileUrl() throws Exception { - String fileLocation = FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - try { - // eg: fileLocation : /dir/fileName.jar (invalid) - FunctionCommon.extractClassLoader(fileLocation); - Assert.fail("should fail with invalid url: without protocol"); - } catch (IllegalArgumentException ie) { - // Ok.. expected exception - } - String fileLocationWithProtocol = "file://" + fileLocation; - // eg: fileLocation : file:///dir/fileName.jar (valid) - FunctionCommon.extractClassLoader(fileLocationWithProtocol); - // eg: fileLocation : file:/dir/fileName.jar (valid) - fileLocationWithProtocol = "file:" + fileLocation; - FunctionCommon.extractClassLoader(fileLocationWithProtocol); - } - - @Test - public void testValidateHttpFileUrl() throws Exception { - - String jarHttpUrl = "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; - FunctionCommon.extractClassLoader(jarHttpUrl); - - jarHttpUrl = "http://_invalidurl_.com"; - try { - // eg: fileLocation : /dir/fileName.jar (invalid) - FunctionCommon.extractClassLoader(jarHttpUrl); - Assert.fail("should fail with invalid url: without protocol"); - } catch (Exception ie) { - // Ok.. expected exception - } - } - @Test public void testDownloadFile() throws Exception { String jarHttpUrl = "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; @@ -128,6 +93,14 @@ public Record process(String input, Context context) throws Exception { } }, false }, + { + new Function>() { + @Override + public CompletableFuture process(String input, Context context) throws Exception { + return null; + } + }, false + }, { new java.util.function.Function() { @Override @@ -144,6 +117,14 @@ public Record apply(String s) { } }, false }, + { + new java.util.function.Function>() { + @Override + public CompletableFuture apply(String s) { + return null; + } + }, false + }, { new WindowFunction() { @Override @@ -160,6 +141,14 @@ public Record process(Collection> input, WindowContext c } }, true }, + { + new WindowFunction>() { + @Override + public CompletableFuture process(Collection> input, WindowContext context) throws Exception { + return null; + } + }, true + }, { new java.util.function.Function, Integer>() { @Override @@ -175,15 +164,26 @@ public Record apply(Collection strings) { return null; } }, true + }, + { + new java.util.function.Function, CompletableFuture>() { + @Override + public CompletableFuture apply(Collection strings) { + return null; + } + }, true } }; } @Test(dataProvider = "function") public void testGetFunctionTypes(Object function, boolean isWindowConfigPresent) { - Class[] types = FunctionCommon.getFunctionTypes(function.getClass(), isWindowConfigPresent); + TypePool typePool = TypePool.Default.of(function.getClass().getClassLoader()); + TypeDefinition[] types = + FunctionCommon.getFunctionTypes(typePool.describe(function.getClass().getName()).resolve(), + isWindowConfigPresent); assertEquals(types.length, 2); - assertEquals(types[0], String.class); - assertEquals(types[1], Integer.class); + assertEquals(types[0].asErasure().getTypeName(), String.class.getName()); + assertEquals(types[1].asErasure().getTypeName(), Integer.class.getName()); } } diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java index d2b387cdf9938..85d481c252422 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java @@ -18,12 +18,23 @@ */ package org.apache.pulsar.functions.utils; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; +import static org.apache.pulsar.common.functions.FunctionConfig.Runtime.PYTHON; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; import com.google.gson.Gson; - -import org.apache.pulsar.client.api.SubscriptionInitialPosition; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.functions.ConsumerConfig; import org.apache.pulsar.common.functions.FunctionConfig; @@ -31,28 +42,29 @@ import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.functions.WindowConfig; import org.apache.pulsar.common.util.Reflections; +import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.functions.api.WindowContext; +import org.apache.pulsar.functions.api.WindowFunction; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; -import static org.apache.pulsar.common.functions.FunctionConfig.Runtime.PYTHON; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; - /** * Unit test of {@link Reflections}. */ @Slf4j public class FunctionConfigUtilsTest { + public static class WordCountWindowFunction implements WindowFunction { + @Override + public Void process(Collection> inputs, WindowContext context) throws Exception { + for (Record input : inputs) { + Arrays.asList(input.getValue().split("\\.")).forEach(word -> context.incrCounter(word, 1)); + } + return null; + } + } + @Test public void testAutoAckConvertFailed() { @@ -62,7 +74,7 @@ public void testAutoAckConvertFailed() { functionConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATMOST_ONCE); assertThrows(IllegalArgumentException.class, () -> { - FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + FunctionConfigUtils.convert(functionConfig); }); } @@ -97,7 +109,7 @@ public void testConvertBackFidelity() { producerConfig.setUseThreadLocalProducers(true); producerConfig.setBatchBuilder("DEFAULT"); functionConfig.setProducerConfig(producerConfig); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); // add default resources @@ -117,7 +129,7 @@ public void testConvertWindow() { functionConfig.setNamespace("test-namespace"); functionConfig.setName("test-function"); functionConfig.setParallelism(1); - functionConfig.setClassName(IdentityFunction.class.getName()); + functionConfig.setClassName(WordCountWindowFunction.class.getName()); Map inputSpecs = new HashMap<>(); inputSpecs.put("test-input", ConsumerConfig.builder().isRegexPattern(true).serdeClassName("test-serde").build()); functionConfig.setInputSpecs(inputSpecs); @@ -138,7 +150,7 @@ public void testConvertWindow() { producerConfig.setUseThreadLocalProducers(true); producerConfig.setBatchBuilder("KEY_BASED"); functionConfig.setProducerConfig(producerConfig); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); // WindowsFunction guarantees convert to FunctionGuarantees. @@ -160,7 +172,7 @@ public void testConvertBatchBuilder() { FunctionConfig functionConfig = createFunctionConfig(); functionConfig.setBatchBuilder("KEY_BASED"); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); assertEquals(functionDetails.getSink().getProducerSpec().getBatchBuilder(), "KEY_BASED"); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); @@ -515,7 +527,6 @@ private FunctionConfig createFunctionConfig() { functionConfig.setUserConfig(new HashMap<>()); functionConfig.setAutoAck(true); functionConfig.setTimeoutMs(2000L); - functionConfig.setWindowConfig(new WindowConfig().setWindowLengthCount(10)); functionConfig.setCleanupSubscription(true); functionConfig.setRuntimeFlags("-Dfoo=bar"); return functionConfig; @@ -549,7 +560,7 @@ public void testDisableForwardSourceMessageProperty() throws InvalidProtocolBuff config.setForwardSourceMessageProperty(true); FunctionConfigUtils.inferMissingArguments(config, false); assertNull(config.getForwardSourceMessageProperty()); - FunctionDetails details = FunctionConfigUtils.convert(config, FunctionConfigUtilsTest.class.getClassLoader()); + FunctionDetails details = FunctionConfigUtils.convert(config); assertFalse(details.getSink().getForwardSourceMessageProperty()); String detailsJson = "'" + JsonFormat.printer().omittingInsignificantWhitespace().print(details) + "'"; log.info("Function details : {}", detailsJson); @@ -636,7 +647,7 @@ public void testMergeDifferentOutputSchemaTypes() { @Test public void testPoolMessages() { FunctionConfig functionConfig = createFunctionConfig(); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); assertFalse(functionDetails.getSource().getInputSpecsMap().get("test-input").getPoolMessages()); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); assertFalse(convertedConfig.getInputSpecs().get("test-input").isPoolMessages()); @@ -646,7 +657,7 @@ public void testPoolMessages() { .poolMessages(true).build()); functionConfig.setInputSpecs(inputSpecs); - functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + functionDetails = FunctionConfigUtils.convert(functionConfig); assertTrue(functionDetails.getSource().getInputSpecsMap().get("test-input").getPoolMessages()); convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java index 701e852fa7d8c..cb55eebbe50b3 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/SinkConfigUtilsTest.java @@ -18,36 +18,38 @@ */ package org.apache.pulsar.functions.utils; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATMOST_ONCE; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; import com.google.common.collect.Lists; import com.google.gson.Gson; +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import lombok.Data; import lombok.NoArgsConstructor; import lombok.experimental.Accessors; import org.apache.pulsar.common.functions.ConsumerConfig; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.Resources; +import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.config.validation.ConfigValidationAnnotations; -import org.apache.pulsar.functions.api.utils.IdentityFunction; +import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.io.core.Sink; +import org.apache.pulsar.io.core.SinkContext; import org.testng.annotations.Test; -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE; -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATMOST_ONCE; -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.expectThrows; - /** * Unit test of {@link SinkConfigUtilsTest}. */ @@ -61,6 +63,27 @@ public static class TestSinkConfig { private String configParameter; } + + public static class NopSink implements Sink { + + @Override + public void open(Map config, SinkContext sinkContext) throws Exception { + + } + + @Override + public void write(Record record) throws Exception { + + } + + @Override + public void close() throws Exception { + + } + } + + + @Test public void testAutoAckConvertFailed() throws IOException { @@ -147,7 +170,7 @@ public void testParseRetainOrderingField() throws IOException { SinkConfig sinkConfig = createSinkConfig(); sinkConfig.setRetainOrdering(testcase); Function.FunctionDetails functionDetails = SinkConfigUtils.convert(sinkConfig, new SinkConfigUtils.ExtractedSinkDetails(null, null)); - assertEquals(functionDetails.getRetainOrdering(), testcase != null ? testcase : false); + assertEquals(functionDetails.getRetainOrdering(), testcase != null ? testcase.booleanValue() : false); SinkConfig result = SinkConfigUtils.convertFromDetails(functionDetails); assertEquals(result.getRetainOrdering(), testcase != null ? testcase : Boolean.valueOf(false)); } @@ -160,7 +183,7 @@ public void testParseKeyRetainOrderingField() throws IOException { SinkConfig sinkConfig = createSinkConfig(); sinkConfig.setRetainKeyOrdering(testcase); Function.FunctionDetails functionDetails = SinkConfigUtils.convert(sinkConfig, new SinkConfigUtils.ExtractedSinkDetails(null, null)); - assertEquals(functionDetails.getRetainKeyOrdering(), testcase != null ? testcase : false); + assertEquals(functionDetails.getRetainKeyOrdering(), testcase != null ? testcase.booleanValue() : false); SinkConfig result = SinkConfigUtils.convertFromDetails(functionDetails); assertEquals(result.getRetainKeyOrdering(), testcase != null ? testcase : Boolean.valueOf(false)); } @@ -276,6 +299,24 @@ public void testMergeDifferentInputSpec() { assertEquals(sinkConfig.getInputSpecs().get("test-input").getReceiverQueueSize().intValue(), 1000); } + @Test + public void testMergeDifferentInputSpecWithInputsSet() { + SinkConfig sinkConfig = createSinkConfig(); + sinkConfig.getInputSpecs().put("test-input", ConsumerConfig.builder().isRegexPattern(false).receiverQueueSize(1000).build()); + + Map inputSpecs = new HashMap<>(); + ConsumerConfig newConsumerConfig = ConsumerConfig.builder().isRegexPattern(false).serdeClassName("test-serde").receiverQueueSize(58).build(); + inputSpecs.put("test-input", newConsumerConfig); + SinkConfig newSinkConfig = createUpdatedSinkConfig("inputSpecs", inputSpecs); + newSinkConfig.setInputs(new ArrayList<>()); + newSinkConfig.getInputs().add("test-input"); + SinkConfig mergedConfig = SinkConfigUtils.validateUpdate(sinkConfig, newSinkConfig); + assertEquals(mergedConfig.getInputSpecs().get("test-input"), newConsumerConfig); + + // make sure original sinkConfig was not modified + assertEquals(sinkConfig.getInputSpecs().get("test-input").getReceiverQueueSize().intValue(), 1000); + } + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Processing Guarantees cannot be altered") public void testMergeDifferentProcessingGuarantees() { SinkConfig sinkConfig = createSinkConfig(); @@ -448,7 +489,7 @@ private SinkConfig createSinkConfig() { sinkConfig.setNamespace("test-namespace"); sinkConfig.setName("test-sink"); sinkConfig.setParallelism(1); - sinkConfig.setClassName(IdentityFunction.class.getName()); + sinkConfig.setClassName(NopSink.class.getName()); Map inputSpecs = new HashMap<>(); inputSpecs.put("test-input", ConsumerConfig.builder().isRegexPattern(true).serdeClassName("test-serde").build()); sinkConfig.setInputSpecs(inputSpecs); @@ -501,13 +542,13 @@ public void testAllowDisableSinkTimeout() { SinkConfig sinkConfig = createSinkConfig(); sinkConfig.setInputSpecs(null); sinkConfig.setTopicsPattern("my-topic-*"); - SinkConfigUtils.validateAndExtractDetails(sinkConfig, this.getClass().getClassLoader(), - true); + LoadedFunctionPackage validatableFunction = + new LoadedFunctionPackage(this.getClass().getClassLoader(), ConnectorDefinition.class); + + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunction, true); sinkConfig.setTimeoutMs(null); - SinkConfigUtils.validateAndExtractDetails(sinkConfig, this.getClass().getClassLoader(), - true); + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunction, true); sinkConfig.setTimeoutMs(0L); - SinkConfigUtils.validateAndExtractDetails(sinkConfig, this.getClass().getClassLoader(), - true); + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunction, true); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java index 01b240d7f97a1..542eb30062a48 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java @@ -69,6 +69,7 @@ import org.apache.pulsar.functions.utils.Actions; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SourceConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; @Data @@ -496,7 +497,7 @@ private File getBuiltinArchive(FunctionDetails.Builder functionDetails) throws I builder.setClassName(sourceClass); functionDetails.setSource(builder); - fillSourceTypeClass(functionDetails, connector.getClassLoader(), sourceClass); + fillSourceTypeClass(functionDetails, connector.getConnectorFunctionPackage(), sourceClass); return archive; } } @@ -512,7 +513,7 @@ private File getBuiltinArchive(FunctionDetails.Builder functionDetails) throws I builder.setClassName(sinkClass); functionDetails.setSink(builder); - fillSinkTypeClass(functionDetails, connector.getClassLoader(), sinkClass); + fillSinkTypeClass(functionDetails, connector.getConnectorFunctionPackage(), sinkClass); return archive; } } @@ -525,8 +526,8 @@ private File getBuiltinArchive(FunctionDetails.Builder functionDetails) throws I } private void fillSourceTypeClass(FunctionDetails.Builder functionDetails, - ClassLoader narClassLoader, String className) throws ClassNotFoundException { - String typeArg = getSourceType(className, narClassLoader).getName(); + ValidatableFunctionPackage functionPackage, String className) { + String typeArg = getSourceType(className, functionPackage.getTypePool()).asErasure().getName(); SourceSpec.Builder sourceBuilder = SourceSpec.newBuilder(functionDetails.getSource()); sourceBuilder.setTypeClassName(typeArg); @@ -541,8 +542,8 @@ private void fillSourceTypeClass(FunctionDetails.Builder functionDetails, } private void fillSinkTypeClass(FunctionDetails.Builder functionDetails, - ClassLoader narClassLoader, String className) throws ClassNotFoundException { - String typeArg = getSinkType(className, narClassLoader).getName(); + ValidatableFunctionPackage functionPackage, String className) { + String typeArg = getSinkType(className, functionPackage.getTypePool()).asErasure().getName(); SinkSpec.Builder sinkBuilder = SinkSpec.newBuilder(functionDetails.getSink()); sinkBuilder.setTypeClassName(typeArg); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 307e63262afb5..70237305a336f 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -663,6 +663,14 @@ public void stop() { if (statsUpdater != null) { statsUpdater.shutdownNow(); } + + if (null != functionsManager) { + functionsManager.close(); + } + + if (null != connectorsManager) { + connectorsManager.close(); + } } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 030367e8864ef..7f7b3b78efbee 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -29,7 +29,6 @@ import static org.apache.pulsar.functions.utils.FunctionCommon.getUniquePackageName; import static org.apache.pulsar.functions.utils.FunctionCommon.isFunctionCodeBuiltin; import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; -import static org.apache.pulsar.functions.worker.rest.api.FunctionsImpl.downloadPackageFile; import com.google.common.base.Utf8; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; @@ -43,6 +42,7 @@ import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Base64; import java.util.Collection; import java.util.LinkedList; @@ -66,6 +66,7 @@ import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationParameters; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -100,6 +101,7 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; @@ -1764,9 +1766,88 @@ private void internalProcessFunctionRequest(final String tenant, final String na } } - protected ClassLoader getClassLoaderFromPackage(String className, - File packageFile, - String narExtractionDirectory) { - return FunctionCommon.getClassLoaderFromPackage(componentType, className, packageFile, narExtractionDirectory); + static File downloadPackageFile(PulsarWorkerService worker, String packageName) + throws IOException, PulsarAdminException { + Path tempDirectory; + if (worker.getWorkerConfig().getDownloadDirectory() != null) { + tempDirectory = Paths.get(worker.getWorkerConfig().getDownloadDirectory()); + } else { + // use the Nar extraction directory as a temporary directory for downloaded files + tempDirectory = Paths.get(worker.getWorkerConfig().getNarExtractionDirectory()); + } + Files.createDirectories(tempDirectory); + File file = Files.createTempFile(tempDirectory, "function", ".tmp").toFile(); + worker.getBrokerAdmin().packages().download(packageName, file.toString()); + return file; + } + + protected File getPackageFile(String functionPkgUrl, String existingPackagePath, InputStream uploadedInputStream) + throws IOException, PulsarAdminException { + File componentPackageFile = null; + if (isNotBlank(functionPkgUrl)) { + componentPackageFile = getPackageFile(functionPkgUrl); + } else if (existingPackagePath.startsWith(Utils.FILE) || existingPackagePath.startsWith(Utils.HTTP)) { + try { + componentPackageFile = FunctionCommon.extractFileFromPkgURL(existingPackagePath); + } catch (Exception e) { + throw new IllegalArgumentException(String.format("Encountered error \"%s\" " + + "when getting %s package from %s", e.getMessage(), + ComponentTypeUtils.toString(componentType), functionPkgUrl)); + } + } else if (Utils.hasPackageTypePrefix(existingPackagePath)) { + componentPackageFile = getPackageFile(existingPackagePath); + } else if (uploadedInputStream != null) { + componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); + } else if (!existingPackagePath.startsWith(Utils.BUILTIN)) { + componentPackageFile = FunctionCommon.createPkgTempFile(); + componentPackageFile.deleteOnExit(); + if (worker().getWorkerConfig().isFunctionsWorkerEnablePackageManagement()) { + worker().getBrokerAdmin().packages().download( + existingPackagePath, + componentPackageFile.getAbsolutePath()); + } else { + WorkerUtils.downloadFromBookkeeper(worker().getDlogNamespace(), + componentPackageFile, existingPackagePath); + } + } + return componentPackageFile; + } + + protected File downloadPackageFile(String packageName) throws IOException, PulsarAdminException { + return downloadPackageFile(worker(), packageName); + } + + protected File getPackageFile(String functionPkgUrl) throws IOException, PulsarAdminException { + if (Utils.hasPackageTypePrefix(functionPkgUrl)) { + return downloadPackageFile(functionPkgUrl); + } else { + if (!Utils.isFunctionPackageUrlSupported(functionPkgUrl)) { + throw new IllegalArgumentException("Function Package url is not valid." + + "supported url (http/https/file)"); + } + try { + return FunctionCommon.extractFileFromPkgURL(functionPkgUrl); + } catch (Exception e) { + throw new IllegalArgumentException(String.format("Encountered error \"%s\" " + + "when getting %s package from %s", e.getMessage(), + ComponentTypeUtils.toString(componentType), functionPkgUrl), e); + } + } + } + + protected ValidatableFunctionPackage getBuiltinFunctionPackage(String archive) { + if (!StringUtils.isEmpty(archive)) { + if (archive.startsWith(org.apache.pulsar.common.functions.Utils.BUILTIN)) { + archive = archive.replaceFirst("^builtin://", ""); + + FunctionArchive function = worker().getFunctionsManager().getFunction(archive); + // check if builtin connector exists + if (function == null) { + throw new IllegalArgumentException("Built-in " + componentType + " is not available"); + } + return function.getFunctionPackage(); + } + } + return null; } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index 4133540cfd1ea..4f1b4477f1de3 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -28,9 +28,6 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.Collection; import java.util.LinkedList; import java.util.List; @@ -44,12 +41,12 @@ import org.apache.pulsar.broker.authentication.AuthenticationParameters; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.functions.Utils; import org.apache.pulsar.common.functions.WorkerInfo; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.FunctionStatus; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; @@ -58,11 +55,14 @@ import org.apache.pulsar.functions.utils.ComponentTypeUtils; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; +import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.FunctionsManager; import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.service.api.Functions; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -767,11 +767,12 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant functionConfig.setTenant(tenant); functionConfig.setNamespace(namespace); functionConfig.setName(componentName); + WorkerConfig workerConfig = worker().getWorkerConfig(); FunctionConfigUtils.inferMissingArguments( - functionConfig, worker().getWorkerConfig().isForwardSourceMessageProperty()); + functionConfig, workerConfig.isForwardSourceMessageProperty()); String archive = functionConfig.getJar(); - ClassLoader classLoader = null; + ValidatableFunctionPackage functionPackage = null; // check if function is builtin and extract classloader if (!StringUtils.isEmpty(archive)) { if (archive.startsWith(org.apache.pulsar.common.functions.Utils.BUILTIN)) { @@ -784,55 +785,39 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (function == null) { throw new IllegalArgumentException(String.format("No Function %s found", archive)); } - classLoader = function.getClassLoader(); + functionPackage = function.getFunctionPackage(); } } - boolean shouldCloseClassLoader = false; + boolean shouldCloseFunctionPackage = false; try { - if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { // if function is not builtin, attempt to extract classloader from package file if it exists - if (classLoader == null && componentPackageFile != null) { - classLoader = getClassLoaderFromPackage(functionConfig.getClassName(), - componentPackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); - shouldCloseClassLoader = true; + if (functionPackage == null && componentPackageFile != null) { + functionPackage = + new FunctionFilePackage(componentPackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), FunctionDefinition.class); + shouldCloseFunctionPackage = true; } - if (classLoader == null) { + if (functionPackage == null) { throw new IllegalArgumentException("Function package is not provided"); } FunctionConfigUtils.ExtractedFunctionDetails functionDetails = FunctionConfigUtils.validateJavaFunction( - functionConfig, classLoader); + functionConfig, functionPackage); return FunctionConfigUtils.convert(functionConfig, functionDetails); } else { - classLoader = FunctionConfigUtils.validate(functionConfig, componentPackageFile); - shouldCloseClassLoader = true; - return FunctionConfigUtils.convert(functionConfig, classLoader); + FunctionConfigUtils.validateNonJavaFunction(functionConfig); + return FunctionConfigUtils.convert(functionConfig); } } finally { - if (shouldCloseClassLoader) { - ClassLoaderUtils.closeClassLoader(classLoader); + if (shouldCloseFunctionPackage && functionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) functionPackage).close(); + } catch (Exception e) { + log.error("Failed to close function file", e); + } } } } - - private File downloadPackageFile(String packageName) throws IOException, PulsarAdminException { - return downloadPackageFile(worker(), packageName); - } - - static File downloadPackageFile(PulsarWorkerService worker, String packageName) - throws IOException, PulsarAdminException { - Path tempDirectory; - if (worker.getWorkerConfig().getDownloadDirectory() != null) { - tempDirectory = Paths.get(worker.getWorkerConfig().getDownloadDirectory()); - } else { - // use the Nar extraction directory as a temporary directory for downloaded files - tempDirectory = Paths.get(worker.getWorkerConfig().getNarExtractionDirectory()); - } - Files.createDirectories(tempDirectory); - File file = Files.createTempFile(tempDirectory, "function", ".tmp").toFile(); - worker.getBrokerAdmin().packages().download(packageName, file.toString()); - return file; - } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java index 1b7ba1522bb2f..8cd020b6b333b 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java @@ -47,19 +47,20 @@ import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.SinkStatus; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.ComponentTypeUtils; -import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; import org.apache.pulsar.functions.utils.SinkConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.service.api.Sinks; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -135,29 +136,14 @@ public void registerSink(final String tenant, String.format("%s %s already exists", ComponentTypeUtils.toString(componentType), sinkName)); } - Function.FunctionDetails functionDetails = null; - boolean isPkgUrlProvided = isNotBlank(sinkPkgUrl); + Function.FunctionDetails functionDetails; File componentPackageFile = null; try { // validate parameters try { - if (isPkgUrlProvided) { - if (Utils.hasPackageTypePrefix(sinkPkgUrl)) { - componentPackageFile = downloadPackageFile(sinkPkgUrl); - } else { - if (!Utils.isFunctionPackageUrlSupported(sinkPkgUrl)) { - throw new IllegalArgumentException( - "Function Package url is not valid. supported url (http/https/file)"); - } - try { - componentPackageFile = FunctionCommon.extractFileFromPkgURL(sinkPkgUrl); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Encountered error \"%s\" when getting %s package from %s", - e.getMessage(), ComponentTypeUtils.toString(componentType), sinkPkgUrl)); - } - } + if (isNotBlank(sinkPkgUrl)) { + componentPackageFile = getPackageFile(sinkPkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, sinkConfig, componentPackageFile); } else { @@ -235,6 +221,7 @@ public void registerSink(final String tenant, } functionMetaDataBuilder.setPackageLocation(packageLocationMetaDataBuilder); + updateRequest(null, functionMetaDataBuilder.build()); } finally { if (componentPackageFile != null && componentPackageFile.exists()) { @@ -305,72 +292,30 @@ public void updateSink(final String tenant, throw new RestException(Response.Status.BAD_REQUEST, e.getMessage()); } - if (existingSinkConfig.equals(mergedConfig) && isBlank(sinkPkgUrl) && uploadedInputStream == null) { + if (existingSinkConfig.equals(mergedConfig) && isBlank(sinkPkgUrl) && uploadedInputStream == null + && (updateOptions == null || !updateOptions.isUpdateAuthData())) { log.error("{}/{}/{} Update contains no changes", tenant, namespace, sinkName); throw new RestException(Response.Status.BAD_REQUEST, "Update contains no change"); } - Function.FunctionDetails functionDetails = null; + Function.FunctionDetails functionDetails; File componentPackageFile = null; try { // validate parameters try { - if (isNotBlank(sinkPkgUrl)) { - if (Utils.hasPackageTypePrefix(sinkPkgUrl)) { - componentPackageFile = downloadPackageFile(sinkPkgUrl); - } else { - try { - componentPackageFile = FunctionCommon.extractFileFromPkgURL(sinkPkgUrl); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Encountered error \"%s\" when getting %s package from %s", - e.getMessage(), ComponentTypeUtils.toString(componentType), sinkPkgUrl)); - } - } - functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, - mergedConfig, componentPackageFile); - - } else if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.FILE) - || existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.HTTP)) { - try { - componentPackageFile = FunctionCommon - .extractFileFromPkgURL(existingComponent.getPackageLocation().getPackagePath()); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Encountered error \"%s\" when getting %s package from %s", - e.getMessage(), ComponentTypeUtils.toString(componentType), sinkPkgUrl)); - } - functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, - mergedConfig, componentPackageFile); - } else if (uploadedInputStream != null) { - - componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); - functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, - mergedConfig, componentPackageFile); - - } else if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.BUILTIN)) { - functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, - mergedConfig, componentPackageFile); - if (!isFunctionCodeBuiltin(functionDetails) - && (componentPackageFile == null || fileDetail == null)) { + componentPackageFile = getPackageFile( + sinkPkgUrl, + existingComponent.getPackageLocation().getPackagePath(), + uploadedInputStream); + functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, + mergedConfig, componentPackageFile); + if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.BUILTIN) + && !isFunctionCodeBuiltin(functionDetails) + && (componentPackageFile == null || fileDetail == null)) { throw new IllegalArgumentException( ComponentTypeUtils.toString(componentType) + " Package is not provided"); } - } else { - componentPackageFile = FunctionCommon.createPkgTempFile(); - componentPackageFile.deleteOnExit(); - if (worker().getWorkerConfig().isFunctionsWorkerEnablePackageManagement()) { - worker().getBrokerAdmin().packages().download( - existingComponent.getPackageLocation().getPackagePath(), - componentPackageFile.getAbsolutePath()); - } else { - WorkerUtils.downloadFromBookkeeper(worker().getDlogNamespace(), componentPackageFile, - existingComponent.getPackageLocation().getPackagePath()); - } - functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, - mergedConfig, componentPackageFile); - } } catch (Exception e) { log.error("Invalid update {} request @ /{}/{}/{}", ComponentTypeUtils.toString(componentType), tenant, namespace, sinkName, e); @@ -713,7 +658,8 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant final String namespace, final String sinkName, final SinkConfig sinkConfig, - final File sinkPackageFile) throws IOException { + final File sinkPackageFile) + throws IOException, PulsarAdminException { // The rest end points take precedence over whatever is there in sinkConfig sinkConfig.setTenant(tenant); @@ -721,7 +667,7 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant sinkConfig.setName(sinkName); org.apache.pulsar.common.functions.Utils.inferMissingArguments(sinkConfig); - ClassLoader classLoader = null; + ValidatableFunctionPackage connectorFunctionPackage = null; // check if sink is builtin and extract classloader if (!StringUtils.isEmpty(sinkConfig.getArchive())) { String archive = sinkConfig.getArchive(); @@ -733,35 +679,39 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (connector == null) { throw new IllegalArgumentException("Built-in sink is not available"); } - classLoader = connector.getClassLoader(); + connectorFunctionPackage = connector.getConnectorFunctionPackage(); } } - boolean shouldCloseClassLoader = false; + boolean shouldCloseFunctionPackage = false; try { // if sink is not builtin, attempt to extract classloader from package file if it exists - if (classLoader == null && sinkPackageFile != null) { - classLoader = getClassLoaderFromPackage(sinkConfig.getClassName(), - sinkPackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); - shouldCloseClassLoader = true; + WorkerConfig workerConfig = worker().getWorkerConfig(); + if (connectorFunctionPackage == null && sinkPackageFile != null) { + connectorFunctionPackage = + new FunctionFilePackage(sinkPackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), ConnectorDefinition.class); + shouldCloseFunctionPackage = true; } - if (classLoader == null) { + if (connectorFunctionPackage == null) { throw new IllegalArgumentException("Sink package is not provided"); } - SinkConfigUtils.ExtractedSinkDetails sinkDetails = SinkConfigUtils.validateAndExtractDetails( - sinkConfig, classLoader, worker().getWorkerConfig().getValidateConnectorConfig()); + SinkConfigUtils.ExtractedSinkDetails sinkDetails = + SinkConfigUtils.validateAndExtractDetails(sinkConfig, connectorFunctionPackage, + workerConfig.getValidateConnectorConfig()); + return SinkConfigUtils.convert(sinkConfig, sinkDetails); } finally { - if (shouldCloseClassLoader) { - ClassLoaderUtils.closeClassLoader(classLoader); + if (shouldCloseFunctionPackage && connectorFunctionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) connectorFunctionPackage).close(); + } catch (Exception e) { + log.error("Failed to connector function file", e); + } } } } - - private File downloadPackageFile(String packageName) throws IOException, PulsarAdminException { - return FunctionsImpl.downloadPackageFile(worker(), packageName); - } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java index 44c4115575a59..59bc5888a3403 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java @@ -25,7 +25,6 @@ import static org.apache.pulsar.functions.worker.rest.RestUtils.throwUnavailableException; import com.google.protobuf.ByteString; import java.io.File; -import java.io.IOException; import java.io.InputStream; import java.net.URI; import java.util.ArrayList; @@ -47,19 +46,20 @@ import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.SourceStatus; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.ComponentTypeUtils; -import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; import org.apache.pulsar.functions.utils.SourceConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.service.api.Sources; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -143,21 +143,7 @@ public void registerSource(final String tenant, // validate parameters try { if (isPkgUrlProvided) { - if (Utils.hasPackageTypePrefix(sourcePkgUrl)) { - componentPackageFile = downloadPackageFile(sourcePkgUrl); - } else { - if (!Utils.isFunctionPackageUrlSupported(sourcePkgUrl)) { - throw new IllegalArgumentException( - "Function Package url is not valid. supported url (http/https/file)"); - } - try { - componentPackageFile = FunctionCommon.extractFileFromPkgURL(sourcePkgUrl); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Encountered error \"%s\" when getting %s package from %s", - e.getMessage(), ComponentTypeUtils.toString(componentType), sourcePkgUrl)); - } - } + componentPackageFile = getPackageFile(sourcePkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, sourceConfig, componentPackageFile); } else { @@ -305,71 +291,29 @@ public void updateSource(final String tenant, throw new RestException(Response.Status.BAD_REQUEST, e.getMessage()); } - if (existingSourceConfig.equals(mergedConfig) && isBlank(sourcePkgUrl) && uploadedInputStream == null) { + if (existingSourceConfig.equals(mergedConfig) && isBlank(sourcePkgUrl) && uploadedInputStream == null + && (updateOptions == null || !updateOptions.isUpdateAuthData())) { log.error("{}/{}/{} Update contains no changes", tenant, namespace, sourceName); throw new RestException(Response.Status.BAD_REQUEST, "Update contains no change"); } - Function.FunctionDetails functionDetails = null; + Function.FunctionDetails functionDetails; File componentPackageFile = null; try { // validate parameters try { - if (isNotBlank(sourcePkgUrl)) { - if (Utils.hasPackageTypePrefix(sourcePkgUrl)) { - componentPackageFile = downloadPackageFile(sourcePkgUrl); - } else { - try { - componentPackageFile = FunctionCommon.extractFileFromPkgURL(sourcePkgUrl); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Encountered error \"%s\" when getting %s package from %s", - e.getMessage(), ComponentTypeUtils.toString(componentType), sourcePkgUrl)); - } - } - functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, - mergedConfig, componentPackageFile); - - } else if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.FILE) - || existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.HTTP)) { - try { - componentPackageFile = FunctionCommon - .extractFileFromPkgURL(existingComponent.getPackageLocation().getPackagePath()); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format("Encountered error \"%s\" when getting %s package from %s", - e.getMessage(), ComponentTypeUtils.toString(componentType), sourcePkgUrl)); - } - functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, - mergedConfig, componentPackageFile); - } else if (uploadedInputStream != null) { - - componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); - functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, - mergedConfig, componentPackageFile); - - } else if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.BUILTIN)) { - functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, - mergedConfig, componentPackageFile); - if (!isFunctionCodeBuiltin(functionDetails) - && (componentPackageFile == null || fileDetail == null)) { - throw new IllegalArgumentException( - ComponentTypeUtils.toString(componentType) + " Package is not provided"); - } - } else { - componentPackageFile = FunctionCommon.createPkgTempFile(); - componentPackageFile.deleteOnExit(); - if (worker().getWorkerConfig().isFunctionsWorkerEnablePackageManagement()) { - worker().getBrokerAdmin().packages().download( - existingComponent.getPackageLocation().getPackagePath(), - componentPackageFile.getAbsolutePath()); - } else { - WorkerUtils.downloadFromBookkeeper(worker().getDlogNamespace(), componentPackageFile, - existingComponent.getPackageLocation().getPackagePath()); - } - functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, - mergedConfig, componentPackageFile); + componentPackageFile = getPackageFile( + sourcePkgUrl, + existingComponent.getPackageLocation().getPackagePath(), + uploadedInputStream); + functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, + mergedConfig, componentPackageFile); + if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.BUILTIN) + && !isFunctionCodeBuiltin(functionDetails) + && (componentPackageFile == null || fileDetail == null)) { + throw new IllegalArgumentException(ComponentTypeUtils.toString(componentType) + + " Package is not provided"); } } catch (Exception e) { log.error("Invalid update {} request @ /{}/{}/{}", ComponentTypeUtils.toString(componentType), tenant, @@ -721,7 +665,7 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant sourceConfig.setName(sourceName); org.apache.pulsar.common.functions.Utils.inferMissingArguments(sourceConfig); - ClassLoader classLoader = null; + ValidatableFunctionPackage connectorFunctionPackage = null; // check if source is builtin and extract classloader if (!StringUtils.isEmpty(sourceConfig.getArchive())) { String archive = sourceConfig.getArchive(); @@ -733,35 +677,38 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (connector == null) { throw new IllegalArgumentException("Built-in source is not available"); } - classLoader = connector.getClassLoader(); + connectorFunctionPackage = connector.getConnectorFunctionPackage(); } } - boolean shouldCloseClassLoader = false; + boolean shouldCloseFunctionPackage = false; try { // if source is not builtin, attempt to extract classloader from package file if it exists - if (classLoader == null && sourcePackageFile != null) { - classLoader = getClassLoaderFromPackage(sourceConfig.getClassName(), - sourcePackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); - shouldCloseClassLoader = true; + WorkerConfig workerConfig = worker().getWorkerConfig(); + if (connectorFunctionPackage == null && sourcePackageFile != null) { + connectorFunctionPackage = + new FunctionFilePackage(sourcePackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), ConnectorDefinition.class); + shouldCloseFunctionPackage = true; } - if (classLoader == null) { + if (connectorFunctionPackage == null) { throw new IllegalArgumentException("Source package is not provided"); } SourceConfigUtils.ExtractedSourceDetails sourceDetails = SourceConfigUtils.validateAndExtractDetails( - sourceConfig, classLoader, worker().getWorkerConfig().getValidateConnectorConfig()); + sourceConfig, connectorFunctionPackage, + workerConfig.getValidateConnectorConfig()); return SourceConfigUtils.convert(sourceConfig, sourceDetails); } finally { - if (shouldCloseClassLoader) { - ClassLoaderUtils.closeClassLoader(classLoader); + if (shouldCloseFunctionPackage && connectorFunctionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) connectorFunctionPackage).close(); + } catch (Exception e) { + log.error("Failed to connector function file", e); + } } } } - - private File downloadPackageFile(String packageName) throws IOException, PulsarAdminException { - return FunctionsImpl.downloadPackageFile(worker(), packageName); - } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java index a64d451628dda..459bf243c1ee2 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionAssignmentTailerTest.java @@ -145,6 +145,9 @@ public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { // test new assignment add functions FunctionRuntimeManager functionRuntimeManager = mock(FunctionRuntimeManager.class); + // trigger an error to be thrown + doThrow(new RuntimeException("test")).when(functionRuntimeManager).processAssignmentMessage(any()); + FunctionAssignmentTailer functionAssignmentTailer = spy(new FunctionAssignmentTailer(functionRuntimeManager, readerBuilder, workerConfig, errorNotifier)); @@ -157,9 +160,6 @@ public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { verify(errorNotifier, times(0)).triggerError(any()); - // trigger an error to be thrown - doThrow(new RuntimeException("test")).when(functionRuntimeManager).processAssignmentMessage(any()); - messageList.add(message2); try { diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java index 7443e1e62d8de..d8902c0030426 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java @@ -381,6 +381,6 @@ public static FunctionConfig createDefaultFunctionConfig() { public static Function.FunctionDetails createDefaultFunctionDetails() { FunctionConfig functionConfig = createDefaultFunctionConfig(); - return FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + return FunctionConfigUtils.convert(functionConfig); } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java index b50e734e0d594..198baa6a1992a 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java @@ -18,1125 +18,82 @@ */ package org.apache.pulsar.functions.worker.rest.api.v2; - -import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import static org.apache.pulsar.functions.utils.FunctionCommon.mergeJson; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import com.google.common.collect.Lists; import com.google.gson.Gson; -import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; -import org.apache.distributedlog.api.namespace.Namespace; -import org.apache.logging.log4j.Level; -import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; import org.apache.pulsar.common.functions.FunctionConfig; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.util.RestException; -import org.apache.pulsar.functions.api.Context; -import org.apache.pulsar.functions.api.Function; -import org.apache.pulsar.functions.instance.InstanceUtils; -import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.proto.Function.FunctionMetaData; -import org.apache.pulsar.functions.proto.Function.PackageLocationMetaData; -import org.apache.pulsar.functions.proto.Function.ProcessingGuarantees; -import org.apache.pulsar.functions.proto.Function.SinkSpec; -import org.apache.pulsar.functions.proto.Function.SourceSpec; -import org.apache.pulsar.functions.proto.Function.SubscriptionType; -import org.apache.pulsar.functions.runtime.RuntimeFactory; -import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.utils.FunctionConfigUtils; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; -import org.apache.pulsar.functions.worker.WorkerConfig; -import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; +import org.apache.pulsar.functions.worker.rest.api.v3.AbstractFunctionApiResourceTest; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; -import org.mockito.MockedStatic; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -/** - * Unit test of {@link FunctionsApiV2Resource}. - */ -public class FunctionApiV2ResourceTest { - - private static final class TestFunction implements Function { - - @Override - public String process(String input, Context context) { - return input; - } - } - - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; - private static final String function = "test-function"; - private static final String outputTopic = "test-output-topic"; - private static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; - private static final String className = TestFunction.class.getName(); - private SubscriptionType subscriptionType = SubscriptionType.FAILOVER; - private static final Map topicsToSerDeClassName = new HashMap<>(); - static { - topicsToSerDeClassName.put("persistent://public/default/test_src", TopicSchema.DEFAULT_SERDE); - } - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; +public class FunctionApiV2ResourceTest extends AbstractFunctionApiResourceTest { private FunctionsImplV2 resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetadata; - private LeaderService mockedLeaderService; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - private static Map mockStaticContexts = new HashMap<>(); - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedFunctionMetadata = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetadata); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); - - FunctionsImpl functions = spy(new FunctionsImpl(() -> mockedWorkerService)); - - this.resource = spy(new FunctionsImplV2(functions)); - - } - - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = - mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - private void mockWorkerUtils() { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - }); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); - } - - private void mockInstanceUtils() { - mockStatic(InstanceUtils.class, ctx -> { - ctx.when(() -> InstanceUtils.calculateSubjectType(any())) - .thenReturn(FunctionDetails.ComponentType.FUNCTION); - }); - } - - // - // Register Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testRegisterFunctionMissingTenant() { - try { - testRegisterFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testRegisterFunctionMissingNamespace() { - try { - testRegisterFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testRegisterFunctionMissingFunctionName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package is not provided") - public void testRegisterFunctionMissingPackage() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "No input topic\\(s\\) specified for the function") - public void testRegisterFunctionMissingInputTopics() throws Exception { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function Package is not provided") - public void testRegisterFunctionMissingPackageDetails() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package does not have" - + " the correct format. Pulsar cannot determine if the package is a NAR package or JAR package. Function " - + "classname is not provided and attempts to load it as a NAR package produced the following error.*") - public void testRegisterFunctionMissingClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class UnknownClass must be in class path") - public void testRegisterFunctionWrongClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - "UnknownClass", - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function parallelism must be a positive number") - public void testRegisterFunctionWrongParallelism() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - -2, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, - expectedExceptionsMessageRegExp = "Output topic persistent://public/default/test_src is also being used as an input topic \\(topics must be one or the other\\)") - public void testRegisterFunctionSameInputOutput() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - topicsToSerDeClassName.keySet().iterator().next(), - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Output topic " + function + - "-output-topic/test:" + " is invalid") - public void testRegisterFunctionWrongOutputTopic() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - function + "-output-topic/test:", - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Encountered error .*. when getting Function package from .*") - public void testRegisterFunctionHttpUrl() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "http://localhost:1234/test"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testRegisterFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String functionPkgUrl) { - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenCallRealMethod(); - }); - - try { - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - details, - functionPkgUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - private void registerDefaultFunction() { - FunctionConfig functionConfig = createDefaultFunctionConfig(); - try { - resource.registerFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function already exists") - public void testRegisterExistedFunction() { - try { - Configurator.setRootLevel(Level.DEBUG); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testRegisterFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new IOException("upload failure")); - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testRegisterFunctionSuccess() throws Exception { - try { - try (MockedStatic mocked = Mockito.mockStatic(WorkerUtils.class)) { - mocked.when(() -> WorkerUtils.uploadToBookKeeper( - any(Namespace.class), - any(InputStream.class), - anyString())).thenAnswer((i) -> null); - mocked.when(() -> WorkerUtils.dumpToTmpFile(any())).thenAnswer(i -> - { - try { - File tmpFile = FunctionCommon.createPkgTempFile(); - tmpFile.deleteOnExit(); - Files.copy((InputStream) i.getArguments()[0], tmpFile.toPath(), REPLACE_EXISTING); - return tmpFile; - } catch (IOException e) { - throw new RuntimeException("Cannot create a temporary file", e); - } - - } - ); - WorkerUtils.uploadToBookKeeper(null, null, null); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace does not exist") - public void testRegisterFunctionNonExistingNamespace() { - try { - this.namespaceList.clear(); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant does not exist") - public void testRegisterFunctionNonexistantTenant() throws Exception { - try { - when(mockedTenants.getTenantInfo(any())).thenThrow(PulsarAdminException.NotFoundException.class); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testRegisterFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registration interrupted") - public void testRegisterFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - doThrow(new IllegalStateException("Function registration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // Update Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testUpdateFunctionMissingTenant() throws Exception { - try { - testUpdateFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Tenant is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testUpdateFunctionMissingNamespace() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Namespace is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testUpdateFunctionMissingFunctionName() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Function name is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingPackage() throws Exception { - try { - mockWorkerUtils(); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingInputTopic() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingClassName() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test - public void testUpdateFunctionChangedParallelism() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism + 1, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + @Override + protected void doSetup() { + super.doSetup(); + this.resource = spy(new FunctionsImplV2(() -> mockedWorkerService)); } - @Test - public void testUpdateFunctionChangedInputs() throws Exception { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( + protected void registerFunction(String tenant, String namespace, String function, InputStream inputStream, + FormDataContentDisposition details, String functionPkgUrl, + FunctionConfig functionConfig) throws IOException { + resource.registerFunction( tenant, namespace, function, - null, - topicsToSerDeClassName, - mockedFormData, - "DifferentOutput", - outputSerdeClassName, - null, - parallelism, + inputStream, + details, + functionPkgUrl, + JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig)), null); } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Input Topics cannot be altered") - public void testUpdateFunctionChangedOutput() throws Exception { - try { - mockWorkerUtils(); - - Map someOtherInput = new HashMap<>(); - someOtherInput.put("DifferentTopic", TopicSchema.DEFAULT_SERDE); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - someOtherInput, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Input Topics cannot be altered"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testUpdateFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String expectedError) throws Exception { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - if (expectedError != null) { - doThrow(new IllegalArgumentException(expectedError)) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - } - - try { - resource.updateFunction( - tenant, - namespace, - function, - inputStream, - details, - null, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - private void updateDefaultFunction() { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - - try { - resource.updateFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testUpdateNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testUpdateFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new IOException("upload failure")); - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testUpdateFunctionSuccess() throws Exception { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } - - @Test - public void testUpdateFunctionWithUrl() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - try { - resource.updateFunction( - tenant, - namespace, - function, - null, - null, - filePackageUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testUpdateFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registeration interrupted") - public void testUpdateFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function registeration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // deregister function - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testDeregisterFunctionMissingTenant() { - try { - - testDeregisterFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + protected void updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + FunctionConfig functionConfig, + AuthenticationParameters authParams, + UpdateOptionsImpl updateOptions) throws IOException { + resource.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, + JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig)), authParams); } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testDeregisterFunctionMissingNamespace() { - try { - testDeregisterFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; + protected File downloadFunction(final String path, final AuthenticationParameters authParams) + throws IOException { + Response response = resource.downloadFunction(path, authParams); + StreamingOutput streamingOutput = readEntity(response, StreamingOutput.class); + File pkgFile = File.createTempFile("testpkg", "nar"); + try (OutputStream output = new FileOutputStream(pkgFile)) { + streamingOutput.write(output); } + return pkgFile; } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testDeregisterFunctionMissingFunctionName() { - try { - testDeregisterFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + private T readEntity(Response response, Class clazz) { + return clazz.cast(response.getEntity()); } - private void testDeregisterFunctionMissingArguments( + protected void testDeregisterFunctionMissingArguments( String tenant, String namespace, String function @@ -1145,112 +102,18 @@ private void testDeregisterFunctionMissingArguments( tenant, namespace, function, - AuthenticationParameters.builder().build()); + null); } - private void deregisterDefaultFunction() { + protected void deregisterDefaultFunction() { resource.deregisterFunction( tenant, namespace, function, - AuthenticationParameters.builder().build()); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testDeregisterNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); - throw re; - } - } - - @Test - public void testDeregisterFunctionSuccess() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - deregisterDefaultFunction(); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to deregister") - public void testDeregisterFunctionFailure() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to deregister")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function deregisteration interrupted") - public void testDeregisterFunctionInterrupted() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function deregisteration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // Get Function Info - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testGetFunctionMissingTenant() throws IOException { - try { - testGetFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testGetFunctionMissingNamespace() throws IOException { - try { - testGetFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testGetFunctionMissingFunctionName() throws IOException { - try { - testGetFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + null); } - private void testGetFunctionMissingArguments( + protected void testGetFunctionMissingArguments( String tenant, String namespace, String function @@ -1258,20 +121,36 @@ private void testGetFunctionMissingArguments( resource.getFunctionInfo( tenant, namespace, - function, - AuthenticationParameters.builder().build() + function, null + ); + } + + protected void testListFunctionsMissingArguments( + String tenant, + String namespace + ) { + resource.listFunctions( + tenant, + namespace, null ); } - private FunctionDetails getDefaultFunctionInfo() throws IOException { + protected List listDefaultFunctions() { + return new Gson().fromJson(readEntity(resource.listFunctions( + tenant, + namespace, null + ), String.class), List.class); + } + + private Function.FunctionDetails getDefaultFunctionInfo() throws IOException { String json = (String) resource.getFunctionInfo( tenant, namespace, function, AuthenticationParameters.builder().build() ).getEntity(); - FunctionDetails.Builder functionDetailsBuilder = FunctionDetails.newBuilder(); + Function.FunctionDetails.Builder functionDetailsBuilder = Function.FunctionDetails.newBuilder(); mergeJson(json, functionDetailsBuilder); return functionDetailsBuilder.build(); } @@ -1292,218 +171,31 @@ public void testGetFunctionSuccess() throws IOException { mockInstanceUtils(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - SinkSpec sinkSpec = SinkSpec.newBuilder() + Function.SinkSpec sinkSpec = Function.SinkSpec.newBuilder() .setTopic(outputTopic) .setSerDeClassName(outputSerdeClassName).build(); - FunctionDetails functionDetails = FunctionDetails.newBuilder() + Function.FunctionDetails functionDetails = Function.FunctionDetails.newBuilder() .setClassName(className) .setSink(sinkSpec) .setName(function) .setNamespace(namespace) - .setProcessingGuarantees(ProcessingGuarantees.ATMOST_ONCE) + .setProcessingGuarantees(Function.ProcessingGuarantees.ATMOST_ONCE) .setAutoAck(true) .setTenant(tenant) .setParallelism(parallelism) - .setSource(SourceSpec.newBuilder().setSubscriptionType(subscriptionType) + .setSource(Function.SourceSpec.newBuilder().setSubscriptionType(subscriptionType) .putAllTopicsToSerDeClassName(topicsToSerDeClassName)).build(); - FunctionMetaData metaData = FunctionMetaData.newBuilder() + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() .setCreateTime(System.currentTimeMillis()) .setFunctionDetails(functionDetails) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) .setVersion(1234) .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - FunctionDetails actual = getDefaultFunctionInfo(); + Function.FunctionDetails actual = getDefaultFunctionInfo(); assertEquals( functionDetails, actual); } - - // - // List Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testListFunctionsMissingTenant() { - try { - testListFunctionsMissingArguments( - null, - namespace - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testListFunctionsMissingNamespace() { - try { - testListFunctionsMissingArguments( - tenant, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testListFunctionsMissingArguments( - String tenant, - String namespace - ) { - resource.listFunctions( - tenant, - namespace, - AuthenticationParameters.builder().build() - ); - - } - - private List listDefaultFunctions() { - return new Gson().fromJson((String) resource.listFunctions( - tenant, - namespace, - AuthenticationParameters.builder().build() - ).getEntity(), List.class); - } - - @Test - public void testListFunctionsSuccess() { - mockInstanceUtils(); - final List functions = Lists.newArrayList("test-1", "test-2"); - final List metaDataList = new LinkedList<>(); - FunctionMetaData functionMetaData1 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-1").build() - ).build(); - FunctionMetaData functionMetaData2 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-2").build() - ).build(); - metaDataList.add(functionMetaData1); - metaDataList.add(functionMetaData2); - when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(metaDataList); - - List functionList = listDefaultFunctions(); - assertEquals(functions, functionList); - } - - @Test - public void testDownloadFunctionHttpUrl() throws Exception { - String jarHttpUrl = - "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; - String testDir = FunctionApiV2ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - FunctionsImplV2 function = new FunctionsImplV2(() -> mockedWorkerService); - StreamingOutput streamOutput = (StreamingOutput) function.downloadFunction(jarHttpUrl, - AuthenticationParameters.builder().build()).getEntity(); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - if (pkgFile.exists()) { - pkgFile.delete(); - } - } - - @Test - public void testDownloadFunctionFile() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String testDir = FunctionApiV2ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - FunctionsImplV2 function = new FunctionsImplV2(() -> mockedWorkerService); - StreamingOutput streamOutput = (StreamingOutput) function.downloadFunction("file:///" + fileLocation, - AuthenticationParameters.builder().build()).getEntity(); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - if (pkgFile.exists()) { - pkgFile.delete(); - } - } - - @Test - public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - try { - resource.registerFunction(tenant, namespace, function, null, null, filePackageUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - @Test - public void testRegisterFunctionWithConflictingFields() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - String actualTenant = "DIFFERENT_TENANT"; - String actualNamespace = "DIFFERENT_NAMESPACE"; - String actualName = "DIFFERENT_NAME"; - this.namespaceList.add(actualTenant + "/" + actualNamespace); - - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - try { - resource.registerFunction(actualTenant, actualNamespace, actualName, null, null, filePackageUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - - public static FunctionConfig createDefaultFunctionConfig() { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - return functionConfig; - } - - public static FunctionDetails createDefaultFunctionDetails() { - FunctionConfig functionConfig = createDefaultFunctionConfig(); - return FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); - } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java new file mode 100644 index 0000000000000..d6aa13dc2bfb9 --- /dev/null +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java @@ -0,0 +1,1367 @@ +/** + * 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.pulsar.functions.worker.rest.api.v3; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import com.google.common.collect.Lists; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import javax.ws.rs.core.Response; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.core.config.Configurator; +import org.apache.pulsar.broker.authentication.AuthenticationParameters; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.RestException; +import org.apache.pulsar.functions.api.Context; +import org.apache.pulsar.functions.api.Function; +import org.apache.pulsar.functions.proto.Function.FunctionDetails; +import org.apache.pulsar.functions.proto.Function.FunctionMetaData; +import org.apache.pulsar.functions.proto.Function.SubscriptionType; +import org.apache.pulsar.functions.source.TopicSchema; +import org.apache.pulsar.functions.utils.FunctionConfigUtils; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + +public abstract class AbstractFunctionApiResourceTest extends AbstractFunctionsResourceTest { + + @Test + public void testListFunctionsSuccess() { + mockInstanceUtils(); + final List functions = Lists.newArrayList("test-1", "test-2"); + final List metaDataList = new LinkedList<>(); + FunctionMetaData functionMetaData1 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder().setName("test-1").build() + ).build(); + FunctionMetaData functionMetaData2 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder().setName("test-2").build() + ).build(); + metaDataList.add(functionMetaData1); + metaDataList.add(functionMetaData2); + when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(metaDataList); + + List functionList = listDefaultFunctions(); + assertEquals(functions, functionList); + } + + @Test + public void testOnlyGetSources() { + List functions = Lists.newArrayList("test-2"); + List functionMetaDataList = new LinkedList<>(); + FunctionMetaData f1 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder() + .setName("test-1") + .setComponentType(FunctionDetails.ComponentType.SOURCE) + .build()).build(); + functionMetaDataList.add(f1); + FunctionMetaData f2 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder() + .setName("test-2") + .setComponentType(FunctionDetails.ComponentType.FUNCTION) + .build()).build(); + functionMetaDataList.add(f2); + FunctionMetaData f3 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder() + .setName("test-3") + .setComponentType(FunctionDetails.ComponentType.SINK) + .build()).build(); + functionMetaDataList.add(f3); + when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(functionMetaDataList); + + List functionList = listDefaultFunctions(); + assertEquals(functions, functionList); + } + + private static final class TestFunction implements Function { + + @Override + public String process(String input, Context context) { + return input; + } + } + + private static final class WrongFunction implements Consumer { + @Override + public void accept(String s) { + + } + } + + protected static final String function = "test-function"; + protected static final String outputTopic = "test-output-topic"; + protected static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; + protected static final String className = TestFunction.class.getName(); + protected SubscriptionType subscriptionType = SubscriptionType.FAILOVER; + protected FunctionMetaData mockedFunctionMetadata; + + + @Override + protected void doSetup() { + this.mockedFunctionMetadata = + FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetadata); + } + + @Override + protected FunctionDetails.ComponentType getComponentType() { + return FunctionDetails.ComponentType.FUNCTION; + } + + + abstract protected void registerFunction(String tenant, String namespace, String function, InputStream inputStream, + FormDataContentDisposition details, String functionPkgUrl, FunctionConfig functionConfig) + throws IOException; + abstract protected void updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + FunctionConfig functionConfig, + AuthenticationParameters authParams, + UpdateOptionsImpl updateOptions) throws IOException; + + abstract protected File downloadFunction(final String path, final AuthenticationParameters authParams) + throws IOException; + + abstract protected void testDeregisterFunctionMissingArguments( + String tenant, + String namespace, + String function + ); + + abstract protected void deregisterDefaultFunction(); + + abstract protected void testGetFunctionMissingArguments( + String tenant, + String namespace, + String function + ) throws IOException; + + abstract protected void testListFunctionsMissingArguments( + String tenant, + String namespace + ); + + abstract protected List listDefaultFunctions(); + + // + // Register Functions + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testRegisterFunctionMissingTenant() throws IOException { + try { + testRegisterFunctionMissingArguments( + null, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testRegisterFunctionMissingNamespace() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + null, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testRegisterFunctionMissingFunctionName() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + null, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package is not " + + "provided") + public void testRegisterFunctionMissingPackage() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "No input topic\\(s\\) " + + "specified for the function") + public void testRegisterFunctionMissingInputTopics() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + null, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function Package is not " + + "provided") + public void testRegisterFunctionMissingPackageDetails() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + null, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, + expectedExceptionsMessageRegExp = "Function class name is not provided.") + public void testRegisterFunctionMissingClassName() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class UnknownClass " + + "must be in class path") + public void testRegisterFunctionWrongClassName() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + "UnknownClass", + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function parallelism must be a" + + " positive number") + public void testRegisterFunctionWrongParallelism() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + -2, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, + expectedExceptionsMessageRegExp = "Output topic persistent://public/default/test_src is also being used " + + "as an input topic \\(topics must be one or the other\\)") + public void testRegisterFunctionSameInputOutput() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + topicsToSerDeClassName.keySet().iterator().next(), + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Output topic " + function + + "-output-topic/test:" + " is invalid") + public void testRegisterFunctionWrongOutputTopic() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + function + "-output-topic/test:", + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Encountered error .*. when " + + "getting Function package from .*") + public void testRegisterFunctionHttpUrl() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + null, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "http://localhost:1234/test"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class .*. does not " + + "implement the correct interface") + public void testRegisterFunctionImplementWrongInterface() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + WrongFunction.class.getName(), + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + private void testRegisterFunctionMissingArguments( + String tenant, + String namespace, + String function, + InputStream inputStream, + Map topicsToSerDeClassName, + FormDataContentDisposition details, + String outputTopic, + String outputSerdeClassName, + String className, + Integer parallelism, + String functionPkgUrl) throws IOException { + FunctionConfig functionConfig = new FunctionConfig(); + if (tenant != null) { + functionConfig.setTenant(tenant); + } + if (namespace != null) { + functionConfig.setNamespace(namespace); + } + if (function != null) { + functionConfig.setName(function); + } + if (topicsToSerDeClassName != null) { + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + } + if (outputTopic != null) { + functionConfig.setOutput(outputTopic); + } + if (outputSerdeClassName != null) { + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + } + if (className != null) { + functionConfig.setClassName(className); + } + if (parallelism != null) { + functionConfig.setParallelism(parallelism); + } + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + + registerFunction(tenant, namespace, function, inputStream, details, functionPkgUrl, functionConfig); + + } + + @Test(expectedExceptions = Exception.class, expectedExceptionsMessageRegExp = "Function config is not provided") + public void testUpdateMissingFunctionConfig() throws IOException { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateFunction( + tenant, + namespace, + function, + mockedInputStream, + mockedFormData, + null, + null, + null, null); + } + + + private void registerDefaultFunction() throws IOException { + registerDefaultFunctionWithPackageUrl(null); + } + + private void registerDefaultFunctionWithPackageUrl(String packageUrl) throws IOException { + FunctionConfig functionConfig = createDefaultFunctionConfig(); + registerFunction(tenant, namespace, function, mockedInputStream, mockedFormData, packageUrl, functionConfig); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function already" + + " exists") + public void testRegisterExistedFunction() throws IOException { + try { + Configurator.setRootLevel(Level.DEBUG); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") + public void testRegisterFunctionUploadFailure() throws IOException { + try { + mockWorkerUtils(ctx -> { + ctx.when(() -> { + WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class)); + } + ).thenThrow(new IOException("upload failure")); + }); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + @Test + public void testRegisterFunctionSuccess() throws IOException { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(timeOut = 20000) + public void testRegisterFunctionSuccessWithPackageName() throws IOException { + registerDefaultFunctionWithPackageUrl("function://public/default/test@v1"); + } + + @Test(timeOut = 20000) + public void testRegisterFunctionFailedWithWrongPackageName() throws PulsarAdminException, IOException { + try { + doThrow(new PulsarAdminException("package name is invalid")) + .when(mockedPackages).download(anyString(), anyString()); + registerDefaultFunctionWithPackageUrl("function://"); + } catch (RestException e) { + // expected exception + assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace does not exist") + public void testRegisterFunctionNonExistingNamespace() throws IOException { + try { + this.namespaceList.clear(); + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant does not exist") + public void testRegisterFunctionNonexistantTenant() throws Exception { + try { + when(mockedTenants.getTenantInfo(any())).thenThrow(PulsarAdminException.NotFoundException.class); + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") + public void testRegisterFunctionFailure() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + doThrow(new IllegalArgumentException("function failed to register")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registration " + + "interrupted") + public void testRegisterFunctionInterrupted() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + doThrow(new IllegalStateException("Function registration interrupted")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + // + // Update Functions + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testUpdateFunctionMissingTenant() throws Exception { + try { + testUpdateFunctionMissingArguments( + null, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Tenant is not provided"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testUpdateFunctionMissingNamespace() throws Exception { + try { + testUpdateFunctionMissingArguments( + tenant, + null, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Namespace is not provided"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testUpdateFunctionMissingFunctionName() throws Exception { + try { + testUpdateFunctionMissingArguments( + tenant, + namespace, + null, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Function name is not provided"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") + public void testUpdateFunctionMissingPackage() throws Exception { + try { + mockWorkerUtils(); + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Update contains no change"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") + public void testUpdateFunctionMissingInputTopic() throws Exception { + try { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + null, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Update contains no change"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") + public void testUpdateFunctionMissingClassName() throws Exception { + try { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism, + "Update contains no change"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testUpdateFunctionChangedParallelism() throws Exception { + try { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism + 1, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testUpdateFunctionChangedInputs() throws Exception { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + "DifferentOutput", + outputSerdeClassName, + null, + parallelism, + null); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Input Topics cannot be altered") + public void testUpdateFunctionChangedOutput() throws Exception { + try { + mockWorkerUtils(); + + Map someOtherInput = new HashMap<>(); + someOtherInput.put("DifferentTopic", TopicSchema.DEFAULT_SERDE); + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + someOtherInput, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism, + "Input Topics cannot be altered"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + private void testUpdateFunctionMissingArguments( + String tenant, + String namespace, + String function, + InputStream inputStream, + Map topicsToSerDeClassName, + FormDataContentDisposition details, + String outputTopic, + String outputSerdeClassName, + String className, + Integer parallelism, + String expectedError) throws Exception { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + FunctionConfig functionConfig = new FunctionConfig(); + if (tenant != null) { + functionConfig.setTenant(tenant); + } + if (namespace != null) { + functionConfig.setNamespace(namespace); + } + if (function != null) { + functionConfig.setName(function); + } + if (topicsToSerDeClassName != null) { + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + } + if (outputTopic != null) { + functionConfig.setOutput(outputTopic); + } + if (outputSerdeClassName != null) { + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + } + if (className != null) { + functionConfig.setClassName(className); + } + if (parallelism != null) { + functionConfig.setParallelism(parallelism); + } + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + + if (expectedError != null) { + doThrow(new IllegalArgumentException(expectedError)) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + } + + updateFunction( + tenant, + namespace, + function, + inputStream, + details, + null, + functionConfig, + null, null); + + } + + private void updateDefaultFunction() throws IOException { + updateDefaultFunctionWithPackageUrl(null); + } + + private void updateDefaultFunctionWithPackageUrl(String packageUrl) throws IOException { + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + + updateFunction( + tenant, + namespace, + function, + mockedInputStream, + mockedFormData, + packageUrl, + functionConfig, + null, null); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't" + + " exist") + public void testUpdateNotExistedFunction() throws IOException { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") + public void testUpdateFunctionUploadFailure() throws Exception { + try { + mockWorkerUtils(ctx -> { + ctx.when(() -> { + WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class)); + + }).thenThrow(new IOException("upload failure")); + }); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + @Test + public void testUpdateFunctionSuccess() throws Exception { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateDefaultFunction(); + } + + @Test + public void testUpdateFunctionWithUrl() throws IOException { + Configurator.setRootLevel(Level.DEBUG); + + String fileLocation = FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + String filePackageUrl = "file://" + fileLocation; + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateFunction( + tenant, + namespace, + function, + null, + null, + filePackageUrl, + functionConfig, + null, null); + + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") + public void testUpdateFunctionFailure() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalArgumentException("function failed to register")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registeration " + + "interrupted") + public void testUpdateFunctionInterrupted() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalStateException("Function registeration interrupted")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + + @Test(timeOut = 20000) + public void testUpdateFunctionSuccessWithPackageName() throws IOException { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + updateDefaultFunctionWithPackageUrl("function://public/default/test@v1"); + } + + @Test(timeOut = 20000) + public void testUpdateFunctionFailedWithWrongPackageName() throws PulsarAdminException, IOException { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + try { + doThrow(new PulsarAdminException("package name is invalid")) + .when(mockedPackages).download(anyString(), anyString()); + registerDefaultFunctionWithPackageUrl("function://"); + } catch (RestException e) { + // expected exception + assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + } + } + + // + // deregister function + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testDeregisterFunctionMissingTenant() { + try { + + testDeregisterFunctionMissingArguments( + null, + namespace, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testDeregisterFunctionMissingNamespace() { + try { + testDeregisterFunctionMissingArguments( + tenant, + null, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testDeregisterFunctionMissingFunctionName() { + try { + testDeregisterFunctionMissingArguments( + tenant, + namespace, + null + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't" + + " exist") + public void testDeregisterNotExistedFunction() { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + deregisterDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); + throw re; + } + } + + @Test + public void testDeregisterFunctionSuccess() { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + deregisterDefaultFunction(); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to deregister") + public void testDeregisterFunctionFailure() throws Exception { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalArgumentException("function failed to deregister")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + deregisterDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function deregisteration " + + "interrupted") + public void testDeregisterFunctionInterrupted() throws Exception { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalStateException("Function deregisteration interrupted")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + deregisterDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + // + // Get Function Info + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testGetFunctionMissingTenant() throws IOException { + try { + testGetFunctionMissingArguments( + null, + namespace, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testGetFunctionMissingNamespace() throws IOException { + try { + testGetFunctionMissingArguments( + tenant, + null, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testGetFunctionMissingFunctionName() throws IOException { + try { + testGetFunctionMissingArguments( + tenant, + namespace, + null + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + // + // List Functions + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testListFunctionsMissingTenant() { + try { + testListFunctionsMissingArguments( + null, + namespace + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testListFunctionsMissingNamespace() { + try { + testListFunctionsMissingArguments( + tenant, + null + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testDownloadFunctionHttpUrl() throws Exception { + String jarHttpUrl = + "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; + File pkgFile = downloadFunction(jarHttpUrl, null); + pkgFile.delete(); + } + + @Test + public void testDownloadFunctionFile() throws Exception { + File file = getPulsarApiExamplesNar(); + File pkgFile = downloadFunction(file.toURI().toString(), null); + Assert.assertTrue(pkgFile.exists()); + Assert.assertEquals(file.length(), pkgFile.length()); + pkgFile.delete(); + } + + @Test + public void testDownloadFunctionBuiltinConnector() throws Exception { + File file = getPulsarApiExamplesNar(); + + WorkerConfig config = new WorkerConfig() + .setUploadBuiltinSinksSources(false); + when(mockedWorkerService.getWorkerConfig()).thenReturn(config); + + registerBuiltinConnector("cassandra", file); + + File pkgFile = downloadFunction("builtin://cassandra", null); + Assert.assertTrue(pkgFile.exists()); + Assert.assertEquals(file.length(), pkgFile.length()); + pkgFile.delete(); + } + + @Test + public void testDownloadFunctionBuiltinFunction() throws Exception { + File file = getPulsarApiExamplesNar(); + + WorkerConfig config = new WorkerConfig() + .setUploadBuiltinSinksSources(false); + when(mockedWorkerService.getWorkerConfig()).thenReturn(config); + + registerBuiltinFunction("exclamation", file); + + File pkgFile = downloadFunction("builtin://exclamation", null); + Assert.assertTrue(pkgFile.exists()); + Assert.assertEquals(file.length(), pkgFile.length()); + pkgFile.delete(); + } + + @Test + public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { + Configurator.setRootLevel(Level.DEBUG); + + File file = getPulsarApiExamplesNar(); + String filePackageUrl = file.toURI().toString(); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig); + + } + + @Test + public void testRegisterFunctionWithConflictingFields() throws Exception { + Configurator.setRootLevel(Level.DEBUG); + String actualTenant = "DIFFERENT_TENANT"; + String actualNamespace = "DIFFERENT_NAMESPACE"; + String actualName = "DIFFERENT_NAME"; + this.namespaceList.add(actualTenant + "/" + actualNamespace); + + File file = getPulsarApiExamplesNar(); + String filePackageUrl = file.toURI().toString(); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + registerFunction(actualTenant, actualNamespace, actualName, null, null, filePackageUrl, functionConfig); + } + + public static FunctionConfig createDefaultFunctionConfig() { + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + return functionConfig; + } + + public static FunctionDetails createDefaultFunctionDetails() { + FunctionConfig functionConfig = createDefaultFunctionConfig(); + return FunctionConfigUtils.convert(functionConfig); + } +} diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java new file mode 100644 index 0000000000000..9da30128f1c22 --- /dev/null +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java @@ -0,0 +1,323 @@ +/** + * 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.pulsar.functions.worker.rest.api.v3; + +import static org.apache.pulsar.functions.source.TopicSchema.DEFAULT_SERDE; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Consumer; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.pulsar.client.admin.Functions; +import org.apache.pulsar.client.admin.Namespaces; +import org.apache.pulsar.client.admin.Packages; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.Tenants; +import org.apache.pulsar.common.functions.FunctionDefinition; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.functions.instance.InstanceUtils; +import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.functions.runtime.RuntimeFactory; +import org.apache.pulsar.functions.source.TopicSchema; +import org.apache.pulsar.functions.utils.LoadedFunctionPackage; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; +import org.apache.pulsar.functions.utils.functions.FunctionArchive; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; +import org.apache.pulsar.functions.utils.io.Connector; +import org.apache.pulsar.functions.utils.io.ConnectorUtils; +import org.apache.pulsar.functions.worker.ConnectorsManager; +import org.apache.pulsar.functions.worker.FunctionMetaDataManager; +import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.FunctionsManager; +import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; +import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; +import org.mockito.Answers; +import org.mockito.MockSettings; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; + +public abstract class AbstractFunctionsResourceTest { + + protected static final String tenant = "test-tenant"; + protected static final String namespace = "test-namespace"; + protected static final Map topicsToSerDeClassName = new HashMap<>(); + protected static final String subscriptionName = "test-subscription"; + protected static final String CASSANDRA_STRING_SINK = "org.apache.pulsar.io.cassandra.CassandraStringSink"; + protected static final int parallelism = 1; + private static final String SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH = "pulsar-io-cassandra.nar.path"; + private static final String SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH = "pulsar-io-twitter.nar.path"; + private static final String SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH = "pulsar-io-invalid.nar.path"; + private static final String SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH = + "pulsar-functions-api-examples.nar.path"; + protected static Map mockStaticContexts = new HashMap<>(); + + static { + topicsToSerDeClassName.put("test_src", DEFAULT_SERDE); + topicsToSerDeClassName.put("persistent://public/default/test_src", TopicSchema.DEFAULT_SERDE); + } + + protected PulsarWorkerService mockedWorkerService; + protected PulsarAdmin mockedPulsarAdmin; + protected Tenants mockedTenants; + protected Namespaces mockedNamespaces; + protected Functions mockedFunctions; + protected TenantInfoImpl mockedTenantInfo; + protected List namespaceList = new LinkedList<>(); + protected FunctionMetaDataManager mockedManager; + protected FunctionRuntimeManager mockedFunctionRunTimeManager; + protected RuntimeFactory mockedRuntimeFactory; + protected Namespace mockedNamespace; + protected InputStream mockedInputStream; + protected FormDataContentDisposition mockedFormData; + protected Function.FunctionMetaData mockedFunctionMetaData; + protected LeaderService mockedLeaderService; + protected Packages mockedPackages; + protected PulsarFunctionTestTemporaryDirectory tempDirectory; + protected ConnectorsManager connectorsManager = new ConnectorsManager(); + protected FunctionsManager functionsManager = new FunctionsManager(); + + public static File getPulsarIOCassandraNar() { + return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH) + , "pulsar-io-cassandra.nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH + " system property")); + } + + public static File getPulsarIOTwitterNar() { + return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH) + , "pulsar-io-twitter.nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH + " system property")); + } + + public static File getPulsarIOInvalidNar() { + return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH) + , "invalid nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH + " system property")); + } + + public static File getPulsarApiExamplesNar() { + return new File(Objects.requireNonNull( + System.getProperty(SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH) + , "pulsar-functions-api-examples.nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH + " system property")); + } + + @BeforeMethod + public final void setup() throws Exception { + this.mockedManager = mock(FunctionMetaDataManager.class); + this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); + this.mockedRuntimeFactory = mock(RuntimeFactory.class); + this.mockedInputStream = mock(InputStream.class); + this.mockedNamespace = mock(Namespace.class); + this.mockedFormData = mock(FormDataContentDisposition.class); + when(mockedFormData.getFileName()).thenReturn("test"); + this.mockedTenantInfo = mock(TenantInfoImpl.class); + this.mockedPulsarAdmin = mock(PulsarAdmin.class); + this.mockedTenants = mock(Tenants.class); + this.mockedNamespaces = mock(Namespaces.class); + this.mockedFunctions = mock(Functions.class); + this.mockedLeaderService = mock(LeaderService.class); + this.mockedPackages = mock(Packages.class); + namespaceList.add(tenant + "/" + namespace); + + this.mockedWorkerService = mock(PulsarWorkerService.class); + when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); + when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); + when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); + when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); + when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); + when(mockedWorkerService.isInitialized()).thenReturn(true); + when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); + when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); + when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); + when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); + when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); + when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); + when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); + when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); + when(mockedLeaderService.isLeader()).thenReturn(true); + doAnswer(invocationOnMock -> { + Files.copy(getDefaultNarFile().toPath(), Paths.get(invocationOnMock.getArgument(1, String.class)), + StandardCopyOption.REPLACE_EXISTING); + return null; + }).when(mockedPackages).download(any(), any()); + + // worker config + WorkerConfig workerConfig = new WorkerConfig() + .setWorkerId("test") + .setWorkerPort(8080) + .setFunctionMetadataTopicName("pulsar/functions") + .setNumFunctionPackageReplicas(3) + .setPulsarServiceUrl("pulsar://localhost:6650/"); + tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); + tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); + when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); + when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); + when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); + + doSetup(); + } + + protected File getDefaultNarFile() { + return getPulsarIOTwitterNar(); + } + + protected void doSetup() throws Exception { + + } + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (tempDirectory != null) { + tempDirectory.delete(); + } + mockStaticContexts.values().forEach(MockedStatic::close); + mockStaticContexts.clear(); + } + + protected void mockStatic(Class classStatic, Consumer> consumer) { + mockStatic(classStatic, withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS), consumer); + } + + private void mockStatic(Class classStatic, MockSettings mockSettings, Consumer> consumer) { + final MockedStatic mockedStatic = mockStaticContexts.computeIfAbsent(classStatic.getName(), + name -> Mockito.mockStatic(classStatic, mockSettings)); + consumer.accept(mockedStatic); + } + + protected void mockWorkerUtils() { + mockWorkerUtils(null); + } + + protected void mockWorkerUtils(Consumer> consumer) { + mockStatic(WorkerUtils.class, withSettings(), ctx -> { + // make dumpToTmpFile return the nar file copy + ctx.when(() -> WorkerUtils.dumpToTmpFile(mockedInputStream)) + .thenAnswer(invocation -> { + Path tempFile = Files.createTempFile("test", ".nar"); + Files.copy(getPulsarApiExamplesNar().toPath(), tempFile, + StandardCopyOption.REPLACE_EXISTING); + return tempFile.toFile(); + }); + ctx.when(() -> WorkerUtils.dumpToTmpFile(any())) + .thenAnswer(Answers.CALLS_REAL_METHODS); + if (consumer != null) { + consumer.accept(ctx); + } + }); + } + + protected void mockInstanceUtils() { + mockStatic(InstanceUtils.class, ctx -> { + ctx.when(() -> InstanceUtils.calculateSubjectType(any())) + .thenReturn(getComponentType()); + }); + } + + protected abstract Function.FunctionDetails.ComponentType getComponentType(); + + public static class LoadedConnector extends Connector { + public LoadedConnector(ConnectorDefinition connectorDefinition) { + super(null, connectorDefinition, null, true); + } + + @Override + public ValidatableFunctionPackage getConnectorFunctionPackage() { + return new LoadedFunctionPackage(getClass().getClassLoader(), ConnectorDefinition.class, + getConnectorDefinition()); + } + } + + + protected void registerBuiltinConnector(String connectorType, String className) { + ConnectorDefinition connectorDefinition = null; + if (className != null) { + connectorDefinition = new ConnectorDefinition(); + // set source and sink class to the same to simplify the test + connectorDefinition.setSinkClass(className); + connectorDefinition.setSourceClass(className); + } + connectorsManager.addConnector(connectorType, new LoadedConnector(connectorDefinition)); + } + + protected void registerBuiltinConnector(String connectorType, File packageFile) { + ConnectorDefinition cntDef; + try { + cntDef = ConnectorUtils.getConnectorDefinition(packageFile); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + connectorsManager.addConnector(connectorType, + new Connector(packageFile.toPath(), cntDef, NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR, true)); + } + + public static class LoadedFunctionArchive extends FunctionArchive { + public LoadedFunctionArchive(FunctionDefinition functionDefinition) { + super(null, functionDefinition, null, true); + } + + @Override + public ValidatableFunctionPackage getFunctionPackage() { + return new LoadedFunctionPackage(getClass().getClassLoader(), FunctionDefinition.class, + getFunctionDefinition()); + } + } + + protected void registerBuiltinFunction(String functionType, String className) { + FunctionDefinition functionDefinition = null; + if (className != null) { + functionDefinition = new FunctionDefinition(); + functionDefinition.setFunctionClass(className); + } + functionsManager.addFunction(functionType, new LoadedFunctionArchive(functionDefinition)); + } + + protected void registerBuiltinFunction(String functionType, File packageFile) { + FunctionDefinition cntDef; + try { + cntDef = FunctionUtils.getFunctionDefinition(packageFile); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + functionsManager.addFunction(functionType, + new FunctionArchive(packageFile.toPath(), cntDef, NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR, true)); + } +} diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java index e449e8669cfe4..ae3c0255ea6fd 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java @@ -21,1693 +21,188 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import com.google.common.collect.Lists; import java.io.File; -import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.net.URL; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.UUID; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.Packages; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; import org.apache.pulsar.common.functions.FunctionConfig; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.util.RestException; -import org.apache.pulsar.functions.api.Context; -import org.apache.pulsar.functions.api.Function; -import org.apache.pulsar.functions.instance.InstanceUtils; -import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.proto.Function.FunctionMetaData; -import org.apache.pulsar.functions.proto.Function.PackageLocationMetaData; -import org.apache.pulsar.functions.proto.Function.ProcessingGuarantees; -import org.apache.pulsar.functions.proto.Function.SinkSpec; -import org.apache.pulsar.functions.proto.Function.SourceSpec; -import org.apache.pulsar.functions.proto.Function.SubscriptionType; -import org.apache.pulsar.functions.runtime.RuntimeFactory; -import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.utils.FunctionConfigUtils; -import org.apache.pulsar.functions.utils.functions.FunctionArchive; -import org.apache.pulsar.functions.utils.io.Connector; -import org.apache.pulsar.functions.worker.ConnectorsManager; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.FunctionsManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; -import org.apache.pulsar.functions.worker.rest.api.v2.FunctionsApiV2Resource; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; -import org.mockito.MockedStatic; -import org.mockito.Mockito; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -/** - * Unit test of {@link FunctionsApiV2Resource}. - */ -public class FunctionApiV3ResourceTest { - - private static final class TestFunction implements Function { - - @Override - public String process(String input, Context context) { - return input; - } - } - - private static final class WrongFunction implements Consumer { - @Override - public void accept(String s) { - - } - } - - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; - private static final String function = "test-function"; - private static final String outputTopic = "test-output-topic"; - private static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; - private static final String className = TestFunction.class.getName(); - private SubscriptionType subscriptionType = SubscriptionType.FAILOVER; - private static final Map topicsToSerDeClassName = new HashMap<>(); - static { - topicsToSerDeClassName.put("persistent://public/default/test_src", TopicSchema.DEFAULT_SERDE); - } - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; - private FunctionsImpl resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetadata; - private LeaderService mockedLeaderService; - private Packages mockedPackages; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - private static Map mockStaticContexts = new HashMap<>(); - - private static final String SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH = - "pulsar-functions-api-examples.nar.path"; - - public static File getPulsarApiExamplesNar() { - return new File(Objects.requireNonNull( - System.getProperty(SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH) - , "pulsar-functions-api-examples.nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH + " system property")); - } - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedPackages = mock(Packages.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedFunctionMetadata = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetadata); - doNothing().when(mockedPackages).download(anyString(), anyString()); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); - - this.resource = spy(new FunctionsImpl(() -> mockedWorkerService)); - } - - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - private void mockWorkerUtils() { - mockWorkerUtils(null); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); - } - - private void mockInstanceUtils() { - mockStatic(InstanceUtils.class, ctx -> { - ctx.when(() -> InstanceUtils.calculateSubjectType(any())) - .thenReturn(FunctionDetails.ComponentType.FUNCTION); - }); - } - - // - // Register Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testRegisterFunctionMissingTenant() { - try { - testRegisterFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testRegisterFunctionMissingNamespace() { - try { - testRegisterFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testRegisterFunctionMissingFunctionName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package is not provided") - public void testRegisterFunctionMissingPackage() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "No input topic\\(s\\) specified for the function") - public void testRegisterFunctionMissingInputTopics() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function Package is not provided") - public void testRegisterFunctionMissingPackageDetails() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package does not have" - + " the correct format. Pulsar cannot determine if the package is a NAR package or JAR package. Function " - + "classname is not provided and attempts to load it as a NAR package produced the following error.*") - public void testRegisterFunctionMissingClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class UnknownClass must be in class path") - public void testRegisterFunctionWrongClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - "UnknownClass", - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function parallelism must be a positive number") - public void testRegisterFunctionWrongParallelism() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - -2, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, - expectedExceptionsMessageRegExp = "Output topic persistent://public/default/test_src is also being used as an input topic \\(topics must be one or the other\\)") - public void testRegisterFunctionSameInputOutput() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - topicsToSerDeClassName.keySet().iterator().next(), - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Output topic " + function + "-output-topic/test:" + " is invalid") - public void testRegisterFunctionWrongOutputTopic() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - function + "-output-topic/test:", - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Encountered error .*. when getting Function package from .*") - public void testRegisterFunctionHttpUrl() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "http://localhost:1234/test"); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class .*. does not implement the correct interface") - public void testRegisterFunctionImplementWrongInterface() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - WrongFunction.class.getName(), - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testRegisterFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String functionPkgUrl) { - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - details, - functionPkgUrl, - functionConfig, - null); - - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function config is not provided") - public void testMissingFunctionConfig() { - resource.registerFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - null, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function config is not provided") - public void testUpdateMissingFunctionConfig() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - resource.updateFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - null, - null, null); - } - - private void registerDefaultFunction() { - registerDefaultFunctionWithPackageUrl(null); - } - - private void registerDefaultFunctionWithPackageUrl(String packageUrl) { - FunctionConfig functionConfig = createDefaultFunctionConfig(); - resource.registerFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - packageUrl, - functionConfig, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function already exists") - public void testRegisterExistedFunction() { - try { - Configurator.setRootLevel(Level.DEBUG); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testRegisterFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> { - WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class)); - } - ).thenThrow(new IOException("upload failure")); - ; - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testRegisterFunctionSuccess() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(timeOut = 20000) - public void testRegisterFunctionSuccessWithPackageName() { - registerDefaultFunctionWithPackageUrl("function://public/default/test@v1"); - } - - @Test(timeOut = 20000) - public void testRegisterFunctionFailedWithWrongPackageName() throws PulsarAdminException { - try { - doThrow(new PulsarAdminException("package name is invalid")) - .when(mockedPackages).download(anyString(), anyString()); - registerDefaultFunctionWithPackageUrl("function://"); - } catch (RestException e) { - // expected exception - assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace does not exist") - public void testRegisterFunctionNonExistingNamespace() { - try { - this.namespaceList.clear(); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant does not exist") - public void testRegisterFunctionNonexistantTenant() throws Exception { - try { - when(mockedTenants.getTenantInfo(any())).thenThrow(PulsarAdminException.NotFoundException.class); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testRegisterFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registration interrupted") - public void testRegisterFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - doThrow(new IllegalStateException("Function registration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - /* - Externally managed runtime, - uploadBuiltinSinksSources == false - Make sure uploadFileToBookkeeper is not called - */ - @Test - public void testRegisterFunctionSuccessK8sNoUpload() throws Exception { - mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(false); - - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new RuntimeException("uploadFileToBookkeeper triggered")); - - }); - - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getFunctionTypes(any(FunctionConfig.class), any(Class.class))).thenReturn(new Class[]{String.class, String.class}); - ctx.when(() -> FunctionCommon.convertRuntime(any(FunctionConfig.Runtime.class))).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - - }); - - doReturn(Function.class).when(mockedClassLoader).loadClass(anyString()); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("exclamation")).thenReturn(functionArchive); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); - - when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - FunctionConfig functionConfig = createDefaultFunctionConfig(); - functionConfig.setJar("builtin://exclamation"); - - try (FileInputStream inputStream = new FileInputStream(getPulsarApiExamplesNar())) { - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - mockedFormData, - null, - functionConfig, - null); - } - } - - /* - Externally managed runtime, - uploadBuiltinSinksSources == true - Make sure uploadFileToBookkeeper is called - */ - @Test - public void testRegisterFunctionSuccessK8sWithUpload() throws Exception { - final String injectedErrMsg = "uploadFileToBookkeeper triggered"; - mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(true); - - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new RuntimeException(injectedErrMsg)); - - }); - - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getFunctionTypes(any(FunctionConfig.class), any(Class.class))).thenReturn(new Class[]{String.class, String.class}); - ctx.when(() -> FunctionCommon.convertRuntime(any(FunctionConfig.Runtime.class))).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - }); - - doReturn(Function.class).when(mockedClassLoader).loadClass(anyString()); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedFunctionsManager.getFunctionArchive(any())).thenReturn(getPulsarApiExamplesNar().toPath()); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); - - when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - FunctionConfig functionConfig = createDefaultFunctionConfig(); - functionConfig.setJar("builtin://exclamation"); - - try (FileInputStream inputStream = new FileInputStream(getPulsarApiExamplesNar())) { - try { - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - mockedFormData, - null, - functionConfig, - null); - Assert.fail(); - } catch (RuntimeException e) { - Assert.assertEquals(e.getMessage(), injectedErrMsg); - } - } - } - - // - // Update Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testUpdateFunctionMissingTenant() throws Exception { - try { - testUpdateFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Tenant is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testUpdateFunctionMissingNamespace() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Namespace is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testUpdateFunctionMissingFunctionName() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Function name is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingPackage() throws Exception { - try { - mockWorkerUtils(); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingInputTopic() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingClassName() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test - public void testUpdateFunctionChangedParallelism() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism + 1, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test - public void testUpdateFunctionChangedInputs() throws Exception { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - "DifferentOutput", - outputSerdeClassName, - null, - parallelism, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Input Topics cannot be altered") - public void testUpdateFunctionChangedOutput() throws Exception { - try { - mockWorkerUtils(); - - Map someOtherInput = new HashMap<>(); - someOtherInput.put("DifferentTopic", TopicSchema.DEFAULT_SERDE); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - someOtherInput, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Input Topics cannot be altered"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testUpdateFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String expectedError) throws Exception { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - if (expectedError != null) { - doThrow(new IllegalArgumentException(expectedError)) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - } - - resource.updateFunction( - tenant, - namespace, - function, - inputStream, - details, - null, - functionConfig, - null, null); - - } - - private void updateDefaultFunction() { - updateDefaultFunctionWithPackageUrl(null); - } - - private void updateDefaultFunctionWithPackageUrl(String packageUrl) { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - - resource.updateFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - packageUrl, - functionConfig, - null, null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testUpdateNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testUpdateFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> { - WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class)); - - }).thenThrow(new IOException("upload failure")); - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testUpdateFunctionSuccess() throws Exception { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } - - @Test - public void testUpdateFunctionWithUrl() { - Configurator.setRootLevel(Level.DEBUG); - - String fileLocation = FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - String filePackageUrl = "file://" + fileLocation; - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - resource.updateFunction( - tenant, - namespace, - function, - null, - null, - filePackageUrl, - functionConfig, - null, null); - - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testUpdateFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registeration interrupted") - public void testUpdateFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function registeration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - - @Test(timeOut = 20000) - public void testUpdateFunctionSuccessWithPackageName() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - updateDefaultFunctionWithPackageUrl("function://public/default/test@v1"); - } - - @Test(timeOut = 20000) - public void testUpdateFunctionFailedWithWrongPackageName() throws PulsarAdminException { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - try { - doThrow(new PulsarAdminException("package name is invalid")) - .when(mockedPackages).download(anyString(), anyString()); - registerDefaultFunctionWithPackageUrl("function://"); - } catch (RestException e) { - // expected exception - assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - } - } - - // - // deregister function - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testDeregisterFunctionMissingTenant() { - try { - - testDeregisterFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testDeregisterFunctionMissingNamespace() { - try { - testDeregisterFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testDeregisterFunctionMissingFunctionName() { - try { - testDeregisterFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testDeregisterFunctionMissingArguments( - String tenant, - String namespace, - String function - ) { - resource.deregisterFunction( - tenant, - namespace, - function, - null); - } - - private void deregisterDefaultFunction() { - resource.deregisterFunction( - tenant, - namespace, - function, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testDeregisterNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); - throw re; - } - } - - @Test - public void testDeregisterFunctionSuccess() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - deregisterDefaultFunction(); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to deregister") - public void testDeregisterFunctionFailure() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to deregister")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } +public class FunctionApiV3ResourceTest extends AbstractFunctionApiResourceTest { + private FunctionsImpl resource; + @Override + protected void doSetup() { + super.doSetup(); + this.resource = spy(new FunctionsImpl(() -> mockedWorkerService)); } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function deregisteration interrupted") - public void testDeregisterFunctionInterrupted() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function deregisteration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } + protected void registerFunction(String tenant, String namespace, String function, InputStream inputStream, + FormDataContentDisposition details, String functionPkgUrl, FunctionConfig functionConfig) { + resource.registerFunction( + tenant, + namespace, + function, + inputStream, + details, + functionPkgUrl, + functionConfig, + null); + } + protected void updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + FunctionConfig functionConfig, + AuthenticationParameters authParams, + UpdateOptionsImpl updateOptions) { + resource.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, + functionConfig, authParams, updateOptions); } - // - // Get Function Info - // + protected StreamingOutput downloadFunction(String tenant, String namespace, String componentName, + AuthenticationParameters authParams) { + return resource.downloadFunction(tenant, namespace, componentName, authParams); + } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testGetFunctionMissingTenant() { - try { - testGetFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; + protected File downloadFunction(final String path, final AuthenticationParameters authParams) throws IOException { + StreamingOutput streamingOutput = resource.downloadFunction(path, authParams); + File pkgFile = File.createTempFile("testpkg", "nar"); + try(OutputStream output = new FileOutputStream(pkgFile)) { + streamingOutput.write(output); } + return pkgFile; } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testGetFunctionMissingNamespace() { - try { - testGetFunctionMissingArguments( + protected void testDeregisterFunctionMissingArguments( + String tenant, + String namespace, + String function + ) { + resource.deregisterFunction( tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + namespace, + function, + null); } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testGetFunctionMissingFunctionName() { - try { - testGetFunctionMissingArguments( + protected void deregisterDefaultFunction() { + resource.deregisterFunction( tenant, namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + function, + null); } - private void testGetFunctionMissingArguments( + protected void testGetFunctionMissingArguments( String tenant, String namespace, String function ) { resource.getFunctionInfo( - tenant, - namespace, - function,null + tenant, + namespace, + function, null ); } - private FunctionConfig getDefaultFunctionInfo() { + protected FunctionConfig getDefaultFunctionInfo() { return resource.getFunctionInfo( - tenant, - namespace, - function, - null - ); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testGetNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - getDefaultFunctionInfo(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); - throw re; - } - } - - @Test - public void testGetFunctionSuccess() { - mockInstanceUtils(); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - SinkSpec sinkSpec = SinkSpec.newBuilder() - .setTopic(outputTopic) - .setSerDeClassName(outputSerdeClassName).build(); - FunctionDetails functionDetails = FunctionDetails.newBuilder() - .setClassName(className) - .setSink(sinkSpec) - .setAutoAck(true) - .setName(function) - .setNamespace(namespace) - .setProcessingGuarantees(ProcessingGuarantees.ATMOST_ONCE) - .setTenant(tenant) - .setParallelism(parallelism) - .setSource(SourceSpec.newBuilder().setSubscriptionType(subscriptionType) - .putAllTopicsToSerDeClassName(topicsToSerDeClassName)).build(); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setCreateTime(System.currentTimeMillis()) - .setFunctionDetails(functionDetails) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) - .setVersion(1234) - .build(); - when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - - FunctionConfig functionConfig = getDefaultFunctionInfo(); - assertEquals( - FunctionConfigUtils.convertFromDetails(functionDetails), - functionConfig); - } - - // - // List Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testListFunctionsMissingTenant() { - try { - testListFunctionsMissingArguments( - null, - namespace - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testListFunctionsMissingNamespace() { - try { - testListFunctionsMissingArguments( tenant, + namespace, + function, null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + ); } - private void testListFunctionsMissingArguments( + protected void testListFunctionsMissingArguments( String tenant, String namespace ) { resource.listFunctions( - tenant, - namespace,null + tenant, + namespace, null ); } - private List listDefaultFunctions() { + protected List listDefaultFunctions() { return resource.listFunctions( - tenant, - namespace,null + tenant, + namespace, null ); } @Test - public void testListFunctionsSuccess() { - mockInstanceUtils(); - final List functions = Lists.newArrayList("test-1", "test-2"); - final List metaDataList = new LinkedList<>(); - FunctionMetaData functionMetaData1 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-1").build() - ).build(); - FunctionMetaData functionMetaData2 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-2").build() - ).build(); - metaDataList.add(functionMetaData1); - metaDataList.add(functionMetaData2); - when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(metaDataList); - - List functionList = listDefaultFunctions(); - assertEquals(functions, functionList); - } - - @Test - public void testOnlyGetSources() { - List functions = Lists.newArrayList("test-2"); - List functionMetaDataList = new LinkedList<>(); - FunctionMetaData f1 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder() - .setName("test-1") - .setComponentType(FunctionDetails.ComponentType.SOURCE) - .build()).build(); - functionMetaDataList.add(f1); - FunctionMetaData f2 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder() - .setName("test-2") - .setComponentType(FunctionDetails.ComponentType.FUNCTION) - .build()).build(); - functionMetaDataList.add(f2); - FunctionMetaData f3 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder() - .setName("test-3") - .setComponentType(FunctionDetails.ComponentType.SINK) - .build()).build(); - functionMetaDataList.add(f3); - when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(functionMetaDataList); - - List functionList = listDefaultFunctions(); - assertEquals(functions, functionList); - } - - @Test - public void testDownloadFunctionHttpUrl() throws Exception { - String jarHttpUrl = - "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - - StreamingOutput streamOutput = resource.downloadFunction(jarHttpUrl, null); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - pkgFile.delete(); - } - - @Test - public void testDownloadFunctionFile() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - - StreamingOutput streamOutput = resource.downloadFunction("file:///" + fileLocation, null); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - Assert.assertEquals(file.length(), pkgFile.length()); - pkgFile.delete(); - } - - @Test - public void testDownloadFunctionBuiltinConnector() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - + public void testDownloadFunctionBuiltinConnectorByName() throws Exception { + File file = getPulsarApiExamplesNar(); WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); + .setUploadBuiltinSinksSources(false); when(mockedWorkerService.getWorkerConfig()).thenReturn(config); - Connector connector = Connector.builder().archivePath(file.toPath()).build(); - ConnectorsManager connectorsManager = mock(ConnectorsManager.class); - when(connectorsManager.getConnector("cassandra")).thenReturn(connector); - when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); - - StreamingOutput streamOutput = resource.downloadFunction("builtin://cassandra", null); - - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - output.flush(); - output.close(); - Assert.assertTrue(pkgFile.exists()); - Assert.assertTrue(pkgFile.exists()); - Assert.assertEquals(file.length(), pkgFile.length()); - pkgFile.delete(); - } - - @Test - public void testDownloadFunctionBuiltinFunction() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - - WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); - when(mockedWorkerService.getWorkerConfig()).thenReturn(config); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - FunctionsManager functionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder().archivePath(file.toPath()).build(); - when(functionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mock(ConnectorsManager.class)); - when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("builtin://cassandra")) + .setFunctionDetails(Function.FunctionDetails.newBuilder().setComponentType(Function.FunctionDetails.ComponentType.SINK)) + .build(); + when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - StreamingOutput streamOutput = resource.downloadFunction("builtin://exclamation", null); + registerBuiltinConnector("cassandra", file); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); + StreamingOutput streamOutput = downloadFunction(tenant, namespace, function, + AuthenticationParameters.builder().build()); + File pkgFile = File.createTempFile("testpkg", "nar"); OutputStream output = new FileOutputStream(pkgFile); streamOutput.write(output); - output.flush(); - output.close(); Assert.assertTrue(pkgFile.exists()); Assert.assertEquals(file.length(), pkgFile.length()); pkgFile.delete(); } @Test - public void testDownloadFunctionBuiltinConnectorByName() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + public void testDownloadFunctionBuiltinFunctionByName() throws Exception { + File file = getPulsarApiExamplesNar(); WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); + .setUploadBuiltinSinksSources(false); when(mockedWorkerService.getWorkerConfig()).thenReturn(config); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("builtin://cassandra")) - .setFunctionDetails(FunctionDetails.newBuilder().setComponentType(FunctionDetails.ComponentType.SINK)) + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("builtin://exclamation")) + .setFunctionDetails( + Function.FunctionDetails.newBuilder().setComponentType(Function.FunctionDetails.ComponentType.FUNCTION)) .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - Connector connector = Connector.builder().archivePath(file.toPath()).build(); - ConnectorsManager connectorsManager = mock(ConnectorsManager.class); - when(connectorsManager.getConnector("cassandra")).thenReturn(connector); - when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); + registerBuiltinFunction("exclamation", file); - StreamingOutput streamOutput = resource.downloadFunction(tenant, namespace, function, + StreamingOutput streamOutput = downloadFunction(tenant, namespace, function, AuthenticationParameters.builder().build()); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); + File pkgFile = File.createTempFile("testpkg", "nar"); OutputStream output = new FileOutputStream(pkgFile); streamOutput.write(output); Assert.assertTrue(pkgFile.exists()); @@ -1715,47 +210,58 @@ public void testDownloadFunctionBuiltinConnectorByName() throws Exception { pkgFile.delete(); } - @Test - public void testDownloadFunctionBuiltinFunctionByName() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); - when(mockedWorkerService.getWorkerConfig()).thenReturn(config); + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't" + + " exist") + public void testGetNotExistedFunction() throws IOException { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + getDefaultFunctionInfo(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); + throw re; + } + } + @Test + public void testGetFunctionSuccess() throws IOException { + mockInstanceUtils(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("builtin://exclamation")) - .setFunctionDetails(FunctionDetails.newBuilder().setComponentType(FunctionDetails.ComponentType.FUNCTION)) - .build(); + Function.SinkSpec sinkSpec = Function.SinkSpec.newBuilder() + .setTopic(outputTopic) + .setSerDeClassName(outputSerdeClassName).build(); + Function.FunctionDetails functionDetails = Function.FunctionDetails.newBuilder() + .setClassName(className) + .setSink(sinkSpec) + .setAutoAck(true) + .setName(function) + .setNamespace(namespace) + .setProcessingGuarantees(Function.ProcessingGuarantees.ATMOST_ONCE) + .setTenant(tenant) + .setParallelism(parallelism) + .setSource(Function.SourceSpec.newBuilder().setSubscriptionType(subscriptionType) + .putAllTopicsToSerDeClassName(topicsToSerDeClassName)).build(); + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setCreateTime(System.currentTimeMillis()) + .setFunctionDetails(functionDetails) + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) + .setVersion(1234) + .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - FunctionsManager functionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder().archivePath(file.toPath()).build(); - when(functionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mock(ConnectorsManager.class)); - when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); - - StreamingOutput streamOutput = resource.downloadFunction(tenant, namespace, function, - AuthenticationParameters.builder().build()); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - Assert.assertEquals(file.length(), pkgFile.length()); - pkgFile.delete(); + FunctionConfig functionConfig = getDefaultFunctionInfo(); + assertEquals( + FunctionConfigUtils.convertFromDetails(functionDetails), + functionConfig); } - @Test - public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function language runtime is " + + "either not set or cannot be determined") + public void testCreateFunctionWithoutSettingRuntime() throws Exception { Configurator.setRootLevel(Level.DEBUG); - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; + File file = getPulsarApiExamplesNar(); + String filePackageUrl = file.toURI().toString(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); FunctionConfig functionConfig = new FunctionConfig(); @@ -1764,82 +270,78 @@ public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { functionConfig.setName(function); functionConfig.setClassName(className); functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); functionConfig.setOutput(outputTopic); functionConfig.setOutputSerdeClassName(outputSerdeClassName); - resource.registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig, null); + registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig); } - @Test - public void testRegisterFunctionWithConflictingFields() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - String actualTenant = "DIFFERENT_TENANT"; - String actualNamespace = "DIFFERENT_NAMESPACE"; - String actualName = "DIFFERENT_NAME"; - this.namespaceList.add(actualTenant + "/" + actualNamespace); + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function config is not provided") + public void testMissingFunctionConfig() throws IOException { + registerFunction(tenant, namespace, function, mockedInputStream, mockedFormData, null, null); + } - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); + /* + Externally managed runtime, + uploadBuiltinSinksSources == false + Make sure uploadFileToBookkeeper is not called + */ + @Test + public void testRegisterFunctionSuccessK8sNoUpload() throws Exception { + mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(false); - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - resource.registerFunction(actualTenant, actualNamespace, actualName, null, null, filePackageUrl, functionConfig, - null); - } + mockStatic(WorkerUtils.class, ctx -> { + ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class))) + .thenThrow(new RuntimeException("uploadFileToBookkeeper triggered")); - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function language runtime is either not set or cannot be determined") - public void testCreateFunctionWithoutSettingRuntime() throws Exception { - Configurator.setRootLevel(Level.DEBUG); + }); - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; + registerBuiltinFunction("exclamation", getPulsarApiExamplesNar()); + when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - resource.registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig, null); + FunctionConfig functionConfig = createDefaultFunctionConfig(); + functionConfig.setJar("builtin://exclamation"); + registerFunction(tenant, namespace, function, null, mockedFormData, null, functionConfig); } - public static FunctionConfig createDefaultFunctionConfig() { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - return functionConfig; - } + /* + Externally managed runtime, + uploadBuiltinSinksSources == true + Make sure uploadFileToBookkeeper is called + */ + @Test + public void testRegisterFunctionSuccessK8sWithUpload() throws Exception { + final String injectedErrMsg = "uploadFileToBookkeeper triggered"; + mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(true); + + mockStatic(WorkerUtils.class, ctx -> { + ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class))) + .thenThrow(new RuntimeException(injectedErrMsg)); + + }); + + registerBuiltinFunction("exclamation", getPulsarApiExamplesNar()); + when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - public static FunctionDetails createDefaultFunctionDetails() { FunctionConfig functionConfig = createDefaultFunctionConfig(); - return FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + functionConfig.setJar("builtin://exclamation"); + + try { + registerFunction(tenant, namespace, function, null, mockedFormData, null, functionConfig); + Assert.fail(); + } catch (RuntimeException e) { + Assert.assertEquals(e.getMessage(), injectedErrMsg); + } } + } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index a061c23eaa6a2..a9927514efd05 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -22,232 +22,73 @@ import static org.apache.pulsar.functions.source.TopicSchema.DEFAULT_SERDE; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.eq; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.nio.file.StandardCopyOption; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.Packages; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; -import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.functions.Utils; import org.apache.pulsar.common.io.SinkConfig; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.apache.pulsar.functions.proto.Function.FunctionMetaData; -import org.apache.pulsar.functions.runtime.RuntimeFactory; -import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SinkConfigUtils; -import org.apache.pulsar.functions.utils.io.Connector; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; -import org.apache.pulsar.functions.worker.ConnectorsManager; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; -import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; import org.apache.pulsar.functions.worker.rest.api.SinksImpl; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.mockito.MockedStatic; import org.mockito.Mockito; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; /** * Unit test of {@link SinksApiV3Resource}. */ -public class SinkApiV3ResourceTest { +public class SinkApiV3ResourceTest extends AbstractFunctionsResourceTest { - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; private static final String sink = "test-sink"; - private static final Map topicsToSerDeClassName = new HashMap<>(); - - static { - topicsToSerDeClassName.put("test_src", DEFAULT_SERDE); - } - - private static final String subscriptionName = "test-subscription"; - private static final String CASSANDRA_STRING_SINK = "org.apache.pulsar.io.cassandra.CassandraStringSink"; - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; + private SinksImpl resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetaData; - private LeaderService mockedLeaderService; - private Packages mockedPackages; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - private static Map mockStaticContexts = new HashMap<>(); - - private static final String SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH = "pulsar-io-cassandra.nar.path"; - - public static File getPulsarIOCassandraNar() { - return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH) - , "pulsar-io-cassandra.nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH + " system property")); - } - - private static final String SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH = "pulsar-io-twitter.nar.path"; - - public static File getPulsarIOTwitterNar() { - return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH) - , "pulsar-io-twitter.nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH + " system property")); - } - - private static final String SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH = "pulsar-io-invalid.nar.path"; - - public static File getPulsarIOInvalidNar() { - return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH) - , "invalid nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH + " system property")); - } - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedPackages = mock(Packages.class); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - doAnswer(invocationOnMock -> { - Files.copy(getPulsarIOCassandraNar().toPath(), Paths.get(invocationOnMock.getArgument(1, String.class)), - StandardCopyOption.REPLACE_EXISTING); - return null; - }).when(mockedPackages).download(any(), any()); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); + @Override + protected void doSetup() { this.resource = spy(new SinksImpl(() -> mockedWorkerService)); - - } - - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = - mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - private void mockWorkerUtils() { - mockWorkerUtils(null); } - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); + @Override + protected Function.FunctionDetails.ComponentType getComponentType() { + return Function.FunctionDetails.ComponentType.SINK; } - private void mockInstanceUtils() { - mockStatic(InstanceUtils.class, ctx -> { - ctx.when(() -> InstanceUtils.calculateSubjectType(any())) - .thenReturn(FunctionDetails.ComponentType.SINK); - }); + @Override + protected File getDefaultNarFile() { + return getPulsarIOCassandraNar(); } - - // - // Register Functions - // - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") public void testRegisterSinkMissingTenant() { try { @@ -327,8 +168,8 @@ public void testRegisterSinkMissingPackage() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink class UnknownClass must " - + "be in class path") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink class UnknownClass not " + + "found") public void testRegisterSinkWrongClassName() { mockInstanceUtils(); try { @@ -349,10 +190,8 @@ public void testRegisterSinkWrongClassName() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink package does not have the" - + " correct format. Pulsar cannot determine if the package is a NAR package" - + " or JAR package. Sink classname is not provided and attempts to load it as a NAR package produced the " - + "following error.") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink package doesn't contain " + + "the META-INF/services/pulsar-io.yaml file.") public void testRegisterSinkMissingPackageDetails() { mockInstanceUtils(); try { @@ -636,13 +475,7 @@ public void testRegisterSinkConflictingFields() throws Exception { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); - SinkConfig sinkConfig = new SinkConfig(); - sinkConfig.setTenant(tenant); - sinkConfig.setNamespace(namespace); - sinkConfig.setName(sink); - sinkConfig.setClassName(CASSANDRA_STRING_SINK); - sinkConfig.setParallelism(parallelism); - sinkConfig.setTopicToSerdeClassName(topicsToSerDeClassName); + SinkConfig sinkConfig = createDefaultSinkConfig(); try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { resource.registerSink( actualTenant, @@ -841,16 +674,18 @@ public void testUpdateSinkDifferentInputs() throws Exception { public void testUpdateSinkDifferentParallelism() throws Exception { mockWorkerUtils(); - testUpdateSinkMissingArguments( - tenant, - namespace, - sink, - null, - mockedFormData, - topicsToSerDeClassName, - CASSANDRA_STRING_SINK, - parallelism + 1, - null); + try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { + testUpdateSinkMissingArguments( + tenant, + namespace, + sink, + inputStream, + mockedFormData, + topicsToSerDeClassName, + CASSANDRA_STRING_SINK, + parallelism + 1, + null); + } } private void testUpdateSinkMissingArguments( @@ -863,29 +698,7 @@ private void testUpdateSinkMissingArguments( String className, Integer parallelism, String expectedError) throws Exception { - mockStatic(ConnectorUtils.class, ctx -> { - ctx.when(() -> ConnectorUtils.getIOSinkClass(any(NarClassLoader.class))) - .thenReturn(CASSANDRA_STRING_SINK); - }); - - mockStatic(ClassLoaderUtils.class, ctx -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mock(NarClassLoader.class)); - ctx.when(() -> FunctionCommon - .convertProcessingGuarantee(eq(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE))) - .thenReturn(ATLEAST_ONCE); - }); - - this.mockedFunctionMetaData = - FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); + mockFunctionCommon(tenant, namespace, sink); SinkConfig sinkConfig = new SinkConfig(); if (tenant != null) { @@ -929,32 +742,7 @@ private void updateDefaultSink() throws Exception { } private void updateDefaultSinkWithPackageUrl(String packageUrl) throws Exception { - SinkConfig sinkConfig = new SinkConfig(); - sinkConfig.setTenant(tenant); - sinkConfig.setNamespace(namespace); - sinkConfig.setName(sink); - sinkConfig.setClassName(CASSANDRA_STRING_SINK); - sinkConfig.setParallelism(parallelism); - sinkConfig.setTopicToSerdeClassName(topicsToSerDeClassName); - - mockStatic(ConnectorUtils.class, ctx -> { - ctx.when(() -> ConnectorUtils.getIOSinkClass(any(NarClassLoader.class))) - .thenReturn(CASSANDRA_STRING_SINK); - }); - - mockStatic(ClassLoaderUtils.class, ctx -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mock(NarClassLoader.class)); - ctx.when(() -> FunctionCommon - .convertProcessingGuarantee(eq(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE))) - .thenReturn(ATLEAST_ONCE); - }); - + SinkConfig sinkConfig = createDefaultSinkConfig(); this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); @@ -988,7 +776,6 @@ public void testUpdateNotExistedSink() throws Exception { public void testUpdateSinkUploadFailure() throws Exception { try { mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( anyString(), any(File.class), @@ -1020,34 +807,10 @@ public void testUpdateSinkWithUrl() throws Exception { String filePackageUrl = getPulsarIOCassandraNar().toURI().toString(); - SinkConfig sinkConfig = new SinkConfig(); - sinkConfig.setTopicToSerdeClassName(topicsToSerDeClassName); - sinkConfig.setTenant(tenant); - sinkConfig.setNamespace(namespace); - sinkConfig.setName(sink); - sinkConfig.setClassName(CASSANDRA_STRING_SINK); - sinkConfig.setParallelism(parallelism); + SinkConfig sinkConfig = createDefaultSinkConfig(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); - mockStatic(ConnectorUtils.class, ctx -> { - ctx.when(() -> ConnectorUtils.getIOSinkClass(any())) - .thenReturn(CASSANDRA_STRING_SINK); - }); - - mockStatic(ClassLoaderUtils.class, ctx -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.extractFileFromPkgURL(any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mock(NarClassLoader.class)); - ctx.when(() -> FunctionCommon - .convertProcessingGuarantee(eq(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE))) - .thenReturn(ATLEAST_ONCE); - }); - this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); @@ -1239,7 +1002,7 @@ public void testDeregisterSinkInterrupted() throws Exception { } @Test - public void testDeregisterSinkBKPackageCleanup() throws IOException { + public void testDeregisterSinkBKPackageCleanup() { mockInstanceUtils(); try (final MockedStatic ctx = Mockito.mockStatic(WorkerUtils.class)) { @@ -1247,15 +1010,15 @@ public void testDeregisterSinkBKPackageCleanup() throws IOException { String packagePath = "public/default/test/591541f0-c7c5-40c0-983b-610c722f90b0-pulsar-io-batch-data-generator-2.7.0.nar"; + FunctionMetaData functionMetaData = FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath)) + .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))) - .thenReturn(FunctionMetaData.newBuilder().setPackageLocation( - Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath).build()).build()); + .thenReturn(functionMetaData); deregisterDefaultSink(); - ctx.verify(() -> { - WorkerUtils.deleteFromBookkeeper(any(), eq(packagePath)); - }, times(1)); + ctx.verify(() -> WorkerUtils.deleteFromBookkeeper(any(), eq(packagePath)), times(1)); } } @@ -1267,16 +1030,16 @@ public void testDeregisterBuiltinSinkBKPackageCleanup() { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); String packagePath = String.format("%s://data-generator", Utils.BUILTIN); + FunctionMetaData functionMetaData = FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath)) + .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))) - .thenReturn(FunctionMetaData.newBuilder().setPackageLocation( - Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath).build()).build()); + .thenReturn(functionMetaData); deregisterDefaultSink(); // if the sink is a builtin sink we shouldn't try to clean it up - ctx.verify(() -> { - WorkerUtils.deleteFromBookkeeper(any(), eq(packagePath)); - }, times(0)); + ctx.verify(() -> WorkerUtils.deleteFromBookkeeper(any(), anyString()), times(0)); } } @@ -1289,22 +1052,22 @@ public void testDeregisterHTTPSinkBKPackageCleanup() { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); String packagePath = "http://foo.com/connector.jar"; + FunctionMetaData functionMetaData = FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath)) + .build(); + when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))) - .thenReturn(FunctionMetaData.newBuilder().setPackageLocation( - Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath).build()).build()); + .thenReturn(functionMetaData); deregisterDefaultSink(); // if the sink is a is download from a http url, we shouldn't try to clean it up - ctx.verify(() -> { - WorkerUtils.deleteFromBookkeeper(any(), eq(packagePath)); - }, times(0)); - + ctx.verify(() -> WorkerUtils.deleteFromBookkeeper(any(), anyString()), times(0)); } } @Test - public void testDeregisterFileSinkBKPackageCleanup() throws IOException { + public void testDeregisterFileSinkBKPackageCleanup() { mockInstanceUtils(); try (final MockedStatic ctx = Mockito.mockStatic(WorkerUtils.class)) { @@ -1312,16 +1075,17 @@ public void testDeregisterFileSinkBKPackageCleanup() throws IOException { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); String packagePath = "file://foo/connector.jar"; + FunctionMetaData functionMetaData = FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath)) + .build(); + when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))) - .thenReturn(FunctionMetaData.newBuilder().setPackageLocation( - Function.PackageLocationMetaData.newBuilder().setPackagePath(packagePath).build()).build()); + .thenReturn(functionMetaData); deregisterDefaultSink(); // if the sink package has a file url, we shouldn't try to clean it up - ctx.verify(() -> { - WorkerUtils.deleteFromBookkeeper(any(), eq(packagePath)); - }, times(0)); + ctx.verify(() -> WorkerUtils.deleteFromBookkeeper(any(), eq(packagePath)), times(0)); } } @@ -1577,6 +1341,14 @@ private SinkConfig createDefaultSinkConfig() { return sinkConfig; } + private void mockFunctionCommon(String tenant, String namespace, String sink) throws IOException { + this.mockedFunctionMetaData = + Function.FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))).thenReturn(mockedFunctionMetaData); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); + } + private FunctionDetails createDefaultFunctionDetails() throws IOException { return SinkConfigUtils.convert(createDefaultSinkConfig(), new SinkConfigUtils.ExtractedSinkDetails(null, null)); @@ -1600,21 +1372,7 @@ public void testRegisterSinkSuccessK8sNoUpload() throws Exception { }); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - - }); - - ConnectorsManager mockedConnManager = mock(ConnectorsManager.class); - Connector connector = Connector.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedConnManager.getConnector("cassandra")).thenReturn(connector); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mockedConnManager); + registerBuiltinConnector("cassandra", getPulsarIOCassandraNar()); when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(false); @@ -1654,23 +1412,7 @@ public void testRegisterSinkSuccessK8sWithUpload() throws Exception { }); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - }); - - ConnectorsManager mockedConnManager = mock(ConnectorsManager.class); - Connector connector = Connector.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedConnManager.getConnector("cassandra")).thenReturn(connector); - when(mockedConnManager.getSinkArchive(any())).thenReturn(getPulsarIOCassandraNar().toPath()); - - when(mockedWorkerService.getConnectorsManager()).thenReturn(mockedConnManager); - + registerBuiltinConnector("cassandra", getPulsarIOCassandraNar()); when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(false); @@ -1695,4 +1437,69 @@ public void testRegisterSinkSuccessK8sWithUpload() throws Exception { } } } + + @Test + public void testUpdateSinkWithNoChange() throws IOException { + mockWorkerUtils(); + + // No change on config, + SinkConfig sinkConfig = createDefaultSinkConfig(); + + mockStatic(SinkConfigUtils.class, ctx -> { + ctx.when(() -> SinkConfigUtils.convertFromDetails(any())).thenReturn(sinkConfig); + }); + + mockFunctionCommon(sinkConfig.getTenant(), sinkConfig.getNamespace(), sinkConfig.getName()); + + // config has not changes and don't update auth, should fail + try { + resource.updateSink( + sinkConfig.getTenant(), + sinkConfig.getNamespace(), + sinkConfig.getName(), + null, + mockedFormData, + null, + sinkConfig, + null, + null); + fail("Update without changes should fail"); + } catch (RestException e) { + assertTrue(e.getMessage().contains("Update contains no change")); + } + + try { + UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); + updateOptions.setUpdateAuthData(false); + resource.updateSink( + sinkConfig.getTenant(), + sinkConfig.getNamespace(), + sinkConfig.getName(), + null, + mockedFormData, + null, + sinkConfig, + null, + updateOptions); + fail("Update without changes should fail"); + } catch (RestException e) { + assertTrue(e.getMessage().contains("Update contains no change")); + } + + // no changes but set the auth-update flag to true, should not fail + UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); + updateOptions.setUpdateAuthData(true); + try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { + resource.updateSink( + sinkConfig.getTenant(), + sinkConfig.getNamespace(), + sinkConfig.getName(), + inputStream, + mockedFormData, + null, + sinkConfig, + null, + updateOptions); + } + } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java index 699cd1e5a183a..3a76ca1a469d3 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java @@ -18,50 +18,32 @@ */ package org.apache.pulsar.functions.worker.rest.api.v3; -import static org.apache.pulsar.functions.worker.rest.api.v3.SinkApiV3ResourceTest.getPulsarIOCassandraNar; -import static org.apache.pulsar.functions.worker.rest.api.v3.SinkApiV3ResourceTest.getPulsarIOInvalidNar; -import static org.apache.pulsar.functions.worker.rest.api.v3.SinkApiV3ResourceTest.getPulsarIOTwitterNar; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.nio.file.StandardCopyOption; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.Packages; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.functions.Utils; import org.apache.pulsar.common.io.SourceConfig; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.utils.IdentityFunction; @@ -71,159 +53,35 @@ import org.apache.pulsar.functions.proto.Function.ProcessingGuarantees; import org.apache.pulsar.functions.proto.Function.SinkSpec; import org.apache.pulsar.functions.proto.Function.SourceSpec; -import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SourceConfigUtils; import org.apache.pulsar.functions.utils.io.ConnectorUtils; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; -import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.mockito.MockedStatic; import org.mockito.Mockito; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; /** * Unit test of {@link SourcesApiV3Resource}. */ -public class SourceApiV3ResourceTest { +public class SourceApiV3ResourceTest extends AbstractFunctionsResourceTest { - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; private static final String source = "test-source"; private static final String outputTopic = "test-output-topic"; private static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; private static final String TWITTER_FIRE_HOSE = "org.apache.pulsar.io.twitter.TwitterFireHose"; - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; private SourcesImpl resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetaData; - private LeaderService mockedLeaderService; - private Packages mockedPackages; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - - private static NarClassLoader narClassLoader; - private static Map mockStaticContexts = new HashMap<>(); - - @BeforeClass - public void setupNarClassLoader() throws IOException { - narClassLoader = NarClassLoaderBuilder.builder().narFile(getPulsarIOTwitterNar()).build(); - } - - @AfterClass(alwaysRun = true) - public void cleanupNarClassLoader() throws IOException { - if (narClassLoader != null) { - narClassLoader.close(); - narClassLoader = null; - } - } - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedPackages = mock(Packages.class); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - doAnswer(invocationOnMock -> { - Files.copy(getPulsarIOTwitterNar().toPath(), Paths.get(invocationOnMock.getArgument(1, String.class)), - StandardCopyOption.REPLACE_EXISTING); - return null; - }).when(mockedPackages).download(any(), any()); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); + @Override + protected void doSetup() { this.resource = spy(new SourcesImpl(() -> mockedWorkerService)); } - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = - mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockWorkerUtils() { - mockStatic(WorkerUtils.class, - ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - }); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); + @Override + protected FunctionDetails.ComponentType getComponentType() { + return FunctionDetails.ComponentType.SOURCE; } // @@ -293,8 +151,8 @@ public void testRegisterSourceMissingSourceName() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source class UnknownClass must" - + " be in class path") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source class UnknownClass not " + + "found in class loader") public void testRegisterSourceWrongClassName() { try { testRegisterSourceMissingArguments( @@ -357,10 +215,8 @@ public void testRegisterSourceMissingPackageDetails() throws IOException { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package does not have the" - + " correct format. Pulsar cannot determine if the package is a NAR package" - + " or JAR package. Source classname is not provided and attempts to load it as a NAR package " - + "produced the following error.") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package doesn't contain" + + " the META-INF/services/pulsar-io.yaml file.") public void testRegisterSourceMissingPackageDetailsAndClassname() { try { testRegisterSourceMissingArguments( @@ -381,8 +237,8 @@ public void testRegisterSourceMissingPackageDetailsAndClassname() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package does not have " - + "the correct format.*") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package doesn't contain" + + " the META-INF/services/pulsar-io.yaml file.") public void testRegisterSourceInvalidJarWithNoSource() throws IOException { try (InputStream inputStream = new FileInputStream(getPulsarIOInvalidNar())) { testRegisterSourceMissingArguments( @@ -520,7 +376,7 @@ public void testUpdateMissingSinkConfig() { } private void registerDefaultSource() throws IOException { - registerDefaultSourceWithPackageUrl("source://public/default/test@v1"); + registerDefaultSourceWithPackageUrl(getPulsarIOTwitterNar().toURI().toString()); } private void registerDefaultSourceWithPackageUrl(String packageUrl) throws IOException { @@ -561,8 +417,6 @@ public void testRegisterSourceUploadFailure() throws Exception { any(File.class), any(Namespace.class))) .thenThrow(new IOException("upload failure")); - - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); }); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(false); @@ -589,7 +443,7 @@ public void testRegisterSourceSuccess() throws Exception { @Test(timeOut = 20000) public void testRegisterSourceSuccessWithPackageName() throws IOException { - registerDefaultSourceWithPackageUrl("source://public/default/test@v1"); + registerDefaultSource(); } @Test(timeOut = 20000) @@ -617,14 +471,7 @@ public void testRegisterSourceConflictingFields() throws Exception { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(true); when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setTenant(tenant); - sourceConfig.setNamespace(namespace); - sourceConfig.setName(source); - sourceConfig.setClassName(TWITTER_FIRE_HOSE); - sourceConfig.setParallelism(parallelism); - sourceConfig.setTopicName(outputTopic); - sourceConfig.setSerdeClassName(outputSerdeClassName); + SourceConfig sourceConfig = createDefaultSourceConfig(); try (InputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { resource.registerSource( actualTenant, @@ -811,38 +658,106 @@ public void testUpdateSourceChangedParallelism() throws Exception { try { mockWorkerUtils(); + try(FileInputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + testUpdateSourceMissingArguments( + tenant, + namespace, + source, + inputStream, + mockedFormData, + outputTopic, + outputSerdeClassName, + TWITTER_FIRE_HOSE, + parallelism + 1, + null); + } + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testUpdateSourceChangedTopic() throws Exception { + mockWorkerUtils(); + + try(FileInputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { testUpdateSourceMissingArguments( tenant, namespace, source, - null, + inputStream, mockedFormData, - outputTopic, + "DifferentTopic", outputSerdeClassName, TWITTER_FIRE_HOSE, - parallelism + 1, + parallelism, null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; } } @Test - public void testUpdateSourceChangedTopic() throws Exception { + public void testUpdateSourceWithNoChange() throws IOException { mockWorkerUtils(); - testUpdateSourceMissingArguments( - tenant, - namespace, - source, - null, - mockedFormData, - "DifferentTopic", - outputSerdeClassName, - TWITTER_FIRE_HOSE, - parallelism, - null); + // No change on config, + SourceConfig sourceConfig = createDefaultSourceConfig(); + mockStatic(SourceConfigUtils.class, ctx -> { + ctx.when(() -> SourceConfigUtils.convertFromDetails(any())).thenReturn(sourceConfig); + }); + + mockFunctionCommon(sourceConfig.getTenant(), sourceConfig.getNamespace(), sourceConfig.getName()); + + // config has not changes and don't update auth, should fail + try { + resource.updateSource( + sourceConfig.getTenant(), + sourceConfig.getNamespace(), + sourceConfig.getName(), + null, + mockedFormData, + null, + sourceConfig, + null, + null); + fail("Update without changes should fail"); + } catch (RestException e) { + assertTrue(e.getMessage().contains("Update contains no change")); + } + + try { + UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); + updateOptions.setUpdateAuthData(false); + resource.updateSource( + sourceConfig.getTenant(), + sourceConfig.getNamespace(), + sourceConfig.getName(), + null, + mockedFormData, + null, + sourceConfig, + null, + updateOptions); + fail("Update without changes should fail"); + } catch (RestException e) { + assertTrue(e.getMessage().contains("Update contains no change")); + } + + // no changes but set the auth-update flag to true, should not fail + UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); + updateOptions.setUpdateAuthData(true); + try (InputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + resource.updateSource( + sourceConfig.getTenant(), + sourceConfig.getNamespace(), + sourceConfig.getName(), + inputStream, + mockedFormData, + null, + sourceConfig, + null, + updateOptions); + } } @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source parallelism must be a " @@ -880,26 +795,7 @@ private void testUpdateSourceMissingArguments( Integer parallelism, String expectedError) throws Exception { - mockStatic(ConnectorUtils.class, c -> { - }); - mockStatic(ClassLoaderUtils.class, c -> { - }); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSourceType(argThat(clazz -> clazz.getName().equals(TWITTER_FIRE_HOSE)))) - .thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())) - .thenReturn(narClassLoader); - - - }); - - this.mockedFunctionMetaData = - FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + mockFunctionCommon(tenant, namespace, function); SourceConfig sourceConfig = new SourceConfig(); if (tenant != null) { @@ -941,50 +837,39 @@ private void testUpdateSourceMissingArguments( } - private void updateDefaultSource() throws Exception { - updateDefaultSourceWithPackageUrl(null); - } - - private void updateDefaultSourceWithPackageUrl(String packageUrl) throws Exception { - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setTenant(tenant); - sourceConfig.setNamespace(namespace); - sourceConfig.setName(source); - sourceConfig.setClassName(TWITTER_FIRE_HOSE); - sourceConfig.setParallelism(parallelism); - sourceConfig.setTopicName(outputTopic); - sourceConfig.setSerdeClassName(outputSerdeClassName); - + private void mockFunctionCommon(String tenant, String namespace, String function) { mockStatic(ConnectorUtils.class, c -> { }); - mockStatic(ClassLoaderUtils.class, c -> { }); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSourceType(argThat(clazz -> clazz.getName().equals(TWITTER_FIRE_HOSE)))) - .thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())) - .thenReturn(narClassLoader); - }); + this.mockedFunctionMetaData = + FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + } + + private void updateDefaultSource() throws Exception { + updateDefaultSourceWithPackageUrl(getPulsarIOTwitterNar().toURI().toString()); + } + + private void updateDefaultSourceWithPackageUrl(String packageUrl) throws Exception { + SourceConfig sourceConfig = createDefaultSourceConfig(); this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); - try (InputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { - resource.updateSource( - tenant, - namespace, - source, - inputStream, - mockedFormData, - packageUrl, - sourceConfig, - null, null); - } + resource.updateSource( + tenant, + namespace, + source, + null, + mockedFormData, + packageUrl, + sourceConfig, + null, null); } @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source test-source doesn't " + @@ -1007,11 +892,25 @@ public void testUpdateSourceUploadFailure() throws Exception { anyString(), any(File.class), any(Namespace.class))).thenThrow(new IOException("upload failure")); - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); }); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(true); - updateDefaultSource(); + SourceConfig sourceConfig = createDefaultSourceConfig(); + this.mockedFunctionMetaData = + FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); + + try(InputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + resource.updateSource( + tenant, + namespace, + source, + inputStream, + mockedFormData, + null, + sourceConfig, + null, null); + } } catch (RestException re) { assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); throw re; @@ -1031,16 +930,9 @@ public void testUpdateSourceSuccess() throws Exception { public void testUpdateSourceWithUrl() throws Exception { Configurator.setRootLevel(Level.DEBUG); - String filePackageUrl = getPulsarIOCassandraNar().toURI().toString(); + String filePackageUrl = getPulsarIOTwitterNar().toURI().toString(); - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setTopicName(outputTopic); - sourceConfig.setSerdeClassName(outputSerdeClassName); - sourceConfig.setTenant(tenant); - sourceConfig.setNamespace(namespace); - sourceConfig.setName(source); - sourceConfig.setClassName(TWITTER_FIRE_HOSE); - sourceConfig.setParallelism(parallelism); + SourceConfig sourceConfig = createDefaultSourceConfig(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(true); mockStatic(ConnectorUtils.class, c -> { @@ -1048,15 +940,6 @@ public void testUpdateSourceWithUrl() throws Exception { mockStatic(ClassLoaderUtils.class, c -> { }); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.extractFileFromPkgURL(any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSourceType(argThat(clazz -> clazz.getName().equals(TWITTER_FIRE_HOSE)))) - .thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())) - .thenReturn(narClassLoader); - }); - this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 4306aaaeabcbd..193a9a6498e7b 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -526,7 +526,7 @@ MIT License * Checker Qual - checker-qual-3.33.0.jar * Annotations - - animal-sniffer-annotations-1.19.jar + - animal-sniffer-annotations-1.21.jar - annotations-4.1.1.4.jar CDDL - 1.0 diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java index d7d8d1423cb8d..1034d48a98502 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java @@ -469,37 +469,18 @@ public synchronized void setupFunctionWorkers(String suffix, FunctionRuntimeType } private void startFunctionWorkersWithProcessContainerFactory(String suffix, int numFunctionWorkers) { - String serviceUrl = "pulsar://pulsar-broker-0:" + PulsarContainer.BROKER_PORT; - String httpServiceUrl = "http://pulsar-broker-0:" + PulsarContainer.BROKER_HTTP_PORT; workerContainers.putAll(runNumContainers( "functions-worker-process-" + suffix, numFunctionWorkers, - (name) -> new WorkerContainer(clusterName, name) - .withNetwork(network) - .withNetworkAliases(name) - // worker settings - .withEnv("PF_workerId", name) - .withEnv("PF_workerHostname", name) - .withEnv("PF_workerPort", "" + PulsarContainer.BROKER_HTTP_PORT) - .withEnv("PF_pulsarFunctionsCluster", clusterName) - .withEnv("PF_pulsarServiceUrl", serviceUrl) - .withEnv("PF_pulsarWebServiceUrl", httpServiceUrl) - // script - .withEnv("clusterName", clusterName) - .withEnv("zookeeperServers", ZKContainer.NAME) - // bookkeeper tools - .withEnv("zkServers", ZKContainer.NAME) + (name) -> createWorkerContainer(name) )); this.startWorkers(); } - private void startFunctionWorkersWithThreadContainerFactory(String suffix, int numFunctionWorkers) { + private WorkerContainer createWorkerContainer(String name) { String serviceUrl = "pulsar://pulsar-broker-0:" + PulsarContainer.BROKER_PORT; String httpServiceUrl = "http://pulsar-broker-0:" + PulsarContainer.BROKER_HTTP_PORT; - workerContainers.putAll(runNumContainers( - "functions-worker-thread-" + suffix, - numFunctionWorkers, - (name) -> new WorkerContainer(clusterName, name) + return new WorkerContainer(clusterName, name) .withNetwork(network) .withNetworkAliases(name) // worker settings @@ -509,13 +490,21 @@ private void startFunctionWorkersWithThreadContainerFactory(String suffix, int n .withEnv("PF_pulsarFunctionsCluster", clusterName) .withEnv("PF_pulsarServiceUrl", serviceUrl) .withEnv("PF_pulsarWebServiceUrl", httpServiceUrl) - .withEnv("PF_functionRuntimeFactoryClassName", "org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory") - .withEnv("PF_functionRuntimeFactoryConfigs_threadGroupName", "pf-container-group") // script .withEnv("clusterName", clusterName) .withEnv("zookeeperServers", ZKContainer.NAME) // bookkeeper tools - .withEnv("zkServers", ZKContainer.NAME) + .withEnv("zkServers", ZKContainer.NAME); + } + + private void startFunctionWorkersWithThreadContainerFactory(String suffix, int numFunctionWorkers) { + workerContainers.putAll(runNumContainers( + "functions-worker-thread-" + suffix, + numFunctionWorkers, + (name) -> createWorkerContainer(name) + .withEnv("PF_functionRuntimeFactoryClassName", + "org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory") + .withEnv("PF_functionRuntimeFactoryConfigs_threadGroupName", "pf-container-group") )); this.startWorkers(); }