You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/04/07 00:43:36 UTC

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

From Ian Maxon <im...@uci.edu>:

Ian Maxon has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )


Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................

[ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

- user model changes: yes
- storage format changes: no
- interface changes: yes

Details:

- Redo the API now that the canonical form of DataverseNames are
acceptable as an external interface.
- The path of the request encodes the library dataverse and name
- POST updates or creates libraries. "type" remains a field in the
  multipart request rather than inferring based on the extension
  as was done in the API before the previous patch
- DELETE again deletes libraries based on the same path they
  were POSTed at.
- Minor dedupe on error printing in NcUdfApiServlet

Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
C asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
D asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
20 files changed, 135 insertions(+), 290 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/32/10932/1

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 a234b9f..9a73d62 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
@@ -25,24 +25,24 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.OpenOption;
 import java.nio.file.Path;
-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.metadata.DataverseName;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.net.URLCodec;
 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;
@@ -61,7 +61,6 @@
 
 public abstract class AbstractNCUdfServlet extends AbstractServlet {
 
-    private final IParserFactory parserFactory;
     INcApplicationContext appCtx;
     INCServiceContext srvCtx;
 
@@ -70,80 +69,38 @@
     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";
+    public static final String NAME_KEY = "name";
+    public static final String DATAVERSE_KEY = "dataverse";
+    private static final URLCodec URL_CODEC = new URLCodec();
 
-    protected enum LibraryOperation {
-        UPSERT,
-        DELETE
-    }
-
-    protected final static class LibraryUploadData {
-
-        final LibraryOperation op;
-        final DataverseName dataverse;
-        final String name;
+    protected static final class LibraryUploadData {
         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();
+        private LibraryUploadData(MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload)
+                throws IOException {
             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(MixedAttribute type, InterfaceHttpData fileUpload)
+                throws IOException {
+            //POST imples replaceIfExists
+            return new LibraryUploadData(type, true, 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,
-            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-
+            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 {
@@ -176,6 +133,10 @@
         }
     }
 
+    protected String getDataverseKey() {
+        return DATAVERSE_KEY;
+    }
+
     URI createDownloadURI(Path file) throws Exception {
         String path = paths[0].substring(0, trims[0]) + GET_UDF_DIST_ENDPOINT + '/' + file.getFileName();
         String host = getHyracksClientConnection().getHost();
@@ -190,67 +151,29 @@
         return hcc;
     }
 
-    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 Pair<DataverseName, String> decodeDvAndLibFromLocalPath(String localPath) throws RuntimeDataException {
+        String[] pathSegments = StringUtils.split(localPath, '/');
+        if (pathSegments.length != 2) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED,
+                    "The URL-encoded dataverse name and library name in the request path");
+        }
+        DataverseName dvName = DataverseName.createFromCanonicalForm(decodeUriSegment(pathSegments[0]));
+        String libName = pathSegments[1];
+        return new Pair<>(dvName, libName);
     }
 
     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);
+            throws IOException {
         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 (typeAttribute == null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_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)) {
+        else if (!isNotAttribute(typeAttribute)) {
             InterfaceHttpData libraryData = requestDecoder.getBodyHttpData(DATA_PARAMETER);
             if (libraryData == null) {
                 throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DATA_PARAMETER);
@@ -258,27 +181,15 @@
                 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);
-            }
+            LibraryUploadData uploadData =
+                    LibraryUploadData.libraryCreationUploadData((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);
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_PARAMETER);
         }
     }
 
@@ -307,4 +218,12 @@
         return HttpResponseStatus.INTERNAL_SERVER_ERROR;
     }
 
+    private static String decodeUriSegment(String uriSegment) {
+        try {
+            return new String(URL_CODEC.decode(uriSegment.getBytes(StandardCharsets.US_ASCII)), StandardCharsets.UTF_8);
+        } catch (DecoderException e) {
+            throw new IllegalArgumentException(e);
+        }
+    }
+
 }
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 717ebf8..fec0b38 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
@@ -20,6 +20,7 @@
 
 import static org.apache.asterix.api.http.server.ServletConstants.SYS_AUTH_HEADER;
 import static org.apache.asterix.common.library.LibraryDescriptor.FIELD_HASH;
+import static org.apache.hyracks.api.exceptions.IFormattedException.getError;
 
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -53,11 +54,11 @@
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
 import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 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;
@@ -77,7 +78,6 @@
 
 public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
-    protected final ILangCompilationProvider compilationProvider;
     protected final IReceptionist receptionist;
 
     protected Path workingDir;
@@ -88,13 +88,17 @@
     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, compilationProvider, httpServerProtocol, httpServerPort);
-        this.compilationProvider = compilationProvider;
+            HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
         this.receptionist = appCtx.getReceptionist();
         this.timeout = appCtx.getExternalProperties().getLibraryDeployTimeout();
     }
 
+    private enum LibraryOperation {
+        UPSERT,
+        DELETE
+    }
+
     @Override
     public void init() throws IOException {
         appCtx = (INcApplicationContext) plainAppCtx;
@@ -183,11 +187,8 @@
                 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(getDataverseKey(), dvAndLibs.getKey().getCanonicalForm());
+                        libraryEntry.put(NAME_KEY, libsInDv.getKey());
                         libraryEntry.put(FIELD_HASH, libsInDv.getValue());
                         libraryList.add(libraryEntry);
                     }
@@ -217,37 +218,29 @@
                 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();
+            writeException(e, response);
             LOGGER.error("Error reading library", e);
         }
     }
 
