You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mb...@apache.org on 2021/03/26 11:11:21 UTC

[asterixdb] 04/15: [NO ISSUE] UDF API improvements

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

mblow pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit 2fbcab0b8bc6e8e47602613212e1a19dcb01c671
Author: Ian Maxon <ia...@maxons.email>
AuthorDate: Tue Mar 16 19:32:42 2021 -0700

    [NO ISSUE] UDF API improvements
    
    - user model changes: yes
    - storage format changes: yes
    - interface changes: yes
    
    Details:
    
    - Make a GET on / return all installed libraries on that node
    - Make GET return JSON wrapped errors
    - Roll all modification functionality into POST
    - Make dataverse, library name, and library language
      specified as multipart form data fields
      (dataverse, name, and type)
    - Make library data a named field in the multipart post
      data body (data)
    - Allow multipart dataverse names
    - Add support for multipart/form-data parameters to TestExecutor
    - Fix resource leaks
    - Require either python.cmd to be set or python.cmd.autolocate to
      be true to automatically attempt to locate python interpreter
    
    Change-Id: Ib6e6ce7debc9c2e07d24163542c1f98886792165
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/9565
    Reviewed-by: Ian Maxon <im...@uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
---
 .../api/http/server/AbstractNCUdfServlet.java      | 206 ++++++++++++++++++---
 .../asterix/api/http/server/BasicAuthServlet.java  |  13 +-
 .../asterix/api/http/server/NCUdfApiServlet.java   | 193 ++++++++++---------
 .../api/http/server/NCUdfRecoveryServlet.java      |   5 +-
 .../asterix/hyracks/bootstrap/NCApplication.java   |   6 +-
 .../asterix/app/external/ExternalUDFLibrarian.java |  46 +++--
 .../app/external/IExternalUDFLibrarian.java        |   7 +-
 .../apache/asterix/test/common/TestExecutor.java   | 145 +++++++++++++--
 asterixdb/asterix-app/src/test/resources/cc.conf   |   1 +
 .../bad-ext-function-ddl-1.2.lib.sqlpp             |   2 +-
 .../create-or-replace-function-1.2.lib.sqlpp       |   2 +-
 .../deterministic/deterministic.1.lib.sqlpp        |   2 +-
 .../exception_create_system_library.1.lib.sqlpp    |   2 +-
 .../getCapital/getCapital.1.lib.sqlpp              |   2 +-
 .../getCapital_open/getCapital_open.1.lib.sqlpp    |   2 +-
 .../library_list_api_multipart.1.post.http}        |   8 +-
 .../library_list_api_multipart.2.post.http}        |   9 +-
 .../library_list_api_multipart.3.post.http}        |   7 +-
 .../library_list_api_multipart.4.post.http}        |   8 +-
 .../library_list_api_multipart.5.post.http}        |   7 +-
 .../keyword_detector/keyword_detector.1.lib.sqlpp  |   2 +-
 .../library_list_api.0.ddl.sqlpp}                  |   4 +-
 .../library_list_api.1.post.http}                  |   8 +-
 .../library_list_api.2.get.http}                   |   4 +-
 .../library_list_api_multipart.0.ddl.sqlpp}        |   9 +-
 .../library_list_api_multipart.1.post.http}        |   9 +-
 .../library_list_api_multipart.2.post.http}        |   8 +-
 .../library_list_api_multipart.3.post.http}        |  10 +-
 .../library_list_api_multipart.4.post.http}        |  10 +-
 .../library_list_api_multipart.5.get.http}         |   4 +-
 .../my_array_sum/my_array_sum.1.lib.sqlpp          |   2 +-
 .../mysentiment/mysentiment.1.lib.sqlpp            |   2 +-
 .../mysentiment_multipart.0.ddl.sqlpp}             |   4 +-
 .../mysentiment_multipart.1.lib.sqlpp}             |   3 +-
 .../mysentiment_multipart.2.ddl.sqlpp}             |   5 +-
 .../mysentiment_multipart.3.query.sqlpp}           |   4 +-
 .../mysentiment_multipart.4.query.sqlpp}           |   4 +-
 .../mysentiment_multipart.5.query.sqlpp}           |   4 +-
 .../mysentiment_multipart.6.ddl.sqlpp}             |   3 +-
 .../external-library/mysum/mysum.1.lib.sqlpp       |   2 +-
 .../mysum_bad_credential.1.lib.sqlpp               |   2 +-
 .../mysum_bad_credential.2.lib.sqlpp               |   2 +-
 .../mysum_bad_credential.3.lib.sqlpp               |   2 +-
 .../mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp    |   2 +-
 .../py_function_error.1.lib.sqlpp                  |   2 +-
 .../py_nested_access/py_nested_access.1.lib.sqlpp  |   2 +-
 .../python-fn-escape/python-fn-escape.1.lib.sqlpp  |   2 +-
 .../return_invalid_type.1.lib.sqlpp                |   2 +-
 .../type_validation/type_validation.1.lib.sqlpp    |   2 +-
 .../udf_metadata/udf_metadata.1.lib.sqlpp          |   4 +-
 .../upperCase/upperCase.1.lib.sqlpp                |   2 +-
 .../exception_create_system_adapter.1.lib.sqlpp    |   2 +-
 ...feed-with-external-adapter-cross-dv.1.lib.sqlpp |   2 +-
 .../feed-with-external-adapter.1.lib.sqlpp         |   2 +-
 .../feed-with-external-function.1.lib.sqlpp        |   2 +-
 .../invalid-library-params.1.regexjson             |   3 +
 .../library_list_api/library_list_ap1.1.regexjson  |   1 +
 .../library_list_api/library_list_ap1.2.regexjson  |   5 +
 .../library_list_ap1.1.regexjson                   |   1 +
 .../library_list_ap1.2.regexjson                   |   1 +
 .../library_list_ap1.3.regexjson                   |   1 +
 .../library_list_ap1.4.regexjson                   |   1 +
 .../library_list_ap1.5.regexjson                   |  20 ++
 .../resources/runtimets/testsuite_it_python.xml    |   5 +
 .../resources/runtimets/testsuite_it_sqlpp.xml     |  20 ++
 .../asterix/common/library/ILibraryManager.java    |   6 +
 .../asterix/common/library/LibraryDescriptor.java  |   8 +-
 .../src/main/resources/asx_errormsg/en.properties  |   2 +-
 .../external/library/ExternalLibraryManager.java   |  76 +++++++-
 .../ExternalScalarPythonFunctionEvaluator.java     |  13 +-
 .../LibraryDeployPrepareOperatorDescriptor.java    |   8 -
 .../external/util/ExternalLibraryUtils.java        |  18 +-
 .../src/main/resources/Catalog.xsd                 |   2 +
 .../control/common/controllers/NCConfig.java       |   5 +-
 74 files changed, 785 insertions(+), 217 deletions(-)

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
index 54e972e..a234b9f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
@@ -22,40 +22,46 @@ import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNEC
 import static org.apache.asterix.common.functions.ExternalFunctionLanguage.JAVA;
 import static org.apache.asterix.common.functions.ExternalFunctionLanguage.PYTHON;
 
+import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.nio.file.Files;
 import java.nio.file.OpenOption;
 import java.nio.file.Path;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
-import org.apache.asterix.common.library.LibraryDescriptor;
 import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.IFormattedException;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
 import org.apache.hyracks.http.server.utils.HttpUtil;
 
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpScheme;
+import io.netty.handler.codec.http.multipart.FileUpload;
+import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
+import io.netty.handler.codec.http.multipart.InterfaceHttpData;
+import io.netty.handler.codec.http.multipart.MixedAttribute;
 
 public abstract class AbstractNCUdfServlet extends AbstractServlet {
 
+    private final IParserFactory parserFactory;
     INcApplicationContext appCtx;
     INCServiceContext srvCtx;
 
@@ -63,13 +69,81 @@ public abstract class AbstractNCUdfServlet extends AbstractServlet {
     private final HttpScheme httpServerProtocol;
     private final int httpServerPort;
 
+    public static final String GET_UDF_DIST_ENDPOINT = "/dist";
+    public static final String DATAVERSE_PARAMETER = "dataverse";
+    public static final String NAME_PARAMETER = "name";
+    public static final String TYPE_PARAMETER = "type";
+    public static final String DELETE_PARAMETER = "delete";
+    public static final String IFEXISTS_PARAMETER = "ifexists";
+    public static final String DATA_PARAMETER = "data";
+
+    protected enum LibraryOperation {
+        UPSERT,
+        DELETE
+    }
+
+    protected final static class LibraryUploadData {
+
+        final LibraryOperation op;
+        final DataverseName dataverse;
+        final String name;
+        final ExternalFunctionLanguage type;
+        final boolean replaceIfExists;
+        final FileUpload fileUpload;
+
+        private LibraryUploadData(LibraryOperation op, List<InterfaceHttpData> dataverse, MixedAttribute name,
+                MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload) throws IOException {
+            this.op = op;
+            List<String> dataverseParts = new ArrayList<>(dataverse.size());
+            for (InterfaceHttpData attr : dataverse) {
+                dataverseParts.add(((MixedAttribute) attr).getValue());
+            }
+            this.dataverse = DataverseName.create(dataverseParts);
+            this.name = name.getValue();
+            this.type = type != null ? getLanguageByTypeParameter(type.getValue()) : null;
+            this.replaceIfExists = replaceIfExists;
+            this.fileUpload = (FileUpload) fileUpload;
+        }
+
+        private LibraryUploadData(LibraryOperation op, DataverseName dataverse, MixedAttribute name,
+                MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload) throws IOException {
+            this.op = op;
+            this.dataverse = dataverse;
+            this.name = name.getValue();
+            this.type = type != null ? getLanguageByTypeParameter(type.getValue()) : null;
+            this.replaceIfExists = replaceIfExists;
+            this.fileUpload = (FileUpload) fileUpload;
+        }
+
+        public static LibraryUploadData libraryCreationUploadData(List<InterfaceHttpData> dataverse,
+                MixedAttribute name, MixedAttribute type, InterfaceHttpData fileUpload) throws IOException {
+            return new LibraryUploadData(LibraryOperation.UPSERT, dataverse, name, type, true, fileUpload);
+        }
+
+        public static LibraryUploadData libraryDeletionUploadData(List<InterfaceHttpData> dataverse,
+                MixedAttribute name, boolean replaceIfExists) throws IOException {
+            return new LibraryUploadData(LibraryOperation.DELETE, dataverse, name, null, replaceIfExists, null);
+        }
+
+        public static LibraryUploadData libraryCreationUploadData(DataverseName dataverse, MixedAttribute name,
+                MixedAttribute type, InterfaceHttpData fileUpload) throws IOException {
+            return new LibraryUploadData(LibraryOperation.UPSERT, dataverse, name, type, true, fileUpload);
+        }
+
+        public static LibraryUploadData libraryDeletionUploadData(DataverseName dataverse, MixedAttribute name,
+                boolean replaceIfExists) throws IOException {
+            return new LibraryUploadData(LibraryOperation.DELETE, dataverse, name, null, replaceIfExists, null);
+        }
+    }
+
     public AbstractNCUdfServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            HttpScheme httpServerProtocol, int httpServerPort) {
+            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
 
         super(ctx, paths);
         this.plainAppCtx = appCtx;
         this.httpServerProtocol = httpServerProtocol;
         this.httpServerPort = httpServerPort;
+        this.parserFactory = compilationProvider.getParserFactory();
     }
 
     void readFromFile(Path filePath, IServletResponse response, String contentType, OpenOption opt) throws Exception {
@@ -103,7 +177,7 @@ public abstract class AbstractNCUdfServlet extends AbstractServlet {
     }
 
     URI createDownloadURI(Path file) throws Exception {
-        String path = paths[0].substring(0, trims[0]) + '/' + file.getFileName();
+        String path = paths[0].substring(0, trims[0]) + GET_UDF_DIST_ENDPOINT + '/' + file.getFileName();
         String host = getHyracksClientConnection().getHost();
         return new URI(httpServerProtocol.toString(), null, host, httpServerPort, path, null, null);
     }
@@ -116,25 +190,105 @@ public abstract class AbstractNCUdfServlet extends AbstractServlet {
         return hcc;
     }
 
-    Pair<DataverseName, String> parseLibraryName(IServletRequest request) throws IllegalArgumentException {
-        String[] path = StringUtils.split(localPath(request), '/');
-        int ln = path.length;
-        if (ln < 2) {
-            return null;
+    protected String getDisplayFormDataverseParameter() {
+        return null;
+    }
+
+    protected String getDataverseParameter() {
+        return DATAVERSE_PARAMETER;
+    }
+
+    private boolean isNotAttribute(InterfaceHttpData field) {
+        return field == null || !field.getHttpDataType().equals(InterfaceHttpData.HttpDataType.Attribute);
+    }
+
+    private boolean areNotAttributes(List<InterfaceHttpData> fields) {
+        return fields == null || fields.stream().map(InterfaceHttpData::getHttpDataType)
+                .anyMatch(httpDataType -> !httpDataType.equals(InterfaceHttpData.HttpDataType.Attribute));
+    }
+
+    protected LibraryUploadData decodeMultiPartLibraryOptions(HttpPostRequestDecoder requestDecoder)
+            throws IOException, CompilationException {
+        List<InterfaceHttpData> dataverseAttributeParts = requestDecoder.getBodyHttpDatas(DATAVERSE_PARAMETER);
+        InterfaceHttpData displayFormDataverseAttribute = null;
+        if (getDisplayFormDataverseParameter() != null) {
+            displayFormDataverseAttribute = requestDecoder.getBodyHttpData(getDisplayFormDataverseParameter());
+        }
+        if (displayFormDataverseAttribute != null && dataverseAttributeParts != null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME,
+                    getDisplayFormDataverseParameter(), getDataverseParameter());
+        }
+        InterfaceHttpData nameAtrribute = requestDecoder.getBodyHttpData(NAME_PARAMETER);
+        InterfaceHttpData typeAttribute = requestDecoder.getBodyHttpData(TYPE_PARAMETER);
+        InterfaceHttpData deleteAttribute = requestDecoder.getBodyHttpData(DELETE_PARAMETER);
+        InterfaceHttpData replaceIfExistsAttribute = requestDecoder.getBodyHttpData(IFEXISTS_PARAMETER);
+        if ((isNotAttribute(displayFormDataverseAttribute)) && (areNotAttributes(dataverseAttributeParts))) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, getDataverseParameter());
+        } else if (isNotAttribute(nameAtrribute)) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, NAME_PARAMETER);
+        } else if ((typeAttribute == null && deleteAttribute == null)) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED,
+                    TYPE_PARAMETER + " or " + DELETE_PARAMETER);
+        } else if (typeAttribute != null && deleteAttribute != null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, TYPE_PARAMETER,
+                    DELETE_PARAMETER);
+        }
+
+        if (!isNotAttribute(deleteAttribute)) {
+            boolean replace = false;
+            if (replaceIfExistsAttribute != null) {
+                replace = Boolean.TRUE.toString()
+                        .equalsIgnoreCase(((MixedAttribute) replaceIfExistsAttribute).getValue());
+            }
+            if (displayFormDataverseAttribute == null) {
+                return LibraryUploadData.libraryDeletionUploadData(dataverseAttributeParts,
+                        (MixedAttribute) nameAtrribute, replace);
+            } else {
+                DataverseName dataverseName = DataverseName
+                        .create(parserFactory.createParser(((MixedAttribute) displayFormDataverseAttribute).getValue())
+                                .parseMultipartIdentifier());
+                return LibraryUploadData.libraryDeletionUploadData(dataverseName, (MixedAttribute) nameAtrribute,
+                        replace);
+            }
+        } else if (!isNotAttribute(typeAttribute)) {
+            InterfaceHttpData libraryData = requestDecoder.getBodyHttpData(DATA_PARAMETER);
+            if (libraryData == null) {
+                throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DATA_PARAMETER);
+            } else if (!libraryData.getHttpDataType().equals(InterfaceHttpData.HttpDataType.FileUpload)) {
+                throw RuntimeDataException.create(ErrorCode.INVALID_REQ_PARAM_VAL, DATA_PARAMETER,
+                        libraryData.getHttpDataType());
+            }
+            LibraryUploadData uploadData;
+            if (displayFormDataverseAttribute == null) {
+                uploadData = LibraryUploadData.libraryCreationUploadData(dataverseAttributeParts,
+                        (MixedAttribute) nameAtrribute, (MixedAttribute) typeAttribute, libraryData);
+            } else {
+                DataverseName dataverseName = DataverseName
+                        .create(parserFactory.createParser(((MixedAttribute) displayFormDataverseAttribute).getValue())
+                                .parseMultipartIdentifier());
+                uploadData = LibraryUploadData.libraryCreationUploadData(dataverseName, (MixedAttribute) nameAtrribute,
+                        (MixedAttribute) typeAttribute, libraryData);
+            }
+            if (uploadData.type == null) {
+                throw RuntimeDataException.create(ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_KIND,
+                        ((MixedAttribute) typeAttribute).getValue());
+            }
+            return uploadData;
+        } else {
+            if (!typeAttribute.getHttpDataType().equals(InterfaceHttpData.HttpDataType.Attribute)) {
+                throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_PARAMETER);
+            }
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DELETE_PARAMETER);
         }
-        String libraryName = path[ln - 1];
-        DataverseName dataverseName = DataverseName.create(Arrays.asList(path), 0, ln - 1);
-        return new Pair<>(dataverseName, libraryName);
     }
 
-    static ExternalFunctionLanguage getLanguageByFileExtension(String fileExtension) {
-        switch (fileExtension) {
-            case LibraryDescriptor.FILE_EXT_ZIP:
-                return JAVA;
-            case LibraryDescriptor.FILE_EXT_PYZ:
-                return PYTHON;
-            default:
-                return null;
+    static ExternalFunctionLanguage getLanguageByTypeParameter(String lang) {
+        if (lang.equalsIgnoreCase(JAVA.name())) {
+            return JAVA;
+        } else if (lang.equalsIgnoreCase(PYTHON.name())) {
+            return PYTHON;
+        } else {
+            return null;
         }
     }
 
@@ -142,9 +296,15 @@ public abstract class AbstractNCUdfServlet extends AbstractServlet {
         if (IFormattedException.matchesAny(e, ErrorCode.UNKNOWN_DATAVERSE, ErrorCode.UNKNOWN_LIBRARY)) {
             return HttpResponseStatus.NOT_FOUND;
         }
+        if (IFormattedException.matchesAny(e, ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNKNOWN_KIND,
+                ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_KIND, ErrorCode.INVALID_REQ_PARAM_VAL,
+                ErrorCode.PARAMETERS_REQUIRED)) {
+            return HttpResponseStatus.BAD_REQUEST;
+        }
         if (e instanceof AlgebricksException) {
             return HttpResponseStatus.BAD_REQUEST;
         }
         return HttpResponseStatus.INTERNAL_SERVER_ERROR;
     }
+
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
index e062cdc..9dc971a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
@@ -130,7 +130,7 @@ public class BasicAuthServlet implements IServlet {
             return false;
         }
         String providedUsername = providedCredentials[0];
-        String storedPw = getStoredCredentials(request).get(providedUsername);
+        String storedPw = getCredential(providedUsername, request);
         if (storedPw == null) {
             LOGGER.debug("Invalid username");
             return false;
@@ -144,8 +144,15 @@ public class BasicAuthServlet implements IServlet {
         }
     }
 
-    protected Map<String, String> getStoredCredentials(IServletRequest request) {
-        return request.getHttpRequest().method().equals(HttpMethod.GET) ? ephemeralCredentials : storedCredentials;
+    private String getCredential(String username, IServletRequest request) {
+        String credential = storedCredentials.get(username);
+        if (credential != null) {
+            return credential;
+        } else if (request != null && request.getHttpRequest().method().equals(HttpMethod.GET)) {
+            return ephemeralCredentials.get(username);
+        } else {
+            return null;
+        }
     }
 
     public static String hashPassword(String password) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
index 0645870..f164938 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.api.http.server.ServletConstants.SYS_AUTH_HEADER;
+import static org.apache.asterix.common.library.LibraryDescriptor.FIELD_HASH;
 
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -31,7 +32,10 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.security.DigestOutputStream;
 import java.security.MessageDigest;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
@@ -44,6 +48,7 @@ import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.api.IReceptionist;
 import org.apache.asterix.common.api.IRequestReference;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
+import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
@@ -53,21 +58,22 @@ import org.apache.asterix.external.util.ExternalLibraryUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.io.IOUtils;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.utils.HttpUtil;
+import org.apache.hyracks.util.JSONUtil;
 import org.apache.hyracks.util.file.FileUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
 import io.netty.buffer.ByteBufInputStream;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpScheme;
-import io.netty.handler.codec.http.multipart.FileUpload;
 import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
-import io.netty.handler.codec.http.multipart.InterfaceHttpData;
 
 public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
@@ -76,12 +82,13 @@ public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
     protected Path workingDir;
     protected String sysAuthHeader;
+    private ILibraryManager libraryManager;
 
     private static final Logger LOGGER = LogManager.getLogger();
 
     public NCUdfApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
             ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
+        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
         this.compilationProvider = compilationProvider;
         this.receptionist = appCtx.getReceptionist();
     }
@@ -89,6 +96,7 @@ public class NCUdfApiServlet extends AbstractNCUdfServlet {
     @Override
     public void init() throws IOException {
         appCtx = (INcApplicationContext) plainAppCtx;
+        this.libraryManager = appCtx.getLibraryManager();
         srvCtx = this.appCtx.getServiceContext();
         workingDir = Paths.get(appCtx.getLibraryManager().getDistributionDir().getAbsolutePath()).normalize();
         initAuth();
@@ -119,27 +127,26 @@ public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
     private void doCreate(DataverseName dataverseName, String libraryName, ExternalFunctionLanguage language,
             String hash, URI downloadURI, boolean replaceIfExists, String sysAuthHeader,
-            IRequestReference requestReference, IServletRequest request, IServletResponse response) throws Exception {
+            IRequestReference requestReference, IServletRequest request) throws Exception {
         INCMessageBroker ncMb = (INCMessageBroker) srvCtx.getMessageBroker();
         MessageFuture responseFuture = ncMb.registerMessageFuture();
         CreateLibraryRequestMessage req = new CreateLibraryRequestMessage(srvCtx.getNodeId(),
                 responseFuture.getFutureId(), dataverseName, libraryName, language, hash, downloadURI, replaceIfExists,
                 sysAuthHeader, requestReference, additionalHttpHeadersFromRequest(request));
-        sendMessage(req, responseFuture, requestReference, request, response);
+        sendMessage(req, responseFuture);
     }
 
     private void doDrop(DataverseName dataverseName, String libraryName, boolean replaceIfExists,
-            IRequestReference requestReference, IServletRequest request, IServletResponse response) throws Exception {
+            IRequestReference requestReference, IServletRequest request) throws Exception {
         INCMessageBroker ncMb = (INCMessageBroker) srvCtx.getMessageBroker();
         MessageFuture responseFuture = ncMb.registerMessageFuture();
         DropLibraryRequestMessage req =
                 new DropLibraryRequestMessage(srvCtx.getNodeId(), responseFuture.getFutureId(), dataverseName,
                         libraryName, replaceIfExists, requestReference, additionalHttpHeadersFromRequest(request));
-        sendMessage(req, responseFuture, requestReference, request, response);
+        sendMessage(req, responseFuture);
     }
 
-    private void sendMessage(ICcAddressedMessage requestMessage, MessageFuture responseFuture,
-            IRequestReference requestReference, IServletRequest request, IServletResponse response) throws Exception {
+    private void sendMessage(ICcAddressedMessage requestMessage, MessageFuture responseFuture) throws Exception {
         // Running on NC -> send 'execute' message to CC
         INCMessageBroker ncMb = (INCMessageBroker) srvCtx.getMessageBroker();
         InternalRequestResponse responseMsg;
@@ -165,107 +172,125 @@ public class NCUdfApiServlet extends AbstractNCUdfServlet {
     @Override
     protected void get(IServletRequest request, IServletResponse response) throws Exception {
         String localPath = localPath(request);
-        while (localPath.startsWith("/")) {
-            localPath = localPath.substring(1);
-        }
-        if (localPath.isEmpty()) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
-        Path filePath = workingDir.resolve(localPath).normalize();
-        if (!filePath.startsWith(workingDir)) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
+        try {
+            if (localPath.equals("/") || localPath.equals("")) {
+                //TODO: nicer way to get this into display form?
+                Map<DataverseName, Map<String, String>> dvToLibHashes =
+                        ExternalLibraryUtils.produceLibraryListing(libraryManager);
+                List<Map<String, Object>> libraryList = new ArrayList<>();
+                for (Map.Entry<DataverseName, Map<String, String>> dvAndLibs : dvToLibHashes.entrySet()) {
+                    for (Map.Entry<String, String> libsInDv : dvAndLibs.getValue().entrySet()) {
+                        Map<String, Object> libraryEntry = new HashMap<>();
+                        List<String> dvParts = dvAndLibs.getKey().getParts();
+                        String dvKey = getDisplayFormDataverseParameter() == null ? getDataverseParameter()
+                                : getDisplayFormDataverseParameter();
+                        libraryEntry.put(dvKey, dvParts.size() > 1 ? dvParts : dvAndLibs.getKey().toString());
+                        libraryEntry.put(NAME_PARAMETER, libsInDv.getKey());
+                        libraryEntry.put(FIELD_HASH, libsInDv.getValue());
+                        libraryList.add(libraryEntry);
+                    }
+                }
+                JsonNode libraryListing = OBJECT_MAPPER.valueToTree(libraryList);
+                response.setStatus(HttpResponseStatus.OK);
+                HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
+                PrintWriter responseWriter = response.writer();
+                JSONUtil.writeNode(responseWriter, libraryListing);
+                responseWriter.flush();
+            } else if (localPath(request).startsWith(GET_UDF_DIST_ENDPOINT)) {
+                localPath = localPath(request).substring(GET_UDF_DIST_ENDPOINT.length());
+                while (localPath.startsWith("/")) {
+                    localPath = localPath.substring(1);
+                }
+                if (localPath.isEmpty()) {
+                    response.setStatus(HttpResponseStatus.BAD_REQUEST);
+                    return;
+                }
+                Path filePath = workingDir.resolve(localPath).normalize();
+                if (!filePath.startsWith(workingDir)) {
+                    response.setStatus(HttpResponseStatus.BAD_REQUEST);
+                    return;
+                }
+                readFromFile(filePath, response, HttpUtil.ContentType.APPLICATION_OCTET_STREAM, null);
+            } else {
+                response.setStatus(HttpResponseStatus.NOT_FOUND);
+            }
+        } catch (Exception e) {
+            response.setStatus(toHttpErrorStatus(e));
+            PrintWriter responseWriter = response.writer();
+            Map<String, String> error = Collections.singletonMap("error", e.getMessage());
+            String errorJson = "";
+            try {
+                errorJson = OBJECT_MAPPER.writeValueAsString(error);
+            } catch (JsonProcessingException ex) {
+                responseWriter.write("{ \"error\": \"Unable to process error message!\" }");
+            }
+            responseWriter.write(errorJson);
+            responseWriter.flush();
+            LOGGER.error("Error reading library", e);
         }
-        readFromFile(filePath, response, HttpUtil.ContentType.APPLICATION_OCTET_STREAM, null);
     }
 
     @Override
     protected void post(IServletRequest request, IServletResponse response) {
         HttpRequest httpRequest = request.getHttpRequest();
-        Pair<DataverseName, String> libraryName = parseLibraryName(request);
-        if (libraryName == null) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
         Path libraryTempFile = null;
         FileOutputStream libTmpOut = null;
         HttpPostRequestDecoder requestDecoder = new HttpPostRequestDecoder(httpRequest);
         try {
-            if (!requestDecoder.hasNext() || requestDecoder.getBodyHttpDatas().size() != 1) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
-                return;
-            }
-            InterfaceHttpData httpData = requestDecoder.getBodyHttpDatas().get(0);
-            if (!httpData.getHttpDataType().equals(InterfaceHttpData.HttpDataType.FileUpload)) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
-                return;
-            }
-            FileUpload fileUpload = (FileUpload) httpData;
-            String fileExt = FilenameUtils.getExtension(fileUpload.getFilename());
-            ExternalFunctionLanguage language = getLanguageByFileExtension(fileExt);
-            if (language == null) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
-                return;
-            }
-            try {
-                IRequestReference requestReference = receptionist.welcome(request);
+            LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
+            IRequestReference requestReference = receptionist.welcome(request);
+            if (uploadData.op == LibraryOperation.UPSERT) {
+                ExternalFunctionLanguage language = uploadData.type;
+                String fileExt = FilenameUtils.getExtension(uploadData.fileUpload.getFilename());
                 libraryTempFile = Files.createTempFile(workingDir, "lib_", '.' + fileExt);
                 if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library " + libraryName.first + "."
-                            + libraryName.second);
+                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library " + uploadData.dataverse
+                            + "." + uploadData.name);
                 }
                 MessageDigest digest = MessageDigest.getInstance("MD5");
                 libTmpOut = new FileOutputStream(libraryTempFile.toFile());
-                OutputStream outStream = new DigestOutputStream(libTmpOut, digest);
-                InputStream uploadInput = new ByteBufInputStream(((FileUpload) httpData).getByteBuf());
-                IOUtils.copyLarge(uploadInput, outStream);
-                outStream.close();
+                try (OutputStream os = new DigestOutputStream(libTmpOut, digest);
+                        InputStream ui = new ByteBufInputStream((uploadData.fileUpload).getByteBuf())) {
+                    IOUtils.copyLarge(ui, os);
+                }
                 URI downloadURI = createDownloadURI(libraryTempFile);
-                doCreate(libraryName.first, libraryName.second, language,
+                doCreate(uploadData.dataverse, uploadData.name, language,
                         ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
-                        requestReference, request, response);
-                response.setStatus(HttpResponseStatus.OK);
-            } catch (Exception e) {
-                response.setStatus(toHttpErrorStatus(e));
-                PrintWriter responseWriter = response.writer();
-                responseWriter.write(e.getMessage());
-                responseWriter.flush();
-                LOGGER.error("Error creating/updating library " + libraryName.first + "." + libraryName.second, e);
+                        requestReference, request);
+            } else if (uploadData.op == LibraryOperation.DELETE) {
+                doDrop(uploadData.dataverse, uploadData.name, uploadData.replaceIfExists, requestReference, request);
             }
+            response.setStatus(HttpResponseStatus.OK);
+            PrintWriter responseWriter = response.writer();
+            String emptyJson = "{}";
+            responseWriter.write(emptyJson);
+            responseWriter.flush();
+        } catch (Exception e) {
+            response.setStatus(toHttpErrorStatus(e));
+            PrintWriter responseWriter = response.writer();
+            Map<String, String> error = Collections.singletonMap("error", e.getMessage());
+            String errorJson = "";
+            try {
+                errorJson = OBJECT_MAPPER.writeValueAsString(error);
+            } catch (JsonProcessingException ex) {
+                responseWriter.write("{ \"error\": \"Unable to process error message!\" }");
+            }
+            responseWriter.write(errorJson);
+            responseWriter.flush();
+            LOGGER.error("Error modifying library", e);
         } finally {
             requestDecoder.destroy();
-            if (libraryTempFile != null) {
-                try {
+            try {
+                if (libraryTempFile != null) {
                     if (libTmpOut != null) {
                         libTmpOut.close();
                     }
                     Files.deleteIfExists(libraryTempFile);
-                } catch (IOException e) {
-                    LOGGER.warn("Could not delete temporary file " + libraryTempFile, e);
                 }
+            } catch (IOException e) {
+                LOGGER.warn("Could not delete temporary file " + libraryTempFile, e);
             }
         }
     }
 
-    @Override
-    protected void delete(IServletRequest request, IServletResponse response) {
-        Pair<DataverseName, String> libraryName = parseLibraryName(request);
-        if (libraryName == null) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
-        try {
-            IRequestReference requestReference = receptionist.welcome(request);
-            doDrop(libraryName.first, libraryName.second, false, requestReference, request, response);
-            response.setStatus(HttpResponseStatus.OK);
-        } catch (Exception e) {
-            response.setStatus(toHttpErrorStatus(e));
-            PrintWriter responseWriter = response.writer();
-            responseWriter.write(e.getMessage());
-            responseWriter.flush();
-            LOGGER.error("Error deleting library " + libraryName.first + "." + libraryName.second, e);
-        }
-    }
-
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
index 2c29d14..1563833 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
@@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
@@ -38,8 +39,8 @@ public class NCUdfRecoveryServlet extends AbstractNCUdfServlet {
     public static final String GET_ALL_UDF_ENDPOINT = "/all";
 
     public NCUdfRecoveryServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
+            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index b0cb4de..c27b3b1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -228,9 +228,9 @@ public class NCApplication extends BaseNCApplication {
                 new NCUdfApiServlet(apiServer.ctx(), new String[] { UDF }, getApplicationContext(),
                         sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
                 auth.getFirst(), auth.getSecond()));
-        apiServer.addServlet(new BasicAuthServlet(
-                apiServer.ctx(), new NCUdfRecoveryServlet(apiServer.ctx(), new String[] { UDF_RECOVERY },
-                        getApplicationContext(), apiServer.getScheme(), apiServer.getAddress().getPort()),
+        apiServer.addServlet(new BasicAuthServlet(apiServer.ctx(),
+                new NCUdfRecoveryServlet(apiServer.ctx(), new String[] { UDF_RECOVERY }, getApplicationContext(),
+                        sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
                 auth.getFirst(), auth.getSecond()));
         apiServer.addServlet(new QueryStatusApiServlet(apiServer.ctx(), getApplicationContext(), QUERY_STATUS));
         apiServer.addServlet(new QueryResultApiServlet(apiServer.ctx(), getApplicationContext(), QUERY_RESULT));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
index 81e35a0..88277e4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
@@ -23,8 +23,7 @@ import java.io.IOException;
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.commons.io.IOUtils;
-import org.apache.http.HttpEntity;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
@@ -32,7 +31,6 @@ import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.AuthCache;
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpDelete;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.entity.ContentType;
@@ -44,32 +42,58 @@ import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 @SuppressWarnings("squid:S134")
 public class ExternalUDFLibrarian implements IExternalUDFLibrarian {
 
     private HttpClient hc;
 
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
     public ExternalUDFLibrarian() {
         hc = new DefaultHttpClient();
     }
 
     @Override
-    public void install(URI path, String libPath, Pair<String, String> credentials) throws Exception {
+    public void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            String type, String libPath, Pair<String, String> credentials) throws Exception {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
         HttpPost post = new HttpPost(path);
         File lib = new File(libPath);
-        HttpEntity file = MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT)
-                .addBinaryBody("lib", lib, ContentType.DEFAULT_BINARY, lib.getName()).build();
-        post.setEntity(file);
+        MultipartEntityBuilder entity = MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT);
+        if (!useDisplayForm) {
+            for (String dvPart : dataverse.getParts()) {
+                entity.addTextBody(dataverseKey, dvPart);
+            }
+        } else {
+            entity.addTextBody(dataverseKey, dataverse.toString());
+        }
+        entity.addTextBody("name", name);
+        entity.addTextBody("type", type);
+        entity.addBinaryBody("data", lib, ContentType.DEFAULT_BINARY, lib.getName()).build();
+        post.setEntity(entity.build());
         HttpResponse response = hc.execute(post, hcCtx);
         handleResponse(response);
     }
 
     @Override
-    public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
+    public void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            Pair<String, String> credentials) throws IOException, AsterixException {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
-        HttpDelete del = new HttpDelete(path);
-        HttpResponse response = hc.execute(del, hcCtx);
+        HttpPost post = new HttpPost(path);
+        MultipartEntityBuilder entity = MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT);
+        if (!useDisplayForm) {
+            for (String dvPart : dataverse.getParts()) {
+                entity.addTextBody(dataverseKey, dvPart);
+            }
+        } else {
+            entity.addTextBody(dataverseKey, dataverse.toString());
+        }
+        entity.addTextBody("name", name);
+        entity.addTextBody("delete", "true");
+        post.setEntity(entity.build());
+        HttpResponse response = hc.execute(post, hcCtx);
         handleResponse(response);
     }
 
@@ -89,7 +113,7 @@ public class ExternalUDFLibrarian implements IExternalUDFLibrarian {
         String resp = null;
         int respCode = response.getStatusLine().getStatusCode();
         if (respCode == 500 || respCode == 400) {
-            resp = IOUtils.toString(response.getEntity().getContent());
+            resp = OBJECT_MAPPER.readTree(response.getEntity().getContent()).get("error").asText();
         }
         response.getEntity().consumeContent();
         if (resp == null && respCode != 200) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
index 2315bec..639475b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
@@ -22,11 +22,14 @@ import java.io.IOException;
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
 public interface IExternalUDFLibrarian {
 
-    void install(URI path, String libPath, Pair<String, String> credentials) throws Exception;
+    void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            String type, String libPath, Pair<String, String> credentials) throws Exception;
 
-    void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException;
+    void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            Pair<String, String> credentials) throws IOException, AsterixException;
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 3cc6353..730ed46 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -82,6 +82,8 @@ import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.utils.Servlets;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
 import org.apache.asterix.lang.sqlpp.util.SqlppStatementUtil;
 import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.utils.MetadataConstants;
@@ -104,14 +106,25 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.HttpStatus;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.AuthCache;
+import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpUriRequest;
 import org.apache.http.client.methods.RequestBuilder;
+import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.entity.ContentType;
 import org.apache.http.entity.StringEntity;
+import org.apache.http.entity.mime.HttpMultipartMode;
+import org.apache.http.entity.mime.MultipartEntityBuilder;
+import org.apache.http.impl.auth.BasicScheme;
+import org.apache.http.impl.client.BasicAuthCache;
+import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
 import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
@@ -173,6 +186,8 @@ public class TestExecutor {
     private static final Pattern VARIABLE_REF_PATTERN = Pattern.compile("\\$(\\w+)");
     private static final Pattern HTTP_PARAM_PATTERN =
             Pattern.compile("param (?<name>[\\w-$]+)(?::(?<type>\\w+))?=(?<value>.*)", Pattern.MULTILINE);
+    private static final Pattern HTTP_AUTH_PATTERN =
+            Pattern.compile("auth (?<username>.*):(?<password>.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_BODY_PATTERN = Pattern.compile("body=(.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_STATUSCODE_PATTERN = Pattern.compile("statuscode (.*)", Pattern.MULTILINE);
     private static final Pattern MAX_RESULT_READS_PATTERN =
@@ -619,6 +634,11 @@ public class TestExecutor {
         return checkResponse(executeHttpRequest(method), responseCodeValidator);
     }
 
+    protected HttpResponse executeAndCheckHttpRequest(HttpUriRequest method, Predicate<Integer> responseCodeValidator,
+            Pair<String, String> credentials) throws Exception {
+        return checkResponse(executeBasicAuthHttpRequest(method, credentials), responseCodeValidator);
+    }
+
     protected HttpResponse executeHttpRequest(HttpUriRequest method) throws Exception {
         // https://issues.apache.org/jira/browse/ASTERIXDB-2315
         ExecutorService executor = Executors.newSingleThreadExecutor();
@@ -642,6 +662,36 @@ public class TestExecutor {
         }
     }
 
+    private HttpResponse executeBasicAuthHttpRequest(HttpUriRequest method, Pair<String, String> credentials)
+            throws Exception {
+        // https://issues.apache.org/jira/browse/ASTERIXDB-2315
+        ExecutorService executor = Executors.newSingleThreadExecutor();
+        CredentialsProvider cp = new BasicCredentialsProvider();
+        cp.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(credentials.first, credentials.second));
+        HttpClientContext hcCtx = HttpClientContext.create();
+        AuthCache ac = new BasicAuthCache();
+        ac.put(new HttpHost(method.getURI().getHost(), method.getURI().getPort(), "http"), new BasicScheme());
+        hcCtx.setAuthCache(ac);
+        CloseableHttpClient client = HttpClients.custom().setRetryHandler(StandardHttpRequestRetryHandler.INSTANCE)
+                .setDefaultCredentialsProvider(cp).build();
+        Future<HttpResponse> future = executor.submit(() -> {
+            try {
+                return client.execute(method, hcCtx);
+            } catch (Exception e) {
+                GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, "Failure executing {}", method, e);
+                throw e;
+            }
+        });
+        try {
+            return future.get();
+        } catch (Exception e) {
+            client.close();
+            throw e;
+        } finally {
+            executor.shutdownNow();
+        }
+    }
+
     protected HttpContext getHttpContext() {
         return null;
     }
@@ -867,14 +917,38 @@ public class TestExecutor {
         return builder.build();
     }
 
+    private boolean isMultipart(Parameter p) {
+        return p != null && (ParameterTypeEnum.MULTIPART_TEXT == p.getType()
+                || ParameterTypeEnum.MULTIPART_BINARY == p.getType());
+    }
+
+    private void addMultipart(MultipartEntityBuilder multipartEntityBuilder, Parameter p) {
+        if (ParameterTypeEnum.MULTIPART_TEXT == p.getType()) {
+            multipartEntityBuilder.addTextBody(p.getName(), p.getValue());
+        } else if (ParameterTypeEnum.MULTIPART_BINARY == p.getType()) {
+            File binary = new File(p.getValue());
+            multipartEntityBuilder.addBinaryBody(p.getName(), binary, ContentType.DEFAULT_BINARY, binary.getName());
+        }
+    }
+
     private HttpUriRequest buildRequest(String method, URI uri, List<Parameter> params, Optional<String> body,
             ContentType contentType) {
         RequestBuilder builder = RequestBuilder.create(method);
         builder.setUri(uri);
+        Optional<MultipartEntityBuilder> mPartBuilder = params.stream()
+                .anyMatch(p -> p.getType() == ParameterTypeEnum.MULTIPART_BINARY
+                        || p.getType() == ParameterTypeEnum.MULTIPART_TEXT)
+                                ? Optional.of(MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT))
+                                : Optional.empty();
         for (Parameter param : params) {
-            builder.addParameter(param.getName(), param.getValue());
+            if (isMultipart(param)) {
+                addMultipart(mPartBuilder.get(), param);
+            } else {
+                builder.addParameter(param.getName(), param.getValue());
+            }
         }
         builder.setCharset(UTF_8);
+        mPartBuilder.ifPresent(mpb -> builder.setEntity(mpb.build()));
         body.ifPresent(s -> builder.setEntity(new StringEntity(s, contentType)));
         return builder.build();
     }
@@ -978,6 +1052,14 @@ public class TestExecutor {
         return response.getEntity().getContent();
     }
 
+    private InputStream executeJSON(OutputFormat fmt, String method, URI uri, List<Parameter> params,
+            Predicate<Integer> responseCodeValidator, Optional<String> body, ContentType contentType,
+            Pair<String, String> credentials) throws Exception {
+        HttpUriRequest request = buildRequest(method, uri, fmt, params, body, contentType);
+        HttpResponse response = executeAndCheckHttpRequest(request, responseCodeValidator, credentials);
+        return response.getEntity().getContent();
+    }
+
     // Method that reads a DDL/Update/Query File
     // and returns the contents as a string
     // This string is later passed to REST API for execution.
@@ -1239,30 +1321,37 @@ public class TestExecutor {
                 // TODO: make this case work well with entity names containing spaces by
                 // looking for \"
                 lines = stripAllComments(statement).trim().split("\n");
+                IParserFactory parserFactory = new SqlppParserFactory();
                 for (String line : lines) {
                     String[] command = line.trim().split(" ");
+                    URI path = createEndpointURI("/admin/udf/");
                     if (command.length < 2) {
                         throw new Exception("invalid library command: " + line);
                     }
-                    String dataverse = command[1];
-                    String library = command[2];
-                    String username = command[3];
-                    String pw = command[4];
                     switch (command[0]) {
                         case "install":
-                            if (command.length != 6) {
+                            if (command.length != 7) {
                                 throw new Exception("invalid library format");
                             }
-                            String libPath = command[5];
-                            URI create = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
-                            librarian.install(create, libPath, new Pair<>(username, pw));
+                            List<String> dataverse = parserFactory.createParser(command[1]).parseMultipartIdentifier();
+                            String library = command[2];
+                            String type = command[3];
+                            String username = command[4];
+                            String pw = command[5];
+                            String libPath = command[6];
+                            librarian.install(path, "dataverse", DataverseName.create(dataverse), false, library, type,
+                                    libPath, new Pair<>(username, pw));
                             break;
                         case "uninstall":
                             if (command.length != 5) {
                                 throw new Exception("invalid library format");
                             }
-                            URI delete = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
-                            librarian.uninstall(delete, new Pair<>(username, pw));
+                            dataverse = parserFactory.createParser(command[1]).parseMultipartIdentifier();
+                            library = command[2];
+                            username = command[3];
+                            pw = command[4];
+                            librarian.uninstall(path, "dataverse", DataverseName.create(dataverse), false, library,
+                                    new Pair<>(username, pw));
                             break;
                         default:
                             throw new Exception("invalid library format");
@@ -1382,9 +1471,16 @@ public class TestExecutor {
         if (!body.isPresent()) {
             body = getBodyFromReference(statement, variableCtx);
         }
+        final Pair<String, String> credentials = extractCredentials(statement);
         InputStream resultStream;
         if ("http".equals(extension)) {
-            resultStream = executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body, contentType);
+            if (credentials != null) {
+                resultStream = executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body,
+                        contentType, credentials);
+            } else {
+                resultStream =
+                        executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body, contentType);
+            }
         } else if ("uri".equals(extension)) {
             resultStream = executeURI(reqType, URI.create(variablesReplaced), fmt, params, statusCodePredicate, body,
                     contentType);
@@ -1855,6 +1951,17 @@ public class TestExecutor {
         return params;
     }
 
+    public static Pair<String, String> extractCredentials(String statement) {
+        List<Parameter> params = new ArrayList<>();
+        final Matcher m = HTTP_AUTH_PATTERN.matcher(statement);
+        while (m.find()) {
+            String username = m.group("username");
+            String password = m.group("password");
+            return new Pair<>(username, password);
+        }
+        return null;
+    }
+
     private static String extractHttpRequestType(String statement) {
         Matcher m = HTTP_REQUEST_TYPE.matcher(statement);
         return m.find() ? m.group(1) : null;
@@ -1904,6 +2011,13 @@ public class TestExecutor {
         return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType);
     }
 
+    private InputStream executeHttp(String ctxType, String endpoint, OutputFormat fmt, List<Parameter> params,
+            Predicate<Integer> statusCodePredicate, Optional<String> body, ContentType contentType,
+            Pair<String, String> credentials) throws Exception {
+        URI uri = createEndpointURI(endpoint);
+        return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType, credentials);
+    }
+
     private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt, List<Parameter> params) throws Exception {
         return executeJSON(fmt, ctxType.toUpperCase(), uri, params);
     }
@@ -1913,6 +2027,13 @@ public class TestExecutor {
         return executeJSON(fmt, ctxType.toUpperCase(), uri, params, responseCodeValidator, body, contentType);
     }
 
+    private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt, List<Parameter> params,
+            Predicate<Integer> responseCodeValidator, Optional<String> body, ContentType contentType,
+            Pair<String, String> credentials) throws Exception {
+        return executeJSON(fmt, ctxType.toUpperCase(), uri, params, responseCodeValidator, body, contentType,
+                credentials);
+    }
+
     public void killNC(String nodeId, CompilationUnit cUnit) throws Exception {
         //get node process id
         OutputFormat fmt = OutputFormat.CLEAN_JSON;
diff --git a/asterixdb/asterix-app/src/test/resources/cc.conf b/asterixdb/asterix-app/src/test/resources/cc.conf
index 1899122..e2cd5b9 100644
--- a/asterixdb/asterix-app/src/test/resources/cc.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc.conf
@@ -33,6 +33,7 @@ nc.api.port=19005
 
 [nc]
 credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
 address=127.0.0.1
 command=asterixnc
 app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
index ac3d3d0..1707699 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install Metadata testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install Metadata testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
index 1650910..185d282 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
@@ -16,4 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=badType
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
similarity index 75%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
index 1650910..01b4982 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
@@ -16,4 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param delete:multipart_text=true
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
index 1650910..01c05d2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
@@ -16,4 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
similarity index 82%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
index 1650910..0b6d882 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
@@ -16,4 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_text=bogus
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
similarity index 85%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
index 1650910..e8de108 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
@@ -16,4 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp
index 63efff4..b1dba3d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install test testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install test testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.0.ddl.sqlpp
similarity index 91%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.0.ddl.sqlpp
index 0f0a05b..76cc70d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.0.ddl.sqlpp
@@ -16,5 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-install test testlib admin admin target/TweetSent.pyz
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
index 1650910..de72c49 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
@@ -16,4 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.2.get.http
similarity index 93%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.2.get.http
index 0f0a05b..5851101 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.2.get.http
@@ -16,5 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-install test testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+ /admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.0.ddl.sqlpp
similarity index 70%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.0.ddl.sqlpp
index ac3d3d0..0c9da1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.0.ddl.sqlpp
@@ -16,4 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install Metadata testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+DROP DATAVERSE externallibtest.foo if exists;
+CREATE DATAVERSE  externallibtest.foo;
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
+DROP DATAVERSE `external`.lib.test if exists;
+CREATE DATAVERSE  `external`.lib.test;
+DROP DATAVERSE externallibtest.foo.bar if exists;
+CREATE DATAVERSE  externallibtest.foo.bar;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
similarity index 74%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
index 1650910..c097ccc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
@@ -16,4 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param dataverse:multipart_text=foo
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
index 1650910..de72c49 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
@@ -16,4 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
index 1650910..16b3596 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
@@ -16,4 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=external
+# param dataverse:multipart_text=lib
+# param dataverse:multipart_text=test
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
index 1650910..97a3c27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
@@ -16,4 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param dataverse:multipart_text=foo
+# param dataverse:multipart_text=bar
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.5.get.http
similarity index 93%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.5.get.http
index 0f0a05b..5851101 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.5.get.http
@@ -16,5 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-install test testlib admin admin target/TweetSent.pyz
+# auth admin:admin
+ /admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp
index 9e53153..f2ddc74 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest2 testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest2 testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
index 1650910..699e565 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.0.ddl.sqlpp
similarity index 91%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.0.ddl.sqlpp
index 0f0a05b..fe2ba4c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.0.ddl.sqlpp
@@ -16,5 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-install test testlib admin admin target/TweetSent.pyz
+DROP DATAVERSE externallib.test if exists;
+CREATE DATAVERSE  externallib.test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
similarity index 91%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
index 0f0a05b..67e22cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
@@ -16,5 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-install test testlib admin admin target/TweetSent.pyz
+install externallib.test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.2.ddl.sqlpp
similarity index 88%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.2.ddl.sqlpp
index 1650910..eff35da 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.2.ddl.sqlpp
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+ USE externallib.test;
+
+create function sentiment(s)
+  as "sentiment", "TweetSent.sentiment" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.3.query.sqlpp
similarity index 93%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.3.query.sqlpp
index 0f0a05b..dbfef4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.3.query.sqlpp
@@ -16,5 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+use externallib.test;
+
+sentiment("bad");
 
-install test testlib admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.4.query.sqlpp
similarity index 93%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.4.query.sqlpp
index 0f0a05b..d89c46d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.4.query.sqlpp
@@ -16,5 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+use externallib.test;
+
+sentiment("great");
 
-install test testlib admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.5.query.sqlpp
similarity index 93%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.5.query.sqlpp
index 0f0a05b..f46bde4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.5.query.sqlpp
@@ -16,5 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+use externallib.test;
+
+sentiment("meh");
 
-install test testlib admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.6.ddl.sqlpp
similarity index 93%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.6.ddl.sqlpp
index 0f0a05b..ef74f58 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.6.ddl.sqlpp
@@ -16,5 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-install test testlib admin admin target/TweetSent.pyz
+DROP DATAVERSE externallib.test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp
index c825a93..4f2f3b6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin bad target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin bad target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp
index 35d6ef8..8f2af66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib root admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java root admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp
index 0f0a05b..ef2c4e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-install test testlib admin admin target/TweetSent.pyz
+install test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
index 0f0a05b..ef2c4e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-install test testlib admin admin target/TweetSent.pyz
+install test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp
index 1650910..699e565 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp
index 6bbdd0e..b2bf929 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp
index b4197f1..009f699 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp
@@ -16,5 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
-install externallibtest2 testlib2 admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest2 testlib2 java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp
index 45cdfd1..fdb4118 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp
@@ -21,4 +21,4 @@
  * Expected Res : Success
  */
 
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp
index 806ad5e..76cd853 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp
@@ -21,4 +21,4 @@
  * Expected Res : Success
  */
 
-install externallibtest2 testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
+install externallibtest2 testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp
index 253c657..2370125 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp
@@ -21,4 +21,4 @@
  * Expected Res : Success
  */
 
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp
index f3ea94f..2b25300 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install udfs testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install udfs testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/invalid-library-params/invalid-library-params.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/invalid-library-params/invalid-library-params.1.regexjson
new file mode 100644
index 0000000..8cabc55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/invalid-library-params/invalid-library-params.1.regexjson
@@ -0,0 +1,3 @@
+{
+	"error": "ASX0053: Parameters type and delete cannot be used together",
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.1.regexjson
new file mode 100644
index 0000000..9e26dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.1.regexjson
@@ -0,0 +1 @@
+{}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.2.regexjson
new file mode 100644
index 0000000..9f289d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.2.regexjson
@@ -0,0 +1,5 @@
+[{
+	"dataverse": "externallibtest",
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+}]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.1.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.1.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.2.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.2.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.3.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.3.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.4.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.4.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.5.regexjson
new file mode 100644
index 0000000..e5d039f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_ap1.5.regexjson
@@ -0,0 +1,20 @@
+[{
+	"dataverse": ["external", "lib", "test"],
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+},
+{
+	"dataverse": "externallibtest",
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+},
+{
+	"dataverse": ["externallibtest", "foo"],
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+},
+{
+	"dataverse": ["externallibtest", "foo", "bar"],
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+}]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
index e4669da..59dec11 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
@@ -29,6 +29,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
+      <compilation-unit name="mysentiment_multipart">
+        <output-dir compare="Text">mysentiment</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
       <compilation-unit name="python-fn-escape">
         <output-dir compare="Text">python-fn-escape</output-dir>
         <expected-error>ImportError: Module was not found in library</expected-error>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
index a9f52b4..28bbb98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
@@ -62,6 +62,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
+      <compilation-unit name="invalid_library_requests">
+        <output-dir compare="Text">mysum_bad_credential</output-dir>
+        <expected-error>ASX3042: Unsupported function language badType</expected-error>
+        <expected-error>ASX1110: The parameters \"type\" and \"delete\" cannot be provided at the same time</expected-error>
+        <expected-error>ASX0049: Parameter(s) type or delete must be specified</expected-error>
+        <expected-error>ASX0047: Invalid value for parameter \"data\": Attribute</expected-error>
+        <expected-error>ASX0049: Parameter(s) data must be specified</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
       <compilation-unit name="mysum_dropinuse">
         <output-dir compare="Text">mysum_dropinuse</output-dir>
         <expected-error>ASX1148: Cannot drop library externallibtest.testlib being used by function externallibtest.mysum(2)</expected-error>
@@ -73,6 +83,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
+      <compilation-unit name="library_list_api">
+        <output-dir compare="Text">library_list_api</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="library_list_api_multipart">
+        <output-dir compare="Text">library_list_api_multipart</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
       <compilation-unit name="getCapital">
         <output-dir compare="Text">getCapital</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
index ef390f4..93fe92d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
@@ -23,9 +23,11 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.security.MessageDigest;
+import java.util.List;
 
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.io.FileReference;
@@ -33,6 +35,10 @@ import org.apache.hyracks.ipc.impl.IPCSystem;
 
 public interface ILibraryManager {
 
+    List<Pair<DataverseName, String>> getLibraryListing() throws IOException;
+
+    String getLibraryHash(DataverseName dataverseName, String libraryName) throws IOException;
+
     ILibrary getLibrary(DataverseName dataverseName, String libraryName) throws HyracksDataException;
 
     void closeLibrary(DataverseName dataverseName, String libraryName) throws HyracksDataException;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java
index 6f128dc..3825d98 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java
@@ -31,14 +31,10 @@ import com.fasterxml.jackson.databind.node.ObjectNode;
  */
 public class LibraryDescriptor implements IJsonSerializable {
 
-    private static final long serialVersionUID = 2L;
+    private static final long serialVersionUID = 3L;
 
     private static final String FIELD_LANGUAGE = "lang";
-    private static final String FIELD_HASH = "hash";
-
-    public static final String FILE_EXT_ZIP = "zip";
-
-    public static final String FILE_EXT_PYZ = "pyz";
+    public static final String FIELD_HASH = "hash_md5";
 
     /**
      * The library's language
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 50f6458..b9ebeb6 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -277,7 +277,7 @@
 3039 = Cannot parse list item of type %1$s
 3040 = Argument type: %1$s
 3041 = Unable to load/instantiate class %1$s
-3042 = UDF of kind %1$s not supported
+3042 = Unsupported function language %1$s
 3043 = Unknown function kind %1$s
 3044 = Library class loader already registered!
 3045 = Cannot handle a function argument of type %1$s
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index c5b9b53..dce5503 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -21,6 +21,7 @@ package org.apache.asterix.external.library;
 import static com.fasterxml.jackson.databind.MapperFeature.SORT_PROPERTIES_ALPHABETICALLY;
 import static com.fasterxml.jackson.databind.SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -41,9 +42,11 @@ import java.nio.file.attribute.BasicFileAttributes;
 import java.security.DigestOutputStream;
 import java.security.KeyStore;
 import java.security.MessageDigest;
+import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.zip.ZipEntry;
@@ -51,17 +54,21 @@ import java.util.zip.ZipFile;
 
 import javax.net.ssl.SSLContext;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
 import org.apache.asterix.common.library.ILibrary;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.library.LibraryDescriptor;
 import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
 import org.apache.commons.compress.archivers.zip.ZipArchiveOutputStream;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpHeaders;
 import org.apache.http.HttpStatus;
@@ -221,7 +228,7 @@ public final class ExternalLibraryManager implements ILibraryManager, ILifeCycle
     }
 
     private FileReference getDataverseDir(DataverseName dataverseName) throws HyracksDataException {
-        return getChildFileRef(storageDir, dataverseName.getCanonicalForm());
+        return getChildFileRef(storageDir, StoragePathUtil.prepareDataverseName(dataverseName));
     }
 
     @Override
@@ -236,6 +243,68 @@ public final class ExternalLibraryManager implements ILibraryManager, ILifeCycle
     }
 
     @Override
+    public List<Pair<DataverseName, String>> getLibraryListing() throws IOException {
+        List<Pair<DataverseName, String>> libs = new ArrayList<>();
+        Files.walkFileTree(storageDirPath, new SimpleFileVisitor<Path>() {
+            @Override
+            public FileVisitResult visitFile(Path currPath, BasicFileAttributes attrs) {
+                //never want to see any files
+                return FileVisitResult.TERMINATE;
+            }
+
+            @Override
+            public FileVisitResult preVisitDirectory(Path currPath, BasicFileAttributes attrs)
+                    throws HyracksDataException {
+                if (currPath.equals(storageDirPath) || currPath.getParent().equals(storageDirPath)) {
+                    return FileVisitResult.CONTINUE;
+                }
+                if (currPath.getFileName().toString().codePointAt(0) == StoragePathUtil.DATAVERSE_CONTINUATION_MARKER) {
+                    return FileVisitResult.CONTINUE;
+                }
+                final String candidateDvAndLib = storageDirPath.toAbsolutePath().normalize()
+                        .relativize(currPath.toAbsolutePath().normalize()).toString();
+                List<String> dvParts = new ArrayList<>();
+                final String[] tokens = StringUtils.split(candidateDvAndLib, File.separatorChar);
+                if (tokens == null || tokens.length < 2) {
+                    //? shouldn't happen
+                    return FileVisitResult.TERMINATE;
+                }
+                //add first part, then all multiparts
+                dvParts.add(tokens[0]);
+                int currToken = 1;
+                for (; currToken < tokens.length && tokens[currToken]
+                        .codePointAt(0) == StoragePathUtil.DATAVERSE_CONTINUATION_MARKER; currToken++) {
+                    dvParts.add(tokens[currToken].substring(1));
+                }
+                //we should only arrive at foo/^bar/^baz/.../^bat/lib
+                //anything else is fishy or empty
+                if (currToken != tokens.length - 1) {
+                    return FileVisitResult.SKIP_SUBTREE;
+                }
+                String candidateLib = tokens[currToken];
+                DataverseName candidateDv = DataverseName.create(dvParts);
+                FileReference candidateLibPath = findLibraryRevDir(candidateDv, candidateLib);
+                if (candidateLibPath != null) {
+                    libs.add(new Pair<>(candidateDv, candidateLib));
+                }
+                return FileVisitResult.SKIP_SUBTREE;
+            }
+        });
+        return libs;
+    }
+
+    @Override
+    public String getLibraryHash(DataverseName dataverseName, String libraryName) throws IOException {
+        FileReference revDir = findLibraryRevDir(dataverseName, libraryName);
+        if (revDir == null) {
+            throw HyracksDataException
+                    .create(AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION, "Library does not exist"));
+        }
+        LibraryDescriptor desc = getLibraryDescriptor(revDir);
+        return desc.getHash();
+    }
+
+    @Override
     public ILibrary getLibrary(DataverseName dataverseName, String libraryName) throws HyracksDataException {
         Pair<DataverseName, String> key = getKey(dataverseName, libraryName);
         synchronized (this) {
@@ -545,9 +614,8 @@ public final class ExternalLibraryManager implements ILibraryManager, ILifeCycle
         outputFile.getFile().createNewFile();
         IFileHandle fHandle = ioManager.open(outputFile, IIOManager.FileReadWriteMode.READ_WRITE,
                 IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-        try {
-            WritableByteChannel outChannel = ioManager.newWritableChannel(fHandle);
-            OutputStream outputStream = Channels.newOutputStream(outChannel);
+        WritableByteChannel outChannel = ioManager.newWritableChannel(fHandle);
+        try (OutputStream outputStream = Channels.newOutputStream(outChannel)) {
             IOUtils.copyLarge(dataStream, outputStream, copyBuffer);
             outputStream.flush();
             ioManager.sync(fHandle, true);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
index f9d30b1..1fa53ea 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
@@ -82,11 +82,18 @@ class ExternalScalarPythonFunctionEvaluator extends ExternalScalarFunctionEvalua
         super(finfo, args, argTypes, ctx);
         IApplicationConfig cfg = ctx.getServiceContext().getAppConfig();
         String pythonPathCmd = cfg.getString(NCConfig.Option.PYTHON_CMD);
+        boolean findPython = cfg.getBoolean(NCConfig.Option.PYTHON_CMD_AUTOLOCATE);
         List<String> pythonArgs = new ArrayList<>();
         if (pythonPathCmd == null) {
-            //if absolute path to interpreter is not specified, use environmental python
-            pythonPathCmd = "/usr/bin/env";
-            pythonArgs.add("python3");
+            //if absolute path to interpreter is not specified, try to use environmental python
+            if (findPython) {
+                pythonPathCmd = "/usr/bin/env";
+                pythonArgs.add("python3");
+            } else {
+                throw HyracksDataException.create(AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                        "Python interpreter not specified, and " + NCConfig.Option.PYTHON_CMD_AUTOLOCATE.ini()
+                                + " is false"));
+            }
         }
         File pythonPath = new File(pythonPathCmd);
         List<String> sitePkgs = new ArrayList<>();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java
index 4f91b1a..c12bb58 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java
@@ -130,17 +130,9 @@ public class LibraryDeployPrepareOperatorDescriptor extends AbstractLibraryOpera
 
                 switch (language) {
                     case JAVA:
-                        if (!LibraryDescriptor.FILE_EXT_ZIP.equals(fileExt)) {
-                            // shouldn't happen
-                            throw new IOException("Unexpected file type: " + fileExt);
-                        }
                         libraryManager.unzip(targetFile, contentsDir);
                         break;
                     case PYTHON:
-                        if (!LibraryDescriptor.FILE_EXT_PYZ.equals(fileExt)) {
-                            // shouldn't happen
-                            throw new IOException("Unexpected file type: " + fileExt);
-                        }
                         boolean extractMsgPack = ctx.getJobletContext().getServiceContext().getAppConfig()
                                 .getBoolean(PYTHON_USE_BUNDLED_MSGPACK);
                         shiv(targetFile, stageDir, contentsDir, extractMsgPack);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java
index 6e3be21..9ab07eb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java
@@ -21,13 +21,18 @@ package org.apache.asterix.external.util;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.security.MessageDigest;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
 
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.util.bytes.HexPrinter;
 
 public class ExternalLibraryUtils {
 
     private ExternalLibraryUtils() {
-
     }
 
     public static String digestToHexString(MessageDigest digest) throws IOException {
@@ -36,4 +41,15 @@ public class ExternalLibraryUtils {
         HexPrinter.printHexString(hashBytes, 0, hashBytes.length, hashBuilder);
         return hashBuilder.toString();
     }
+
+    public static Map<DataverseName, Map<String, String>> produceLibraryListing(ILibraryManager libraryManager)
+            throws IOException {
+        List<Pair<DataverseName, String>> libs = libraryManager.getLibraryListing();
+        Map<DataverseName, Map<String, String>> dvToLibHashes = new TreeMap<>();
+        for (Pair<DataverseName, String> lib : libs) {
+            dvToLibHashes.computeIfAbsent(lib.first, h -> new TreeMap<>()).put(lib.getSecond(),
+                    libraryManager.getLibraryHash(lib.first, lib.second));
+        }
+        return dvToLibHashes;
+    }
 }
diff --git a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
index 508236a..4ea9e0e 100644
--- a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
+++ b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
@@ -298,6 +298,8 @@
       <xs:restriction base="xs:string">
          <xs:enumeration value="string"/>
          <xs:enumeration value="json"/>
+         <xs:enumeration value="multipart_text"/>
+         <xs:enumeration value="multipart_binary"/>
       </xs:restriction>
    </xs:simpleType>
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index d803732..22b240a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -94,6 +94,7 @@ public class NCConfig extends ControllerConfig {
         IO_WORKERS_PER_PARTITION(POSITIVE_INTEGER, 2),
         IO_QUEUE_SIZE(POSITIVE_INTEGER, 10),
         PYTHON_CMD(STRING, (String) null),
+        PYTHON_CMD_AUTOLOCATE(BOOLEAN, false),
         PYTHON_ADDITIONAL_PACKAGES(STRING_ARRAY, new String[0]),
         PYTHON_USE_BUNDLED_MSGPACK(BOOLEAN, true),
         PYTHON_ARGS(STRING_ARRAY, (String[]) null),
@@ -235,13 +236,15 @@ public class NCConfig extends ControllerConfig {
                 case IO_QUEUE_SIZE:
                     return "Length of the queue used for requests to write and read";
                 case PYTHON_CMD:
-                    return "Absolute path to python interpreter. Defaults to environmental Python3";
+                    return "Absolute path to python interpreter";
                 case PYTHON_ADDITIONAL_PACKAGES:
                     return "List of additional paths, separated by a path separator character, to add to sys.path behind msgpack and library package paths";
                 case PYTHON_USE_BUNDLED_MSGPACK:
                     return "True to include bundled msgpack on Python sys.path, false to use system-provided msgpack";
                 case PYTHON_ARGS:
                     return "Python args to pass to Python interpreter";
+                case PYTHON_CMD_AUTOLOCATE:
+                    return "Whether or not to attempt to automatically set PYTHON_CMD to a usable interpreter";
                 case CREDENTIAL_FILE:
                     return "Path to HTTP basic credentials";
                 default: