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

asterixdb git commit: [NO ISSUE][*DB] Enable loading external UDF for AsterixHyracksIntegrationUtil

Repository: asterixdb
Updated Branches:
  refs/heads/master d25513ccb -> 85e6f43c8


[NO ISSUE][*DB] Enable loading external UDF for AsterixHyracksIntegrationUtil

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

Add external UDF support for AsterixHyracksIntegrationUtil. Developer
may use following system properties to install external UDF for local
environment.
"external.lib" : udf package path.
"external.lib.dataverse": dataverse name (default: test).
"external.lib.libname": library name (default: testlib).

Change-Id: I333238b944b1c6d48a2b80b87a25f600522e0cc4
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2078
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/85e6f43c
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/85e6f43c
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/85e6f43c

Branch: refs/heads/master
Commit: 85e6f43c8204b6a2769aac326306d35fe79ba276
Parents: d25513c
Author: Xikui Wang <xk...@gmail.com>
Authored: Fri Nov 10 11:51:27 2017 -0800
Committer: Michael Blow <mb...@apache.org>
Committed: Sat Nov 11 10:22:10 2017 -0800

----------------------------------------------------------------------
 .../common/AsterixHyracksIntegrationUtil.java   | 28 +++++-
 .../app/external/ExternalLibraryUtils.java      |  2 +-
 .../app/external/ExternalUDFLibrarian.java      | 98 +++++++++++++++++++
 .../app/external/IExternalUDFLibrarian.java     | 31 ++++++
 .../app/bootstrap/TestNodeController.java       |  6 +-
 .../asterix/app/external/TestLibrarian.java     | 99 --------------------
 .../asterix/test/common/ITestLibrarian.java     | 31 ------
 .../asterix/test/common/TestExecutor.java       |  5 +-
 .../asterix/test/runtime/LangExecutionUtil.java | 10 +-
 9 files changed, 165 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 50c3ff6..2a8a831 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -29,11 +29,14 @@ import java.util.List;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.app.external.ExternalUDFLibrarian;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.PropertiesAccessor;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.hyracks.bootstrap.NCApplication;
 import org.apache.commons.io.FileUtils;
@@ -91,7 +94,7 @@ public class AsterixHyracksIntegrationUtil {
             ncApplication.registerConfig(ncConfigManager);
             nodeControllers.add(
                     new NodeControllerService(fixupIODevices(createNCConfig(nodeId, ncConfigManager)), ncApplication));
-        } ;
+        }
 
         cc.start();
 
@@ -122,6 +125,22 @@ public class AsterixHyracksIntegrationUtil {
         this.ncs = nodeControllers.toArray(new NodeControllerService[nodeControllers.size()]);
     }
 
+    public void init(boolean deleteOldInstanceData, String externalLibPath) throws Exception {
+        List<ILibraryManager> libraryManagers = new ArrayList<>();
+        ExternalUDFLibrarian librarian = new ExternalUDFLibrarian(libraryManagers);
+        librarian.cleanup();
+        init(deleteOldInstanceData);
+        if (externalLibPath != null && externalLibPath.length() != 0) {
+            libraryManagers.add(((ICcApplicationContext) cc.getApplicationContext()).getLibraryManager());
+            for (NodeControllerService nc : ncs) {
+                INcApplicationContext runtimeCtx = (INcApplicationContext) nc.getApplicationContext();
+                libraryManagers.add(runtimeCtx.getLibraryManager());
+            }
+            librarian.install(System.getProperty("external.lib.dataverse", "test"),
+                    System.getProperty("external.lib.libname", "testlib"), externalLibPath);
+        }
+    }
+
     public ClusterControllerService getClusterControllerService() {
         return cc;
     }
@@ -271,14 +290,15 @@ public class AsterixHyracksIntegrationUtil {
     public static void main(String[] args) throws Exception {
         AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
         try {
-            integrationUtil.run(Boolean.getBoolean("cleanup.start"), Boolean.getBoolean("cleanup.shutdown"));
+            integrationUtil.run(Boolean.getBoolean("cleanup.start"), Boolean.getBoolean("cleanup.shutdown"),
+                    System.getProperty("external.lib", ""));
         } catch (Exception e) {
             LOGGER.log(Level.WARNING, "Unexpected exception", e);
             System.exit(1);
         }
     }
 
-    protected void run(boolean cleanupOnStart, boolean cleanupOnShutdown) throws Exception {
+    protected void run(boolean cleanupOnStart, boolean cleanupOnShutdown, String loadExternalLibs) throws Exception {
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
@@ -291,7 +311,7 @@ public class AsterixHyracksIntegrationUtil {
         });
         System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, "asterix-build-configuration.xml");
 
-        init(cleanupOnStart);
+        init(cleanupOnStart, loadExternalLibs);
         while (true) {
             Thread.sleep(10000);
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
index 5f86c28..4271d55 100755
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
@@ -346,7 +346,7 @@ public class ExternalLibraryUtils {
         if (jarsInLibDir.length > 1) {
             throw new Exception("Incorrect library structure: found multiple library jars");
         }
-        if (jarsInLibDir.length < 0) {
+        if (jarsInLibDir.length <= 0) {
             throw new Exception("Incorrect library structure: could not find library jar");
         }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
new file mode 100644
index 0000000..b1d2159
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.external;
+
+import java.io.File;
+import java.io.IOException;
+import java.rmi.RemoteException;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.event.service.AsterixEventServiceUtil;
+import org.apache.commons.io.FileUtils;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public class ExternalUDFLibrarian implements IExternalUDFLibrarian {
+
+    // The following list includes a library manager for the CC
+    // and library managers for NCs (one-per-NC).
+    private final List<ILibraryManager> libraryManagers;
+
+    public ExternalUDFLibrarian(List<ILibraryManager> libraryManagers) {
+        this.libraryManagers = libraryManagers;
+    }
+
+    @Override
+    public void install(String dvName, String libName, String libPath) throws Exception {
+        // get the directory of the to be installed libraries
+        File installLibDir = ExternalLibraryUtils.getLibraryInstallDir();
+        // directory exists?
+        if (!installLibDir.exists()) {
+            installLibDir.mkdir();
+        }
+        // copy the library file into the directory
+        File destinationDir = new File(
+                installLibDir.getAbsolutePath() + File.separator + dvName + File.separator + libName);
+        FileUtils.deleteQuietly(destinationDir);
+        destinationDir.mkdirs();
+        try {
+            AsterixEventServiceUtil.unzip(libPath, destinationDir.getAbsolutePath());
+        } catch (Exception e) {
+
+            throw new Exception("Couldn't unzip the file: " + libPath, e);
+        }
+
+        for (ILibraryManager libraryManager : libraryManagers) {
+            ExternalLibraryUtils.registerLibrary(libraryManager, dvName, libName);
+        }
+        // get library file
+        // install if needed (add functions, adapters, datasources, parsers to the metadata)
+        // <Not required for use>
+        ExternalLibraryUtils.installLibraryIfNeeded(dvName, destinationDir, new HashMap<>());
+    }
+
+    @Override
+    public void uninstall(String dvName, String libName) throws RemoteException, AsterixException, ACIDException {
+        ExternalLibraryUtils.uninstallLibrary(dvName, libName);
+        for (ILibraryManager libraryManager : libraryManagers) {
+            libraryManager.deregisterLibraryClassLoader(dvName, libName);
+        }
+    }
+
+    public static void removeLibraryDir() throws IOException {
+        File installLibDir = ExternalLibraryUtils.getLibraryInstallDir();
+        FileUtils.deleteQuietly(installLibDir);
+    }
+
+    public void cleanup() throws AsterixException, RemoteException, ACIDException {
+        for (ILibraryManager libraryManager : libraryManagers) {
+            List<Pair<String, String>> libs = libraryManager.getAllLibraries();
+            for (Pair<String, String> dvAndLib : libs) {
+                ExternalLibraryUtils.uninstallLibrary(dvAndLib.first, dvAndLib.second);
+                libraryManager.deregisterLibraryClassLoader(dvAndLib.first, dvAndLib.second);
+            }
+        }
+        // get the directory of the to be installed libraries
+        File installLibDir = ExternalLibraryUtils.getLibraryInstallDir();
+        FileUtils.deleteQuietly(installLibDir);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
new file mode 100644
index 0000000..9a17444
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.app.external;
+
+import java.io.IOException;
+import java.rmi.RemoteException;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
+
+public interface IExternalUDFLibrarian {
+    public void install(String dvName, String libName, String libPath) throws IOException, Exception;
+
+    public void uninstall(String dvName, String libName) throws RemoteException, AsterixException, ACIDException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index b155b51..1810517 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -27,7 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.logging.Logger;
 
-import org.apache.asterix.app.external.TestLibrarian;
+import org.apache.asterix.app.external.ExternalUDFLibrarian;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.nc.TransactionSubsystem;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
@@ -137,7 +137,7 @@ public class TestNodeController {
             File outdir = new File(PATH_ACTUAL);
             outdir.mkdirs();
             // remove library directory
-            TestLibrarian.removeLibraryDir();
+            ExternalUDFLibrarian.removeLibraryDir();
             ExecutionTestUtil.setUp(cleanupOnStart,
                     testConfigFileName == null ? TEST_CONFIG_FILE_NAME : testConfigFileName,
                     ExecutionTestUtil.integrationUtil, runHDFS);
@@ -148,7 +148,7 @@ public class TestNodeController {
     }
 
     public void deInit() throws Exception {
-        TestLibrarian.removeLibraryDir();
+        ExternalUDFLibrarian.removeLibraryDir();
         ExecutionTestUtil.tearDown(cleanupOnStop);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/TestLibrarian.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/TestLibrarian.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/TestLibrarian.java
deleted file mode 100644
index 76fa308..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/TestLibrarian.java
+++ /dev/null
@@ -1,99 +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.
- */
-package org.apache.asterix.app.external;
-
-import java.io.File;
-import java.io.IOException;
-import java.rmi.RemoteException;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.event.service.AsterixEventServiceUtil;
-import org.apache.asterix.test.common.ITestLibrarian;
-import org.apache.commons.io.FileUtils;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-
-public class TestLibrarian implements ITestLibrarian {
-
-    // The following list includes a library manager for the CC
-    // and library managers for NCs (one-per-NC).
-    private final List<ILibraryManager> libraryManagers;
-
-    public TestLibrarian(List<ILibraryManager> libraryManagers) {
-        this.libraryManagers = libraryManagers;
-    }
-
-    @Override
-    public void install(String dvName, String libName, String libPath) throws Exception {
-        // get the directory of the to be installed libraries
-        File installLibDir = ExternalLibraryUtils.getLibraryInstallDir();
-        // directory exists?
-        if (!installLibDir.exists()) {
-            installLibDir.mkdir();
-        }
-        // copy the library file into the directory
-        File destinationDir = new File(
-                installLibDir.getAbsolutePath() + File.separator + dvName + File.separator + libName);
-        FileUtils.deleteQuietly(destinationDir);
-        destinationDir.mkdirs();
-        try {
-            AsterixEventServiceUtil.unzip(libPath, destinationDir.getAbsolutePath());
-        } catch (Exception e) {
-
-            throw new Exception("Couldn't unzip the file: " + libPath, e);
-        }
-
-        for (ILibraryManager libraryManager : libraryManagers) {
-            ExternalLibraryUtils.registerLibrary(libraryManager, dvName, libName);
-        }
-        // get library file
-        // install if needed (add functions, adapters, datasources, parsers to the metadata)
-        // <Not required for use>
-        ExternalLibraryUtils.installLibraryIfNeeded(dvName, destinationDir, new HashMap<>());
-    }
-
-    @Override
-    public void uninstall(String dvName, String libName) throws RemoteException, AsterixException, ACIDException {
-        ExternalLibraryUtils.uninstallLibrary(dvName, libName);
-        for (ILibraryManager libraryManager : libraryManagers) {
-            libraryManager.deregisterLibraryClassLoader(dvName, libName);
-        }
-    }
-
-    public static void removeLibraryDir() throws IOException {
-        File installLibDir = ExternalLibraryUtils.getLibraryInstallDir();
-        FileUtils.deleteQuietly(installLibDir);
-    }
-
-    public void cleanup() throws AsterixException, RemoteException, ACIDException {
-        for (ILibraryManager libraryManager : libraryManagers) {
-            List<Pair<String, String>> libs = libraryManager.getAllLibraries();
-            for (Pair<String, String> dvAndLib : libs) {
-                ExternalLibraryUtils.uninstallLibrary(dvAndLib.first, dvAndLib.second);
-                libraryManager.deregisterLibraryClassLoader(dvAndLib.first, dvAndLib.second);
-            }
-        }
-        // get the directory of the to be installed libraries
-        File installLibDir = ExternalLibraryUtils.getLibraryInstallDir();
-        FileUtils.deleteQuietly(installLibDir);
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ITestLibrarian.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ITestLibrarian.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ITestLibrarian.java
deleted file mode 100644
index d661710..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ITestLibrarian.java
+++ /dev/null
@@ -1,31 +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.
- */
-package org.apache.asterix.test.common;
-
-import java.io.IOException;
-import java.rmi.RemoteException;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-
-public interface ITestLibrarian {
-    public void install(String dvName, String libName, String libPath) throws IOException, Exception;
-
-    public void uninstall(String dvName, String libName) throws RemoteException, AsterixException, ACIDException;
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
----------------------------------------------------------------------
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 4ca9131..0d66256 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
@@ -58,6 +58,7 @@ import java.util.logging.Logger;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.asterix.app.external.IExternalUDFLibrarian;
 import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.utils.Servlets;
@@ -134,7 +135,7 @@ public class TestExecutor {
      */
     protected final List<InetSocketAddress> endpoints;
     protected int endpointSelector;
-    protected ITestLibrarian librarian;
+    protected IExternalUDFLibrarian librarian;
     private Map<File, TestLoop> testLoops = new HashMap<>();
 
     public TestExecutor() {
@@ -153,7 +154,7 @@ public class TestExecutor {
         this.endpoints = endpoints;
     }
 
-    public void setLibrarian(ITestLibrarian librarian) {
+    public void setLibrarian(IExternalUDFLibrarian librarian) {
         this.librarian = librarian;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/85e6f43c/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
index d80eabc..78bed6a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
@@ -31,7 +31,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.asterix.app.external.TestLibrarian;
+import org.apache.asterix.app.external.ExternalUDFLibrarian;
 import org.apache.asterix.common.config.ClusterProperties;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.test.common.TestExecutor;
@@ -56,7 +56,7 @@ public class LangExecutionUtil {
     private static final List<String> badTestCases = new ArrayList<>();
     private static TestExecutor testExecutor;
 
-    private static TestLibrarian librarian;
+    private static ExternalUDFLibrarian librarian;
     private static final int repeat = Integer.getInteger("test.repeat", 1);
 
     public static void setUp(String configFile, TestExecutor executor) throws Exception {
@@ -64,8 +64,8 @@ public class LangExecutionUtil {
         File outdir = new File(PATH_ACTUAL);
         outdir.mkdirs();
         List<ILibraryManager> libraryManagers = ExecutionTestUtil.setUp(cleanupOnStart, configFile);
-        TestLibrarian.removeLibraryDir();
-        librarian = new TestLibrarian(libraryManagers);
+        ExternalUDFLibrarian.removeLibraryDir();
+        librarian = new ExternalUDFLibrarian(libraryManagers);
         testExecutor.setLibrarian(librarian);
         if (repeat != 1) {
             System.out.println("FYI: each test will be run " + repeat + " times.");
@@ -79,7 +79,7 @@ public class LangExecutionUtil {
             // Check whether there are leaked threads.
             checkThreadLeaks();
         } finally {
-            TestLibrarian.removeLibraryDir();
+            ExternalUDFLibrarian.removeLibraryDir();
             ExecutionTestUtil.tearDown(cleanupOnStop);
             ExecutionTestUtil.integrationUtil.removeTestStorageFiles();
             if (!badTestCases.isEmpty()) {