-    @Override
-    protected void post(IServletRequest request, IServletResponse response) {
+    private void handleModification(IServletRequest request, IServletResponse response, LibraryOperation op) {
         HttpRequest httpRequest = request.getHttpRequest();
         Path libraryTempFile = null;
         FileOutputStream libTmpOut = null;
-        HttpPostRequestDecoder requestDecoder = new HttpPostRequestDecoder(httpRequest);
+        HttpPostRequestDecoder requestDecoder = null;
+        String localPath = localPath(request);
         try {
-            LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
+            Pair<DataverseName, String> dvAndName = decodeDvAndLibFromLocalPath(localPath);
             IRequestReference requestReference = receptionist.welcome(request);
-            if (uploadData.op == LibraryOperation.UPSERT) {
+            if (op == LibraryOperation.UPSERT) {
+                requestDecoder = new HttpPostRequestDecoder(httpRequest);
+                LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
                 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 " + uploadData.dataverse
-                            + "." + uploadData.name);
+                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library "
+                            + dvAndName.getFirst().getCanonicalForm() + "." + dvAndName.getSecond());
                 }
                 MessageDigest digest = MessageDigest.getInstance("MD5");
                 libTmpOut = new FileOutputStream(libraryTempFile.toFile());
@@ -256,11 +249,12 @@
                     IOUtils.copyLarge(ui, os);
                 }
                 URI downloadURI = createDownloadURI(libraryTempFile);
-                doCreate(uploadData.dataverse, uploadData.name, language,
+                doCreate(dvAndName.getFirst(), dvAndName.getSecond(), language,
                         ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
                         requestReference, request);
-            } else if (uploadData.op == LibraryOperation.DELETE) {
-                doDrop(uploadData.dataverse, uploadData.name, uploadData.replaceIfExists, requestReference, request);
+            } else if (op == LibraryOperation.DELETE) {
+                //DELETE semantics imply ifExists
+                doDrop(dvAndName.getFirst(), dvAndName.getSecond(), false, requestReference, request);
             }
             response.setStatus(HttpResponseStatus.OK);
             PrintWriter responseWriter = response.writer();
@@ -268,20 +262,12 @@
             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);
+            writeException(e, response);
+            LOGGER.info("Error modifying library", e);
         } finally {
-            requestDecoder.destroy();
+            if (requestDecoder != null) {
+                requestDecoder.destroy();
+            }
             try {
                 if (libraryTempFile != null) {
                     if (libTmpOut != null) {
@@ -295,4 +281,28 @@
         }
     }
 
+    private void writeException(Exception e, IServletResponse response) {
+        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();
+    }
+
+    @Override
+    protected void post(IServletRequest request, IServletResponse response) {
+        handleModification(request, response, LibraryOperation.UPSERT);
+    }
+
+    @Override
+    protected void delete(IServletRequest request, IServletResponse response) {
+        handleModification(request, response, LibraryOperation.DELETE);
+    }
+
 }
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 1563833..2c29d14 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,7 +24,6 @@
 
 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;
@@ -39,8 +38,8 @@
     public static final String GET_ALL_UDF_ENDPOINT = "/all";
 
     public NCUdfRecoveryServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
+            HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, 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 c27b3b1..c148c92 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
@@ -224,13 +224,14 @@
                 parseCredentialMap(((NodeControllerService) ncServiceCtx.getControllerService()).getConfiguration()
                         .getCredentialFilePath()));
         Pair<Map<String, String>, Map<String, String>> auth = BasicAuthServlet.generateSysAuthHeader(apiServer.ctx());
-        apiServer.addServlet(new BasicAuthServlet(apiServer.ctx(),
-                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(),
-                        sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
+        apiServer
+                .addServlet(new BasicAuthServlet(apiServer.ctx(),
+                        new NCUdfApiServlet(apiServer.ctx(), new String[] { UDF }, getApplicationContext(),
+                                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()),
                 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 88277e4..2450025 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,7 +23,6 @@
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
@@ -31,6 +30,7 @@
 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;
@@ -56,20 +56,11 @@
     }
 
     @Override
-    public void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            String type, String libPath, Pair<String, String> credentials) throws Exception {
+    public void install(URI path, 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);
         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());
@@ -78,22 +69,10 @@
     }
 
     @Override
-    public void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            Pair<String, String> credentials) throws IOException, AsterixException {
+    public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
-        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);
+        HttpDelete del = new HttpDelete(path);
+        HttpResponse response = hc.execute(del, hcCtx);
         handleResponse(response);
     }
 
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 639475b..998fa78 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,14 +22,10 @@
 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 type, 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, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            Pair<String, String> credentials) throws IOException, AsterixException;
+    void uninstall(URI path, 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 453ed59..c262c43 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,8 +82,6 @@
 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;
@@ -1326,10 +1324,11 @@
                 // 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/");
+                    String dataverse = command[1];
+                    String library = command[2];
+                    URI path = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
                     if (command.length < 2) {
                         throw new Exception("invalid library command: " + line);
                     }
@@ -1338,25 +1337,19 @@
                             if (command.length != 7) {
                                 throw new Exception("invalid library format");
                             }
-                            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));
+                            librarian.install(path, type, libPath, new Pair<>(username, pw));
                             break;
                         case "uninstall":
                             if (command.length != 5) {
                                 throw new Exception("invalid library format");
                             }
-                            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));
+                            librarian.uninstall(path, new Pair<>(username, pw));
                             break;
                         default:
                             throw new Exception("invalid library format");
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
similarity index 91%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
index 185d282..a9e043a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
similarity index 86%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
index e8de108..43a40a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
@@ -17,8 +17,5 @@
  * under the License.
  */
 # auth admin:admin
-# param dataverse:multipart_text=Default
-# param name:multipart_text=testlib
-# param type:multipart_text=java
 
-/admin/udf
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
index 01c05d2..9815651 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
@@ -17,8 +17,6 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
index 0b6d882..5353086 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
index e8de108..28b64f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
@@ -17,8 +17,6 @@
  * under the License.
  */
 # auth admin:admin
-# param dataverse:multipart_text=Default
-# param name:multipart_text=testlib
 # param type:multipart_text=java
 
-/admin/udf
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
deleted file mode 100644
index 01b4982..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-# 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/library_list_api/library_list_api.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
index de72c49..d9d6236 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
index c097ccc..3f7bdb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
@@ -17,10 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest%2Ffoo/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
index de72c49..d9d6236 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
index 16b3596..9ad58b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
@@ -17,11 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/external%2Flib%2Ftest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
index 97a3c27..a731831 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
@@ -17,11 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest%2Ffoo%2Fbar/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
index c896e0d..f0cb7cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
@@ -1,5 +1,5 @@
 [{
-	"dataverse": ["external", "lib", "test"],
+	"dataverse": "external/lib/test",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 },
@@ -9,12 +9,12 @@
 	"name": "testlib"
 },
 {
-	"dataverse": ["externallibtest", "foo"],
+	"dataverse": "externallibtest/foo",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 },
 {
-	"dataverse": ["externallibtest", "foo", "bar"],
+	"dataverse": "externallibtest/foo/bar",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 }]
\ No newline at end of file
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 28bbb98..28cbabb 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
@@ -65,8 +65,8 @@
       <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>ASX1117: Cannot find library with name testlib</expected-error>
+        <expected-error>ASX0049: Parameter(s) type 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>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 1
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-MessageType: newchange

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ian Maxon <im...@uci.edu>:

Ian Maxon has submitted this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................

[ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

- user model changes: yes
- storage format changes: no
- interface changes: yes

Details:

- Redo the API now that the canonical form of DataverseNames are
acceptable as an external interface.
- The path of the request encodes the library dataverse and name
- POST updates or creates libraries. "type" remains a field in the
  multipart request rather than inferring based on the extension
  as was done in the API before the previous patch
- DELETE again deletes libraries based on the same path they
  were POSTed at.
- Minor dedupe on error printing in NcUdfApiServlet

Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
C asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
D asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
22 files changed, 139 insertions(+), 291 deletions(-)

Approvals:
  Michael Blow: Looks good to me, approved
  Jenkins: Verified; Verified



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 a234b9f..eb424af 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
@@ -28,21 +28,18 @@
 import java.nio.file.Files;
 import java.nio.file.OpenOption;
 import java.nio.file.Path;
-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.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;
@@ -61,7 +58,6 @@
 
 public abstract class AbstractNCUdfServlet extends AbstractServlet {
 
-    private final IParserFactory parserFactory;
     INcApplicationContext appCtx;
     INCServiceContext srvCtx;
 
@@ -70,80 +66,37 @@
     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";
+    public static final String NAME_KEY = "name";
+    public static final String DATAVERSE_KEY = "dataverse";
 
-    protected enum LibraryOperation {
-        UPSERT,
-        DELETE
-    }
-
-    protected final static class LibraryUploadData {
-
-        final LibraryOperation op;
-        final DataverseName dataverse;
-        final String name;
+    protected static final class LibraryUploadData {
         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();
+        private LibraryUploadData(MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload)
+                throws IOException {
             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(MixedAttribute type, InterfaceHttpData fileUpload)
+                throws IOException {
+            //POST imples replaceIfExists
+            return new LibraryUploadData(type, true, 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,
-            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-
+            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 {
@@ -176,6 +129,10 @@
         }
     }
 
+    protected String getDataverseKey() {
+        return DATAVERSE_KEY;
+    }
+
     URI createDownloadURI(Path file) throws Exception {
         String path = paths[0].substring(0, trims[0]) + GET_UDF_DIST_ENDPOINT + '/' + file.getFileName();
         String host = getHyracksClientConnection().getHost();
@@ -190,67 +147,29 @@
         return hcc;
     }
 
-    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 Pair<DataverseName, String> decodeDvAndLibFromLocalPath(String localPath) throws RuntimeDataException {
+        String[] pathSegments = StringUtils.split(localPath, '/');
+        if (pathSegments.length != 2) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED,
+                    "The URL-encoded " + getDataverseKey() + " name and library name in the request path");
+        }
+        DataverseName dvName = DataverseName.createFromCanonicalForm(ServletUtil.decodeUriSegment(pathSegments[0]));
+        String libName = ServletUtil.decodeUriSegment(pathSegments[1]);
+        return new Pair<>(dvName, libName);
     }
 
     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);
+            throws IOException {
         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 (typeAttribute == null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_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)) {
+        else if (!isNotAttribute(typeAttribute)) {
             InterfaceHttpData libraryData = requestDecoder.getBodyHttpData(DATA_PARAMETER);
             if (libraryData == null) {
                 throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DATA_PARAMETER);
@@ -258,27 +177,15 @@
                 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);
-            }
+            LibraryUploadData uploadData =
+                    LibraryUploadData.libraryCreationUploadData((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);
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_PARAMETER);
         }
     }
 
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 717ebf8..fec0b38 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
@@ -20,6 +20,7 @@
 
 import static org.apache.asterix.api.http.server.ServletConstants.SYS_AUTH_HEADER;
 import static org.apache.asterix.common.library.LibraryDescriptor.FIELD_HASH;
+import static org.apache.hyracks.api.exceptions.IFormattedException.getError;
 
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -53,11 +54,11 @@
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
 import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 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;
@@ -77,7 +78,6 @@
 
 public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
-    protected final ILangCompilationProvider compilationProvider;
     protected final IReceptionist receptionist;
 
     protected Path workingDir;
@@ -88,13 +88,17 @@
     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, compilationProvider, httpServerProtocol, httpServerPort);
-        this.compilationProvider = compilationProvider;
+            HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
         this.receptionist = appCtx.getReceptionist();
         this.timeout = appCtx.getExternalProperties().getLibraryDeployTimeout();
     }
 
+    private enum LibraryOperation {
+        UPSERT,
+        DELETE
+    }
+
     @Override
     public void init() throws IOException {
         appCtx = (INcApplicationContext) plainAppCtx;
@@ -183,11 +187,8 @@
                 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(getDataverseKey(), dvAndLibs.getKey().getCanonicalForm());
+                        libraryEntry.put(NAME_KEY, libsInDv.getKey());
                         libraryEntry.put(FIELD_HASH, libsInDv.getValue());
                         libraryList.add(libraryEntry);
                     }
@@ -217,37 +218,29 @@
                 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();
+            writeException(e, response);
             LOGGER.error("Error reading library", e);
         }
     }
 
-    @Override
-    protected void post(IServletRequest request, IServletResponse response) {
+    private void handleModification(IServletRequest request, IServletResponse response, LibraryOperation op) {
         HttpRequest httpRequest = request.getHttpRequest();
         Path libraryTempFile = null;
         FileOutputStream libTmpOut = null;
-        HttpPostRequestDecoder requestDecoder = new HttpPostRequestDecoder(httpRequest);
+        HttpPostRequestDecoder requestDecoder = null;
+        String localPath = localPath(request);
         try {
-            LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
+            Pair<DataverseName, String> dvAndName = decodeDvAndLibFromLocalPath(localPath);
             IRequestReference requestReference = receptionist.welcome(request);
-            if (uploadData.op == LibraryOperation.UPSERT) {
+            if (op == LibraryOperation.UPSERT) {
+                requestDecoder = new HttpPostRequestDecoder(httpRequest);
+                LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
                 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 " + uploadData.dataverse
-                            + "." + uploadData.name);
+                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library "
+                            + dvAndName.getFirst().getCanonicalForm() + "." + dvAndName.getSecond());
                 }
                 MessageDigest digest = MessageDigest.getInstance("MD5");
                 libTmpOut = new FileOutputStream(libraryTempFile.toFile());
@@ -256,11 +249,12 @@
                     IOUtils.copyLarge(ui, os);
                 }
                 URI downloadURI = createDownloadURI(libraryTempFile);
-                doCreate(uploadData.dataverse, uploadData.name, language,
+                doCreate(dvAndName.getFirst(), dvAndName.getSecond(), language,
                         ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
                         requestReference, request);
-            } else if (uploadData.op == LibraryOperation.DELETE) {
-                doDrop(uploadData.dataverse, uploadData.name, uploadData.replaceIfExists, requestReference, request);
+            } else if (op == LibraryOperation.DELETE) {
+                //DELETE semantics imply ifExists
+                doDrop(dvAndName.getFirst(), dvAndName.getSecond(), false, requestReference, request);
             }
             response.setStatus(HttpResponseStatus.OK);
             PrintWriter responseWriter = response.writer();
@@ -268,20 +262,12 @@
             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);
+            writeException(e, response);
+            LOGGER.info("Error modifying library", e);
         } finally {
-            requestDecoder.destroy();
+            if (requestDecoder != null) {
+                requestDecoder.destroy();
+            }
             try {
                 if (libraryTempFile != null) {
                     if (libTmpOut != null) {
@@ -295,4 +281,28 @@
         }
     }
 
+    private void writeException(Exception e, IServletResponse response) {
+        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();
+    }
+
+    @Override
+    protected void post(IServletRequest request, IServletResponse response) {
+        handleModification(request, response, LibraryOperation.UPSERT);
+    }
+
+    @Override
+    protected void delete(IServletRequest request, IServletResponse response) {
+        handleModification(request, response, LibraryOperation.DELETE);
+    }
+
 }
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 1563833..2c29d14 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,7 +24,6 @@
 
 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;
@@ -39,8 +38,8 @@
     public static final String GET_ALL_UDF_ENDPOINT = "/all";
 
     public NCUdfRecoveryServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
+            HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
index b97d88a..51fa326 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
@@ -20,12 +20,15 @@
 
 import static org.apache.asterix.api.http.server.ServletConstants.RESULTSET_ATTR;
 
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.net.URLCodec;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.client.result.ResultSet;
@@ -54,4 +57,13 @@
         List<String> values = request.getParameterValues(dataverseParameterName);
         return !values.isEmpty() ? DataverseName.create(values) : null;
     }
+
+    public static String decodeUriSegment(String uriSegment) {
+        try {
+            return new String(URLCodec.decodeUrl(uriSegment.getBytes(StandardCharsets.US_ASCII)),
+                    StandardCharsets.UTF_8);
+        } catch (DecoderException e) {
+            throw new IllegalArgumentException(e);
+        }
+    }
 }
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 c27b3b1..c148c92 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
@@ -224,13 +224,14 @@
                 parseCredentialMap(((NodeControllerService) ncServiceCtx.getControllerService()).getConfiguration()
                         .getCredentialFilePath()));
         Pair<Map<String, String>, Map<String, String>> auth = BasicAuthServlet.generateSysAuthHeader(apiServer.ctx());
-        apiServer.addServlet(new BasicAuthServlet(apiServer.ctx(),
-                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(),
-                        sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
+        apiServer
+                .addServlet(new BasicAuthServlet(apiServer.ctx(),
+                        new NCUdfApiServlet(apiServer.ctx(), new String[] { UDF }, getApplicationContext(),
+                                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()),
                 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 88277e4..2450025 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,7 +23,6 @@
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
@@ -31,6 +30,7 @@
 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;
@@ -56,20 +56,11 @@
     }
 
     @Override
-    public void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            String type, String libPath, Pair<String, String> credentials) throws Exception {
+    public void install(URI path, 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);
         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());
@@ -78,22 +69,10 @@
     }
 
     @Override
-    public void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            Pair<String, String> credentials) throws IOException, AsterixException {
+    public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
-        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);
+        HttpDelete del = new HttpDelete(path);
+        HttpResponse response = hc.execute(del, hcCtx);
         handleResponse(response);
     }
 
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 639475b..998fa78 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,14 +22,10 @@
 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 type, 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, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            Pair<String, String> credentials) throws IOException, AsterixException;
+    void uninstall(URI path, 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 453ed59..f347b77 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
@@ -41,8 +41,10 @@
 import java.net.Socket;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.URLEncoder;
 import java.nio.CharBuffer;
 import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.text.MessageFormat;
@@ -82,8 +84,6 @@
 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;
@@ -1326,10 +1326,12 @@
                 // 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/");
+                    //TODO: this is not right. URLEncoder does not properly encode paths.
+                    String dataverse = URLEncoder.encode(command[1], StandardCharsets.US_ASCII.name());
+                    String library = URLEncoder.encode(command[2], StandardCharsets.US_ASCII.name());
+                    URI path = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
                     if (command.length < 2) {
                         throw new Exception("invalid library command: " + line);
                     }
@@ -1338,25 +1340,19 @@
                             if (command.length != 7) {
                                 throw new Exception("invalid library format");
                             }
-                            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));
+                            librarian.install(path, type, libPath, new Pair<>(username, pw));
                             break;
                         case "uninstall":
                             if (command.length != 5) {
                                 throw new Exception("invalid library format");
                             }
-                            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));
+                            librarian.uninstall(path, new Pair<>(username, pw));
                             break;
                         default:
                             throw new Exception("invalid library format");
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
similarity index 91%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
index 185d282..a9e043a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
similarity index 86%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
index e8de108..43a40a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
@@ -17,8 +17,5 @@
  * under the License.
  */
 # auth admin:admin
-# param dataverse:multipart_text=Default
-# param name:multipart_text=testlib
-# param type:multipart_text=java
 
-/admin/udf
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
index 01c05d2..9815651 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
@@ -17,8 +17,6 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
index 0b6d882..5353086 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
index e8de108..28b64f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
@@ -17,8 +17,6 @@
  * under the License.
  */
 # auth admin:admin
-# param dataverse:multipart_text=Default
-# param name:multipart_text=testlib
 # param type:multipart_text=java
 
-/admin/udf
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
deleted file mode 100644
index 01b4982..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-# 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/library_list_api/library_list_api.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
index de72c49..d9d6236 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
index c097ccc..3f7bdb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
@@ -17,10 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest%2Ffoo/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
index de72c49..d9d6236 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
index 16b3596..9ad58b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
@@ -17,11 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/external%2Flib%2Ftest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
index 97a3c27..a731831 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
@@ -17,11 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest%2Ffoo%2Fbar/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
index 67e22cb..c1aa309 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallib.test testlib python 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/results/external-library/library_list_api_multipart/library_list_api.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
index c896e0d..f0cb7cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
@@ -1,5 +1,5 @@
 [{
-	"dataverse": ["external", "lib", "test"],
+	"dataverse": "external/lib/test",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 },
@@ -9,12 +9,12 @@
 	"name": "testlib"
 },
 {
-	"dataverse": ["externallibtest", "foo"],
+	"dataverse": "externallibtest/foo",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 },
 {
-	"dataverse": ["externallibtest", "foo", "bar"],
+	"dataverse": "externallibtest/foo/bar",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 }]
\ No newline at end of file
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 28bbb98..28cbabb 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
@@ -65,8 +65,8 @@
       <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>ASX1117: Cannot find library with name testlib</expected-error>
+        <expected-error>ASX0049: Parameter(s) type 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>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 5
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-MessageType: merged

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ian Maxon <im...@uci.edu>:

Ian Maxon has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )


Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................

[ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

- user model changes: yes
- storage format changes: no
- interface changes: yes

Details:

- Redo the API now that the canonical form of DataverseNames are
acceptable as an external interface.
- The path of the request encodes the library dataverse and name
- POST updates or creates libraries. "type" remains a field in the
  multipart request rather than inferring based on the extension
  as was done in the API before the previous patch
- DELETE again deletes libraries based on the same path they
  were POSTed at.
- Minor dedupe on error printing in NcUdfApiServlet

Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
C asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
D asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
20 files changed, 135 insertions(+), 290 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/32/10932/1

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 a234b9f..9a73d62 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
@@ -25,24 +25,24 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.OpenOption;
 import java.nio.file.Path;
-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.metadata.DataverseName;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.net.URLCodec;
 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;
@@ -61,7 +61,6 @@
 
 public abstract class AbstractNCUdfServlet extends AbstractServlet {
 
-    private final IParserFactory parserFactory;
     INcApplicationContext appCtx;
     INCServiceContext srvCtx;
 
@@ -70,80 +69,38 @@
     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";
+    public static final String NAME_KEY = "name";
+    public static final String DATAVERSE_KEY = "dataverse";
+    private static final URLCodec URL_CODEC = new URLCodec();
 
-    protected enum LibraryOperation {
-        UPSERT,
-        DELETE
-    }
-
-    protected final static class LibraryUploadData {
-
-        final LibraryOperation op;
-        final DataverseName dataverse;
-        final String name;
+    protected static final class LibraryUploadData {
         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();
+        private LibraryUploadData(MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload)
+                throws IOException {
             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(MixedAttribute type, InterfaceHttpData fileUpload)
+                throws IOException {
+            //POST imples replaceIfExists
+            return new LibraryUploadData(type, true, 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,
-            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-
+            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 {
@@ -176,6 +133,10 @@
         }
     }
 
+    protected String getDataverseKey() {
+        return DATAVERSE_KEY;
+    }
+
     URI createDownloadURI(Path file) throws Exception {
         String path = paths[0].substring(0, trims[0]) + GET_UDF_DIST_ENDPOINT + '/' + file.getFileName();
         String host = getHyracksClientConnection().getHost();
@@ -190,67 +151,29 @@
         return hcc;
     }
 
-    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 Pair<DataverseName, String> decodeDvAndLibFromLocalPath(String localPath) throws RuntimeDataException {
+        String[] pathSegments = StringUtils.split(localPath, '/');
+        if (pathSegments.length != 2) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED,
+                    "The URL-encoded dataverse name and library name in the request path");
+        }
+        DataverseName dvName = DataverseName.createFromCanonicalForm(decodeUriSegment(pathSegments[0]));
+        String libName = pathSegments[1];
+        return new Pair<>(dvName, libName);
     }
 
     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);
+            throws IOException {
         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 (typeAttribute == null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_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)) {
+        else if (!isNotAttribute(typeAttribute)) {
             InterfaceHttpData libraryData = requestDecoder.getBodyHttpData(DATA_PARAMETER);
             if (libraryData == null) {
                 throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DATA_PARAMETER);
@@ -258,27 +181,15 @@
                 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);
-            }
+            LibraryUploadData uploadData =
+                    LibraryUploadData.libraryCreationUploadData((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);
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_PARAMETER);
         }
     }
 
@@ -307,4 +218,12 @@
         return HttpResponseStatus.INTERNAL_SERVER_ERROR;
     }
 
+    private static String decodeUriSegment(String uriSegment) {
+        try {
+            return new String(URL_CODEC.decode(uriSegment.getBytes(StandardCharsets.US_ASCII)), StandardCharsets.UTF_8);
+        } catch (DecoderException e) {
+            throw new IllegalArgumentException(e);
+        }
+    }
+
 }
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 717ebf8..fec0b38 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
@@ -20,6 +20,7 @@
 
 import static org.apache.asterix.api.http.server.ServletConstants.SYS_AUTH_HEADER;
 import static org.apache.asterix.common.library.LibraryDescriptor.FIELD_HASH;
+import static org.apache.hyracks.api.exceptions.IFormattedException.getError;
 
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -53,11 +54,11 @@
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
 import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 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;
@@ -77,7 +78,6 @@
 
 public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
-    protected final ILangCompilationProvider compilationProvider;
     protected final IReceptionist receptionist;
 
     protected Path workingDir;
@@ -88,13 +88,17 @@
     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, compilationProvider, httpServerProtocol, httpServerPort);
-        this.compilationProvider = compilationProvider;
+            HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
         this.receptionist = appCtx.getReceptionist();
         this.timeout = appCtx.getExternalProperties().getLibraryDeployTimeout();
     }
 
+    private enum LibraryOperation {
+        UPSERT,
+        DELETE
+    }
+
     @Override
     public void init() throws IOException {
         appCtx = (INcApplicationContext) plainAppCtx;
@@ -183,11 +187,8 @@
                 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(getDataverseKey(), dvAndLibs.getKey().getCanonicalForm());
+                        libraryEntry.put(NAME_KEY, libsInDv.getKey());
                         libraryEntry.put(FIELD_HASH, libsInDv.getValue());
                         libraryList.add(libraryEntry);
                     }
@@ -217,37 +218,29 @@
                 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();
+            writeException(e, response);
             LOGGER.error("Error reading library", e);
         }
     }
 
-    @Override
-    protected void post(IServletRequest request, IServletResponse response) {
+    private void handleModification(IServletRequest request, IServletResponse response, LibraryOperation op) {
         HttpRequest httpRequest = request.getHttpRequest();
         Path libraryTempFile = null;
         FileOutputStream libTmpOut = null;
-        HttpPostRequestDecoder requestDecoder = new HttpPostRequestDecoder(httpRequest);
+        HttpPostRequestDecoder requestDecoder = null;
+        String localPath = localPath(request);
         try {
-            LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
+            Pair<DataverseName, String> dvAndName = decodeDvAndLibFromLocalPath(localPath);
             IRequestReference requestReference = receptionist.welcome(request);
-            if (uploadData.op == LibraryOperation.UPSERT) {
+            if (op == LibraryOperation.UPSERT) {
+                requestDecoder = new HttpPostRequestDecoder(httpRequest);
+                LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
                 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 " + uploadData.dataverse
-                            + "." + uploadData.name);
+                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library "
+                            + dvAndName.getFirst().getCanonicalForm() + "." + dvAndName.getSecond());
                 }
                 MessageDigest digest = MessageDigest.getInstance("MD5");
                 libTmpOut = new FileOutputStream(libraryTempFile.toFile());
@@ -256,11 +249,12 @@
                     IOUtils.copyLarge(ui, os);
                 }
                 URI downloadURI = createDownloadURI(libraryTempFile);
-                doCreate(uploadData.dataverse, uploadData.name, language,
+                doCreate(dvAndName.getFirst(), dvAndName.getSecond(), language,
                         ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
                         requestReference, request);
-            } else if (uploadData.op == LibraryOperation.DELETE) {
-                doDrop(uploadData.dataverse, uploadData.name, uploadData.replaceIfExists, requestReference, request);
+            } else if (op == LibraryOperation.DELETE) {
+                //DELETE semantics imply ifExists
+                doDrop(dvAndName.getFirst(), dvAndName.getSecond(), false, requestReference, request);
             }
             response.setStatus(HttpResponseStatus.OK);
             PrintWriter responseWriter = response.writer();
@@ -268,20 +262,12 @@
             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);
+            writeException(e, response);
+            LOGGER.info("Error modifying library", e);
         } finally {
-            requestDecoder.destroy();
+            if (requestDecoder != null) {
+                requestDecoder.destroy();
+            }
             try {
                 if (libraryTempFile != null) {
                     if (libTmpOut != null) {
@@ -295,4 +281,28 @@
         }
     }
 
+    private void writeException(Exception e, IServletResponse response) {
+        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();
+    }
+
+    @Override
+    protected void post(IServletRequest request, IServletResponse response) {
+        handleModification(request, response, LibraryOperation.UPSERT);
+    }
+
+    @Override
+    protected void delete(IServletRequest request, IServletResponse response) {
+        handleModification(request, response, LibraryOperation.DELETE);
+    }
+
 }
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 1563833..2c29d14 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,7 +24,6 @@
 
 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;
@@ -39,8 +38,8 @@
     public static final String GET_ALL_UDF_ENDPOINT = "/all";
 
     public NCUdfRecoveryServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
+            HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, 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 c27b3b1..c148c92 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
@@ -224,13 +224,14 @@
                 parseCredentialMap(((NodeControllerService) ncServiceCtx.getControllerService()).getConfiguration()
                         .getCredentialFilePath()));
         Pair<Map<String, String>, Map<String, String>> auth = BasicAuthServlet.generateSysAuthHeader(apiServer.ctx());
-        apiServer.addServlet(new BasicAuthServlet(apiServer.ctx(),
-                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(),
-                        sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
+        apiServer
+                .addServlet(new BasicAuthServlet(apiServer.ctx(),
+                        new NCUdfApiServlet(apiServer.ctx(), new String[] { UDF }, getApplicationContext(),
+                                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()),
                 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 88277e4..2450025 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,7 +23,6 @@
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
@@ -31,6 +30,7 @@
 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;
@@ -56,20 +56,11 @@
     }
 
     @Override
-    public void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            String type, String libPath, Pair<String, String> credentials) throws Exception {
+    public void install(URI path, 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);
         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());
@@ -78,22 +69,10 @@
     }
 
     @Override
-    public void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            Pair<String, String> credentials) throws IOException, AsterixException {
+    public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
-        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);
+        HttpDelete del = new HttpDelete(path);
+        HttpResponse response = hc.execute(del, hcCtx);
         handleResponse(response);
     }
 
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 639475b..998fa78 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,14 +22,10 @@
 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 type, 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, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
-            Pair<String, String> credentials) throws IOException, AsterixException;
+    void uninstall(URI path, 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 453ed59..c262c43 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,8 +82,6 @@
 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;
@@ -1326,10 +1324,11 @@
                 // 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/");
+                    String dataverse = command[1];
+                    String library = command[2];
+                    URI path = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
                     if (command.length < 2) {
                         throw new Exception("invalid library command: " + line);
                     }
@@ -1338,25 +1337,19 @@
                             if (command.length != 7) {
                                 throw new Exception("invalid library format");
                             }
-                            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));
+                            librarian.install(path, type, libPath, new Pair<>(username, pw));
                             break;
                         case "uninstall":
                             if (command.length != 5) {
                                 throw new Exception("invalid library format");
                             }
-                            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));
+                            librarian.uninstall(path, new Pair<>(username, pw));
                             break;
                         default:
                             throw new Exception("invalid library format");
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
similarity index 91%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
index 185d282..a9e043a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
similarity index 86%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
index e8de108..43a40a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
@@ -17,8 +17,5 @@
  * under the License.
  */
 # auth admin:admin
-# param dataverse:multipart_text=Default
-# param name:multipart_text=testlib
-# param type:multipart_text=java
 
-/admin/udf
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
index 01c05d2..9815651 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
@@ -17,8 +17,6 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
index 0b6d882..5353086 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
similarity index 90%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
index e8de108..28b64f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
@@ -17,8 +17,6 @@
  * under the License.
  */
 # auth admin:admin
-# param dataverse:multipart_text=Default
-# param name:multipart_text=testlib
 # param type:multipart_text=java
 
-/admin/udf
+/admin/udf/Default/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
deleted file mode 100644
index 01b4982..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-# 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/library_list_api/library_list_api.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
index de72c49..d9d6236 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
index c097ccc..3f7bdb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
@@ -17,10 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest%2Ffoo/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
index de72c49..d9d6236 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
@@ -17,9 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
index 16b3596..9ad58b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
@@ -17,11 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/external%2Flib%2Ftest/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
index 97a3c27..a731831 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
@@ -17,11 +17,7 @@
  * under the License.
  */
 # 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
+/admin/udf/externallibtest%2Ffoo%2Fbar/testlib
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
index c896e0d..f0cb7cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
@@ -1,5 +1,5 @@
 [{
-	"dataverse": ["external", "lib", "test"],
+	"dataverse": "external/lib/test",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 },
@@ -9,12 +9,12 @@
 	"name": "testlib"
 },
 {
-	"dataverse": ["externallibtest", "foo"],
+	"dataverse": "externallibtest/foo",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 },
 {
-	"dataverse": ["externallibtest", "foo", "bar"],
+	"dataverse": "externallibtest/foo/bar",
 	"hash_md5": "R{[a-zA-Z0-9-]+}",
 	"name": "testlib"
 }]
\ No newline at end of file
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 28bbb98..28cbabb 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
@@ -65,8 +65,8 @@
       <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>ASX1117: Cannot find library with name testlib</expected-error>
+        <expected-error>ASX0049: Parameter(s) type 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>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 1
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-MessageType: newchange

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/11813/ : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 1
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Wed, 07 Apr 2021 01:44:50 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/11826/ : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 4
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Wed, 07 Apr 2021 20:43:41 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ian Maxon <im...@uci.edu>:

Ian Maxon has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 4:

(6 comments)

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java 
File asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java:

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@158 
PS3, Line 158: Pair<DataverseName, String> 
> consider having a FQN class for this, actually, we probably should extract a common base class from  […]
yeah that's a good idea. can I make it a separate change? because i think it would need to also modify Library and a lot of other places where the qualified library name is used rather than the relative one.


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@165 
PS3, Line 165: pathSegments[1]
> what are the naming requirements for a lib name?  shouldn't we decode here?
i don't think it's been defined but i'll say whatever restrictions there are on dataset names. 'cause they're more or less analogous naming schemes. so yes it could contain something illegal in a url.


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@221 
PS3, Line 221:     private static String decodeUriSegment(String uriSegment) {
> doesn't need to be this change, but we do "this" a bunch of places, i wonder if we should relocate t […]
i moved it to ServletUtil because we don't import commons-codec in hyracks-http and i didn't want to introduce more dependencies


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@223 
PS3, Line 223: URL_CODEC.decode
> looks like you can eliminate the instance and replace this instance method with UrlCodec. […]
Done


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java 
File asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java:

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java@1331 
PS3, Line 1331: dataverse
> encodeUriSegment
Done


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java@1331 
PS3, Line 1331: library
> should this get encoded too?
yes, done.



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 4
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Wed, 07 Apr 2021 19:40:20 +0000
Gerrit-HasComments: Yes
Gerrit-Has-Labels: No
Comment-In-Reply-To: Michael Blow <mb...@apache.org>
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ian Maxon <im...@uci.edu>:

Hello Jenkins, 

I'd like you to reexamine a change. Please visit

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932

to look at the new patch set (#2).

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................

[ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

- user model changes: yes
- storage format changes: no
- interface changes: yes

Details:

- Redo the API now that the canonical form of DataverseNames are
acceptable as an external interface.
- The path of the request encodes the library dataverse and name
- POST updates or creates libraries. "type" remains a field in the
  multipart request rather than inferring based on the extension
  as was done in the API before the previous patch
- DELETE again deletes libraries based on the same path they
  were POSTed at.
- Minor dedupe on error printing in NcUdfApiServlet

Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.1.post.http
C asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.2.delete.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.3.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.4.post.http
R asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/invalid_library_requests.5.post.http
D asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
21 files changed, 136 insertions(+), 291 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/32/10932/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 2
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-MessageType: newpatchset

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Michael Blow <mb...@apache.org>:

Michael Blow has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 4: Code-Review+2

(2 comments)

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java 
File asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java:

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@158 
PS3, Line 158: Pair<DataverseName, String> 
> yeah that's a good idea. […]
Ack


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/4/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java 
File asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java:

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/4/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java@1331 
PS4, Line 1331: //TODO: this is not right. URLEncoder does not properly encode paths.
i don't think there's a problem here, this is not a path, but a path segment



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 4
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Wed, 07 Apr 2021 22:10:07 +0000
Gerrit-HasComments: Yes
Gerrit-Has-Labels: Yes
Comment-In-Reply-To: Michael Blow <mb...@apache.org>
Comment-In-Reply-To: Ian Maxon <im...@uci.edu>
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/11814/ : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 2
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Wed, 07 Apr 2021 03:41:41 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Michael Blow <mb...@apache.org>:

Michael Blow has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 3:

(6 comments)

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java 
File asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java:

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@158 
PS3, Line 158: Pair<DataverseName, String> 
consider having a FQN class for this, actually, we probably should extract a common base class from DatasetFullyQualifiedName, and create a LibraryFullyQualifiedName?  Extensions that come to mind could also use the same, say for couplings or pails or something like...


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@165 
PS3, Line 165: pathSegments[1]
what are the naming requirements for a lib name?  shouldn't we decode here?


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@221 
PS3, Line 221:     private static String decodeUriSegment(String uriSegment) {
doesn't need to be this change, but we do "this" a bunch of places, i wonder if we should relocate this method (and probably also += a decodeUri) to HttpUtil?


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java@223 
PS3, Line 223: URL_CODEC.decode
looks like you can eliminate the instance and replace this instance method with UrlCodec.decodeUrl() static method- that's all this instance method does is wrap that.


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java 
File asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java:

https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java@1331 
PS3, Line 1331: dataverse
encodeUriSegment


https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932/3/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java@1331 
PS3, Line 1331: library
should this get encoded too?



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 3
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Wed, 07 Apr 2021 10:21:37 +0000
Gerrit-HasComments: Yes
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[cheshire-cat]: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932 )

Change subject: [ASTERIXDB-2872][API] UDF API changes for new DataverseName usage
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/wcHrcoCQhyDbursk6 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/10932
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: cheshire-cat
Gerrit-Change-Id: Icec828b119fd959760281c4f4cc49449b179546b
Gerrit-Change-Number: 10932
Gerrit-PatchSet: 1
Gerrit-Owner: Ian Maxon <im...@uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-CC: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 07 Apr 2021 00:53:58 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment