You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by xi...@apache.org on 2018/03/01 16:48:46 UTC

asterixdb git commit: [ASTERIXDB-2148][FUN] Add init parameter for external UDF

Repository: asterixdb
Updated Branches:
  refs/heads/master cbfe8f9b6 -> 03ddd0cde


[ASTERIXDB-2148][FUN] Add init parameter for external UDF

- user model changes: no
- storage format changes: no
- interface changes: yes
  Added parameters to LibraryFunction in library.xsd.

Details:
By enabling init function in external UDF, a user can reuse the same UDF
implementation with different paramters. One example can be to reuse the
same Machine Learning algorithm with different model files by assigning
different file paths to the UDF parameter.

Change-Id: I567ce0bcac288267595b2565e53fea61e16fbd65
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2107
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


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

Branch: refs/heads/master
Commit: 03ddd0cded4d93634d4fd99521cb15aeb8badfc8
Parents: cbfe8f9
Author: Xikui Wang <xk...@gmail.com>
Authored: Wed Feb 28 16:36:22 2018 -0800
Committer: Xikui Wang <xk...@gmail.com>
Committed: Thu Mar 1 08:48:14 2018 -0800

----------------------------------------------------------------------
 .../KeywordsDetector_List1.txt                  |   2 +
 .../KeywordsDetector_List2.txt                  |   2 +
 .../app/external/ExternalLibraryUtils.java      | 103 ++++++++++--------
 .../app/external/ExternalUDFLibrarian.java      |   8 +-
 .../keyword_detector/keyword_detector.1.ddl.aql |  44 ++++++++
 .../keyword_detector.2.update.aql               |  24 ++++
 .../keyword_detector/keyword_detector.3.lib.aql |  19 ++++
 .../keyword_detector.4.update.aql               |  30 +++++
 .../keyword_detector.5.query.aql                |  27 +++++
 .../keyword_detector.1.ddl.sqlpp                |  44 ++++++++
 .../keyword_detector.2.update.sqlpp             |  24 ++++
 .../keyword_detector.3.lib.sqlpp                |  19 ++++
 .../keyword_detector.4.update.sqlpp             |  28 +++++
 .../keyword_detector.5.query.sqlpp              |  26 +++++
 .../keyword_detector/keyword_detector.1.adm     |   5 +
 .../validate-default-library.1.adm              |   2 +
 .../test/resources/runtimets/testsuite_it.xml   |   5 +
 .../resources/runtimets/testsuite_it_sqlpp.xml  |   5 +
 .../asterix/common/library/ILibraryManager.java |  25 ++++-
 .../asterix/external/api/IFunctionHelper.java   |  16 ++-
 .../external/library/ExternalFunction.java      |   8 +-
 .../library/ExternalLibraryManager.java         |  12 ++
 .../external/library/JavaFunctionHelper.java    |  21 ++--
 .../library/RuntimeExternalFunctionUtil.java    | 109 -------------------
 .../src/main/resources/schema/library.xsd       |   6 +-
 .../library/KeywordsDetectorFactory.java        |  31 ++++++
 .../library/KeywordsDetectorFunction.java       |  67 ++++++++++++
 .../src/test/resources/library_descriptor.xml   |  48 +++++---
 .../asterix/lang/aql/parser/FunctionParser.java |   8 +-
 .../lang/sqlpp/parser/FunctionParser.java       |   2 +-
 .../asterix/metadata/entities/Function.java     |  10 +-
 .../FunctionTupleTranslator.java                |   2 +-
 .../functions/ExternalFunctionCompilerUtil.java |  10 +-
 .../om/functions/ExternalFunctionInfo.java      |   2 +-
 .../om/functions/IExternalFunctionInfo.java     |   2 +-
 .../functionDataset/functionDataset.1.adm       |   2 +
 36 files changed, 584 insertions(+), 214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/data/external_function/KeywordsDetector_List1.txt
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/data/external_function/KeywordsDetector_List1.txt b/asterixdb/asterix-app/data/external_function/KeywordsDetector_List1.txt
new file mode 100644
index 0000000..1cef71e
--- /dev/null
+++ b/asterixdb/asterix-app/data/external_function/KeywordsDetector_List1.txt
@@ -0,0 +1,2 @@
+Hugh
+Schwan
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/data/external_function/KeywordsDetector_List2.txt
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/data/external_function/KeywordsDetector_List2.txt b/asterixdb/asterix-app/data/external_function/KeywordsDetector_List2.txt
new file mode 100644
index 0000000..3f5ac3b
--- /dev/null
+++ b/asterixdb/asterix-app/data/external_function/KeywordsDetector_List2.txt
@@ -0,0 +1,2 @@
+Kottke
+Liz
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/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 e60444f..c125a40 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
@@ -75,14 +75,11 @@ public class ExternalLibraryUtils {
             // get the list of files in the directory
             for (File dataverseDir : installLibDir.listFiles(File::isDirectory)) {
                 for (File libraryDir : dataverseDir.listFiles(File::isDirectory)) {
-                    // for each file (library), register library
-                    registerLibrary(externalLibraryManager, dataverseDir.getName(), libraryDir.getName());
-                    // is metadata node?
-                    if (isMetadataNode) {
-                        // get library file
-                        // install if needed (i,e, add the functions, adapters, datasources, parsers to the metadata)
-                        installLibraryIfNeeded(dataverseDir.getName(), libraryDir, uninstalledLibs);
-                    }
+                    // For each file (library), register classloader and configure its parameter.
+                    // If current node is Metadata Node, add the library to metadata.
+                    registerClassLoader(externalLibraryManager, dataverseDir.getName(), libraryDir.getName());
+                    configureLibrary(externalLibraryManager, dataverseDir.getName(), libraryDir, uninstalledLibs,
+                            isMetadataNode);
                 }
             }
         }
@@ -184,15 +181,9 @@ public class ExternalLibraryUtils {
         return true;
     }
 
-    /**
-     * Each element of a library is installed as part of a transaction. Any
-     * failure in installing an element does not effect installation of other
-     * libraries.
-     */
-    protected static void installLibraryIfNeeded(String dataverse, final File libraryDir,
-            Map<String, List<String>> uninstalledLibs) throws Exception {
-
-        String libraryName = libraryDir.getName().trim();
+    private static void addLibraryToMetadata(Map<String, List<String>> uninstalledLibs, String dataverse,
+            String libraryName, ExternalLibrary library) throws ACIDException, RemoteException {
+        // Modify metadata accordingly
         List<String> uninstalledLibsInDv = uninstalledLibs.get(dataverse);
         // was this library just un-installed?
         boolean wasUninstalled = uninstalledLibsInDv != null && uninstalledLibsInDv.contains(libraryName);
@@ -214,23 +205,6 @@ public class ExternalLibraryUtils {
                 LOGGER.info("Added library " + libraryName + " to Metadata");
             }
 
-            // Get the descriptor
-            String[] libraryDescriptors = libraryDir.list((dir, name) -> name.endsWith(".xml"));
-
-            if (libraryDescriptors == null) {
-                throw new IOException("Unable to list files in directory " + libraryDir);
-            }
-
-            if (libraryDescriptors.length == 0) {
-                // should be fine. library was installed but its content was not added to metadata
-                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                return;
-            } else if (libraryDescriptors.length > 1) {
-                throw new IllegalStateException("More than 1 library descriptors defined");
-            }
-
-            ExternalLibrary library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
-
             // Get the dataverse
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
             if (dv == null) {
@@ -240,19 +214,22 @@ public class ExternalLibraryUtils {
             // Add functions
             if (library.getLibraryFunctions() != null) {
                 for (LibraryFunction function : library.getLibraryFunctions().getLibraryFunction()) {
-                    String[] fargs = function.getArguments().trim().split(",");
+                    String[] fargs = function.getArgumentType().trim().split(",");
+                    String functionFullName = getExternalFunctionFullName(libraryName, function.getName().trim());
+                    String functionReturnType = function.getReturnType().trim();
+                    String functionDefinition = function.getDefinition().trim();
+                    String functionLanguage = library.getLanguage().trim();
+                    String functionType = function.getFunctionType().trim();
                     List<String> args = new ArrayList<>();
                     for (String arg : fargs) {
                         args.add(arg);
                     }
-                    FunctionSignature signature = new FunctionSignature(dataverse,
-                            libraryName + "#" + function.getName().trim(), args.size());
-                    Function f = new Function(signature, args, function.getReturnType().trim(),
-                            function.getDefinition().trim(), library.getLanguage().trim(),
-                            function.getFunctionType().trim(), null);
+                    FunctionSignature signature = new FunctionSignature(dataverse, functionFullName, args.size());
+                    Function f = new Function(signature, args, functionReturnType, functionDefinition, functionLanguage,
+                            functionType, null);
                     MetadataManager.INSTANCE.addFunction(mdTxnCtx, f);
                     if (LOGGER.isInfoEnabled()) {
-                        LOGGER.info("Installed function: " + libraryName + "#" + function.getName().trim());
+                        LOGGER.info("Installed function: " + functionFullName);
                     }
                 }
             }
@@ -265,7 +242,7 @@ public class ExternalLibraryUtils {
             if (library.getLibraryAdapters() != null) {
                 for (LibraryAdapter adapter : library.getLibraryAdapters().getLibraryAdapter()) {
                     String adapterFactoryClass = adapter.getFactoryClass().trim();
-                    String adapterName = libraryName + "#" + adapter.getName().trim();
+                    String adapterName = getExternalFunctionFullName(libraryName, adapter.getName().trim());
                     AdapterIdentifier aid = new AdapterIdentifier(dataverse, adapterName);
                     DatasourceAdapter dsa =
                             new DatasourceAdapter(aid, adapterFactoryClass, IDataSourceAdapter.AdapterType.EXTERNAL);
@@ -289,14 +266,48 @@ public class ExternalLibraryUtils {
     }
 
     /**
+     * Each element of a library is installed as part of a transaction. Any
+     * failure in installing an element does not effect installation of other
+     * libraries.
+     */
+    protected static void configureLibrary(ILibraryManager libraryManager, String dataverse, final File libraryDir,
+            Map<String, List<String>> uninstalledLibs, boolean isMetadataNode) throws Exception {
+
+        String libraryName = libraryDir.getName().trim();
+        String[] libraryDescriptors = libraryDir.list((dir, name) -> name.endsWith(".xml"));
+
+        if (libraryDescriptors == null) {
+            throw new IOException("Unable to list files in directory " + libraryDir);
+        }
+
+        if (libraryDescriptors.length > 1) {
+            throw new IllegalStateException("More than 1 library descriptors defined");
+        }
+
+        // Prepare possible parameters
+        ExternalLibrary library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
+        if (library.getLibraryFunctions() != null) {
+            library.getLibraryFunctions().getLibraryFunction().forEach(fun -> {
+                if (fun.getParameters() != null) {
+                    libraryManager.addFunctionParameters(dataverse,
+                            getExternalFunctionFullName(libraryName, fun.getName()), fun.getParameters());
+                }
+            });
+        }
+        if (isMetadataNode) {
+            addLibraryToMetadata(uninstalledLibs, dataverse, libraryName, library);
+        }
+    }
+
+    /**
      * register the library class loader with the external library manager
      *
      * @param dataverse
      * @param libraryName
      * @throws Exception
      */
-    protected static void registerLibrary(ILibraryManager externalLibraryManager, String dataverse, String libraryName)
-            throws Exception {
+    protected static void registerClassLoader(ILibraryManager externalLibraryManager, String dataverse,
+            String libraryName) throws Exception {
         // get the class loader
         ClassLoader classLoader = getLibraryClassLoader(dataverse, libraryName);
         // register it with the external library manager
@@ -414,4 +425,8 @@ public class ExternalLibraryUtils {
         return uninstallDir;
     }
 
+    public static String getExternalFunctionFullName(String libraryName, String functionName) {
+        return libraryName + "#" + functionName;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/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
index 29a4b36..7246925 100644
--- 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
@@ -101,12 +101,10 @@ public class ExternalUDFLibrarian implements IExternalUDFLibrarian {
         }
 
         for (ILibraryManager libraryManager : libraryManagers) {
-            ExternalLibraryUtils.registerLibrary(libraryManager, dvName, libName);
+            ExternalLibraryUtils.registerClassLoader(libraryManager, dvName, libName);
+            ExternalLibraryUtils.configureLibrary(libraryManager, dvName, destinationDir, new HashMap<>(),
+                    libraryManagers.indexOf(libraryManager) != 0);
         }
-        // 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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.1.ddl.aql
new file mode 100644
index 0000000..e408cfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.1.ddl.aql
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+/*
+ * Description     : Detect ids with different settings. First name in list1. Last name in list2.
+ * Expected Result : Success
+ * Date            : 21th July 2016
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type InputRecordType as closed {
+id:int64,
+fname:string,
+lname:string,
+age:int64,
+dept:string
+}
+
+create type DetectResultType as open{
+id:int64,
+sensitive: boolean
+}
+
+create dataset EmpDataset(InputRecordType) primary key id;
+create dataset Res1(DetectResultType) primary key id;
+create dataset Res2(DetectResultType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.2.update.aql
new file mode 100644
index 0000000..8648266
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use dataverse test;
+
+load dataset EmpDataset
+using localfs
+(("path"="asterix_nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.3.lib.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.3.lib.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.3.lib.aql
new file mode 100644
index 0000000..dbdfe16
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.3.lib.aql
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install test testlib target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.4.update.aql
new file mode 100644
index 0000000..c485a34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.4.update.aql
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+use dataverse test;
+
+insert into dataset Res1(
+for $i in dataset EmpDataset
+return testlib#fnameDetector($i));
+
+insert into dataset Res2(
+for $i in dataset EmpDataset
+return testlib#lnameDetector($i));
+
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.5.query.aql
new file mode 100644
index 0000000..0bb82b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/external-library/keyword_detector/keyword_detector.5.query.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use dataverse test;
+
+for $i1 in dataset Res1
+for $i2 in dataset Res2
+where $i1.id = $i2.id and ($i1.sensitive or $i2.sensitive)
+order by $i1.id
+return $i1.id
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.ddl.sqlpp
new file mode 100644
index 0000000..4402130
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+/*
+ * Description     : Detect ids with different settings. First name in list1. Last name in list2.
+ * Expected Result : Success
+ * Date            : 21th July 2016
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type InputRecordType as closed {
+id:int64,
+fname:string,
+lname:string,
+age:int64,
+dept:string
+};
+
+create type DetectResultType as open{
+id:int64,
+sensitive: boolean
+};
+
+create dataset EmpDataset(InputRecordType) primary key id;
+create dataset Res1(DetectResultType) primary key id;
+create dataset Res2(DetectResultType) primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.2.update.sqlpp
new file mode 100644
index 0000000..4d1dc7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+load dataset EmpDataset
+using localfs
+(("path"="asterix_nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.3.lib.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.3.lib.sqlpp
new file mode 100644
index 0000000..dbdfe16
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.3.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install test testlib target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.4.update.sqlpp
new file mode 100644
index 0000000..f74b917
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.4.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into Res1(
+select value testlib#fnameDetector(t) from EmpDataset t);
+
+insert into Res2(
+select value testlib#lnameDetector(t2) from EmpDataset t2);
+
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.5.query.sqlpp
new file mode 100644
index 0000000..4c1273e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+SELECT value r1.id
+FROM Res1 r1, Res2 r2
+WHERE r1.id = r2.id AND (r1.sensitive OR r2.sensitive)
+ORDER BY r1.id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/keyword_detector/keyword_detector.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/keyword_detector/keyword_detector.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/keyword_detector/keyword_detector.1.adm
new file mode 100644
index 0000000..f26c702
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/keyword_detector/keyword_detector.1.adm
@@ -0,0 +1,5 @@
+711
+721
+751
+761
+1427
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
index 403237e..f0ad2b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
@@ -2,7 +2,9 @@
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#addHashTagsInPlace", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "org.apache.asterix.external.library.AddHashTagsInPlaceFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "org.apache.asterix.external.library.AllTypesFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "org.apache.asterix.external.library.EchoDelayFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#fnameDetector", "Arity": "1", "Params": [ "InputRecordType" ], "ReturnType": "DetectResultType", "Definition": "org.apache.asterix.external.library.KeywordsDetectorFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "org.apache.asterix.external.library.CapitalFinderFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#lnameDetector", "Arity": "1", "Params": [ "InputRecordType" ], "ReturnType": "DetectResultType", "Definition": "org.apache.asterix.external.library.KeywordsDetectorFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "org.apache.asterix.external.library.SumFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "org.apache.asterix.external.library.ParseTweetFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "org.apache.asterix.external.library.UpperCaseFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it.xml
index 3b0cc17..2b7d4f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it.xml
@@ -47,5 +47,10 @@
         <output-dir compare="Text">upperCase</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="keyword_detector">
+        <output-dir compare="Text">keyword_detector</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
----------------------------------------------------------------------
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 c6fa388..1d591e1 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
@@ -54,6 +54,11 @@
         <expected-error>External UDF cannot produce expected result.</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="keyword_detector">
+        <output-dir compare="Text">keyword_detector</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="feeds">
     <test-case FilePath="feeds">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
index 3aef2c5..ec02692 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
@@ -34,13 +34,13 @@ public interface ILibraryManager {
      * @param libraryName
      * @param classLoader
      */
-    public void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader)
+    void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader)
             throws HyracksDataException;
 
     /**
      * @return all registered libraries.
      */
-    public List<Pair<String, String>> getAllLibraries();
+    List<Pair<String, String>> getAllLibraries();
 
     /**
      * De-registers a library class loader.
@@ -48,7 +48,7 @@ public interface ILibraryManager {
      * @param dataverseName
      * @param libraryName
      */
-    public void deregisterLibraryClassLoader(String dataverseName, String libraryName);
+    void deregisterLibraryClassLoader(String dataverseName, String libraryName);
 
     /**
      * Finds a class loader for a given pair of dataverse name and library name.
@@ -57,5 +57,22 @@ public interface ILibraryManager {
      * @param libraryName
      * @return the library class loader associated with the dataverse and library.
      */
-    public ClassLoader getLibraryClassLoader(String dataverseName, String libraryName);
+    ClassLoader getLibraryClassLoader(String dataverseName, String libraryName);
+
+    /**
+     * Add function parameters  to library manager if it exists.
+     * @param dataverseName
+     * @param fullFunctionName
+     * @param parameters
+     */
+
+    void addFunctionParameters(String dataverseName, String fullFunctionName, List<String> parameters);
+
+    /**
+     * Get a list of parameters.
+     * @param dataverseName
+     * @param fullFunctionName
+     * @return A list contains all pre-specified function parameters.
+     */
+    List<String> getFunctionParameters(String dataverseName, String fullFunctionName);
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
index 9e57342..3be8b3a 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IFunctionHelper.java
@@ -21,17 +21,21 @@ package org.apache.asterix.external.api;
 import org.apache.asterix.external.library.java.JTypeTag;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
+import java.util.List;
+
 public interface IFunctionHelper {
 
-    public IJObject getArgument(int index);
+    IJObject getArgument(int index);
+
+    IJObject getResultObject();
 
-    public IJObject getResultObject();
+    void setResult(IJObject result) throws HyracksDataException;
 
-    public void setResult(IJObject result) throws HyracksDataException;
+    boolean isValidResult();
 
-    public boolean isValidResult();
+    IJObject getObject(JTypeTag jtypeTag) throws HyracksDataException;
 
-    public IJObject getObject(JTypeTag jtypeTag) throws HyracksDataException;
+    void reset();
 
-    public void reset();
+    List<String> getParameters();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
index 63028ac..3866665 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
@@ -63,12 +63,14 @@ public abstract class ExternalFunction implements IExternalFunction {
         for (int i = 0; i < args.length; i++) {
             argumentEvaluators[i] = args[i].createScalarEvaluator(context);
         }
-        functionHelper = new JavaFunctionHelper(finfo, resultBuffer);
 
+        ILibraryManager libraryManager = appCtx.getLibraryManager();
         String[] fnameComponents = finfo.getFunctionIdentifier().getName().split("#");
         String functionLibary = fnameComponents[0];
         String dataverse = finfo.getFunctionIdentifier().getNamespace();
-        ILibraryManager libraryManager = appCtx.getLibraryManager();
+
+        functionHelper = new JavaFunctionHelper(finfo, resultBuffer,
+                libraryManager.getFunctionParameters(dataverse, finfo.getFunctionIdentifier().getName()));
         ClassLoader libraryClassLoader = libraryManager.getLibraryClassLoader(dataverse, functionLibary);
         String classname = finfo.getFunctionBody().trim();
         Class<?> clazz;
@@ -94,7 +96,7 @@ public abstract class ExternalFunction implements IExternalFunction {
             argumentEvaluators[i].evaluate(tuple, inputVal);
 
             // Type-cast the source array based on the input type that this function wants to receive.
-            ATypeTag targetTypeTag = finfo.getParamList().get(i).getTypeTag();
+            ATypeTag targetTypeTag = finfo.getArgumentList().get(i).getTypeTag();
             ATypeTag sourceTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
                     .deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
             if (sourceTypeTag != targetTypeTag) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index 19a519f..b9ce2bd 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.external.library;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -31,6 +32,7 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 public class ExternalLibraryManager implements ILibraryManager {
 
     private final Map<String, ClassLoader> libraryClassLoaders = new HashMap<>();
+    private final Map<String, List<String>> externalFunctionParameters = new HashMap<>();
 
     @Override
     public void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader)
@@ -69,6 +71,16 @@ public class ExternalLibraryManager implements ILibraryManager {
         return libraryClassLoaders.get(key);
     }
 
+    @Override
+    public void addFunctionParameters(String dataverseName, String fullFunctionName, List<String> parameters) {
+        externalFunctionParameters.put(dataverseName + "." + fullFunctionName, parameters);
+    }
+
+    @Override
+    public List<String> getFunctionParameters(String dataverseName, String fullFunctionName) {
+        return externalFunctionParameters.getOrDefault(dataverseName + "." + fullFunctionName, Collections.emptyList());
+    }
+
     private static String getKey(String dataverseName, String libraryName) {
         return dataverseName + "." + libraryName;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
index 617bbbc..a250b4c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
@@ -20,6 +20,7 @@ package org.apache.asterix.external.library;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -52,27 +53,28 @@ public class JavaFunctionHelper implements IFunctionHelper {
     private final IDataOutputProvider outputProvider;
     private final IJObject[] arguments;
     private IJObject resultHolder;
-    private final IObjectPool<IJObject, IAType> objectPool =
-            new ListObjectPool<IJObject, IAType>(JTypeObjectFactory.INSTANCE);
+    private final IObjectPool<IJObject, IAType> objectPool = new ListObjectPool<>(JTypeObjectFactory.INSTANCE);
     private final JObjectPointableVisitor pointableVisitor;
     private final PointableAllocator pointableAllocator;
     private final Map<Integer, TypeInfo> poolTypeInfo;
+    private final List<String> parameters;
 
     private boolean isValidResult = false;
 
-    public JavaFunctionHelper(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider)
-            throws HyracksDataException {
+    public JavaFunctionHelper(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider,
+            List<String> parameters) {
         this.finfo = finfo;
         this.outputProvider = outputProvider;
         this.pointableVisitor = new JObjectPointableVisitor();
         this.pointableAllocator = new PointableAllocator();
-        this.arguments = new IJObject[finfo.getParamList().size()];
+        this.arguments = new IJObject[finfo.getArgumentList().size()];
         int index = 0;
-        for (IAType param : finfo.getParamList()) {
+        for (IAType param : finfo.getArgumentList()) {
             this.arguments[index++] = objectPool.allocate(param);
         }
         this.resultHolder = objectPool.allocate(finfo.getReturnType());
-        this.poolTypeInfo = new HashMap<Integer, TypeInfo>();
+        this.poolTypeInfo = new HashMap<>();
+        this.parameters = parameters;
 
     }
 
@@ -116,7 +118,7 @@ public class JavaFunctionHelper implements IFunctionHelper {
     public void setArgument(int index, IValueReference valueReference) throws IOException, AsterixException {
         IVisitablePointable pointable = null;
         IJObject jObject = null;
-        IAType type = finfo.getParamList().get(index);
+        IAType type = finfo.getArgumentList().get(index);
         switch (type.getTypeTag()) {
             case OBJECT:
                 pointable = pointableAllocator.allocateRecordValue(type);
@@ -193,4 +195,7 @@ public class JavaFunctionHelper implements IFunctionHelper {
         objectPool.reset();
     }
 
+    public List<String> getParameters() {
+        return parameters;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/RuntimeExternalFunctionUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/RuntimeExternalFunctionUtil.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/RuntimeExternalFunctionUtil.java
deleted file mode 100755
index 8b7b06e..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/RuntimeExternalFunctionUtil.java
+++ /dev/null
@@ -1,109 +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.external.library;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
-
-public class RuntimeExternalFunctionUtil {
-
-    private static Map<String, ClassLoader> libraryClassLoaders = new HashMap<>();
-
-    public static void registerLibraryClassLoader(String dataverseName, String libraryName, ClassLoader classLoader)
-            throws RuntimeDataException {
-        String key = dataverseName + "." + libraryName;
-        synchronized (libraryClassLoaders) {
-            if (libraryClassLoaders.get(dataverseName) != null) {
-                throw new RuntimeDataException(ErrorCode.LIBRARY_EXTERNAL_LIBRARY_CLASS_REGISTERED);
-            }
-            libraryClassLoaders.put(key, classLoader);
-        }
-    }
-
-    public static ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
-        String key = dataverseName + "." + libraryName;
-        synchronized (libraryClassLoaders) {
-            return libraryClassLoaders.get(key);
-        }
-    }
-
-    public static IFunctionDescriptor getFunctionDescriptor(IFunctionInfo finfo, IApplicationContext appCtx)
-            throws RuntimeDataException {
-        switch (((IExternalFunctionInfo) finfo).getKind()) {
-            case SCALAR:
-                return getScalarFunctionDescriptor(finfo, appCtx);
-            case AGGREGATE:
-            case UNNEST:
-            case STATEFUL:
-                throw new RuntimeDataException(ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_NAME,
-                        finfo.getFunctionIdentifier().getName());
-        }
-        return null;
-    }
-
-    private static AbstractScalarFunctionDynamicDescriptor getScalarFunctionDescriptor(IFunctionInfo finfo,
-            IApplicationContext appCtx) {
-        return new ExternalScalarFunctionDescriptor(finfo, appCtx);
-    }
-
-    public static ByteBuffer allocateArgumentBuffers(IAType type) {
-        switch (type.getTypeTag()) {
-            case INTEGER:
-                return ByteBuffer.allocate(4);
-            case STRING:
-                return ByteBuffer.allocate(32 * 1024);
-            default:
-                return ByteBuffer.allocate(32 * 1024);
-        }
-    }
-
-    public static IAObject allocateArgumentObjects(IAType type) {
-        switch (type.getTypeTag()) {
-            case INTEGER:
-                return new AMutableInt32(0);
-            case STRING:
-                return new AMutableString("");
-            default:
-                return null;
-            /*
-            ARecordType recordType = (ARecordType) type;
-            IAType[] fieldTypes = recordType.getFieldTypes();
-            IAObject[] fields = new IAObject[fieldTypes.length];
-            for (int i = 0; i < fields.length; i++) {
-                fields[i] = allocateArgumentObjects(fieldTypes[i]);
-            }
-            return new AMutableRecord((ARecordType) type, fields);
-            */
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/main/resources/schema/library.xsd
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/resources/schema/library.xsd b/asterixdb/asterix-external-data/src/main/resources/schema/library.xsd
index 75a0801..edf23ec 100644
--- a/asterixdb/asterix-external-data/src/main/resources/schema/library.xsd
+++ b/asterixdb/asterix-external-data/src/main/resources/schema/library.xsd
@@ -25,11 +25,12 @@
     <!-- definition of simple types -->
     <xs:element name="language" type="xs:string" />
     <xs:element name="name" type="xs:string" />
-    <xs:element name="arguments" type="xs:string" />
+    <xs:element name="argument_type" type="xs:string" />
     <xs:element name="return_type" type="xs:string" />
     <xs:element name="function_type" type="xs:string" />
     <xs:element name="definition" type="xs:string" />
     <xs:element name="factory_class" type="xs:string" />
+    <xs:element name="parameters" type="xs:string" />
 
 
     <!-- definition of complex elements -->
@@ -38,9 +39,10 @@
             <xs:sequence>
                 <xs:element ref="lib:name" />
                 <xs:element ref="lib:function_type" />
-                <xs:element ref="lib:arguments" />
+                <xs:element ref="lib:argument_type" />
                 <xs:element ref="lib:return_type" />
                 <xs:element ref="lib:definition" />
+                <xs:element ref="lib:parameters" minOccurs="0" maxOccurs="unbounded"/>
             </xs:sequence>
         </xs:complexType>
     </xs:element>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFactory.java
new file mode 100644
index 0000000..f75033d
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFactory.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.external.library;
+
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
+
+public class KeywordsDetectorFactory implements IFunctionFactory {
+
+    @Override
+    public IExternalScalarFunction getExternalFunction() {
+        return new KeywordsDetectorFunction();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFunction.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFunction.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFunction.java
new file mode 100644
index 0000000..d12f080
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/KeywordsDetectorFunction.java
@@ -0,0 +1,67 @@
+/*
+ * 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.external.library;
+
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
+import org.apache.asterix.external.library.java.JObjects;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+public class KeywordsDetectorFunction implements IExternalScalarFunction {
+
+    private ArrayList<String> keywordsList;
+    private String dictPath, fieldName;
+    private List<String> functionParameters;
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        JObjects.JRecord inputRecord = (JObjects.JRecord) functionHelper.getArgument(0);
+        JObjects.JRecord outputRecord = (JObjects.JRecord) functionHelper.getResultObject();
+        JObjects.JBoolean chkVal = new JObjects.JBoolean(false);
+        String fieldValue = ((JObjects.JString) inputRecord.getValueByName(fieldName)).getValue();
+
+        chkVal.setValue(keywordsList.contains(fieldValue));
+
+        outputRecord.setField("id", inputRecord.getValueByName("id"));
+        outputRecord.setField("sensitive", chkVal);
+        functionHelper.setResult(outputRecord);
+    }
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) throws Exception {
+        keywordsList = new ArrayList<>();
+        functionParameters = functionHelper.getParameters();
+        if (functionParameters.size() < 2) {
+            throw new IllegalArgumentException("Expected more parameters. Please check your UDF configuration.");
+        }
+        dictPath = functionParameters.get(0);
+        fieldName = functionParameters.get(1);
+        Files.lines(Paths.get(dictPath)).forEach(keyword -> keywordsList.add(keyword));
+    }
+
+    @Override
+    public void deinitialize() {
+        // no op
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml b/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
index 4658a57..6b59041 100644
--- a/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
+++ b/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
@@ -20,65 +20,83 @@
   <language>JAVA</language>
   <libraryFunctions>
     <libraryFunction>
+      <name>fnameDetector</name>
       <function_type>SCALAR</function_type>
+      <argument_type>InputRecordType</argument_type>
+      <return_type>DetectResultType</return_type>
+      <definition>org.apache.asterix.external.library.KeywordsDetectorFactory</definition>
+      <parameters>data/external_function/KeywordsDetector_List1.txt</parameters>
+      <parameters>fname</parameters>
+    </libraryFunction>
+    <libraryFunction>
+      <name>lnameDetector</name>
+      <function_type>SCALAR</function_type>
+      <argument_type>InputRecordType</argument_type>
+      <return_type>DetectResultType</return_type>
+      <definition>org.apache.asterix.external.library.KeywordsDetectorFactory</definition>
+      <parameters>data/external_function/KeywordsDetector_List2.txt</parameters>
+      <parameters>lname</parameters>
+    </libraryFunction>
+    <libraryFunction>
       <name>parseTweet</name>
-      <arguments>TweetInputType</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>TweetInputType</argument_type>
       <return_type>TweetOutputType</return_type>
       <definition>org.apache.asterix.external.library.ParseTweetFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>addHashTags</name>
-      <arguments>Tweet</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>Tweet</argument_type>
       <return_type>ProcessedTweet</return_type>
       <definition>org.apache.asterix.external.library.AddHashTagsFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>addHashTagsInPlace</name>
-      <arguments>Tweet</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>Tweet</argument_type>
       <return_type>ProcessedTweet</return_type>
       <definition>org.apache.asterix.external.library.AddHashTagsInPlaceFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>mysum</name>
-      <arguments>AINT32,AINT32</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>AINT32,AINT32</argument_type>
       <return_type>AINT32</return_type>
       <definition>org.apache.asterix.external.library.SumFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>getCapital</name>
-      <arguments>ASTRING</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>ASTRING</argument_type>
       <return_type>CountryCapitalType</return_type>
       <definition>org.apache.asterix.external.library.CapitalFinderFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>toUpper</name>
-      <arguments>TextType</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>TextType</argument_type>
       <return_type>TextType</return_type>
       <definition>org.apache.asterix.external.library.UpperCaseFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>allTypes</name>
-      <arguments>AllType</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>AllType</argument_type>
       <return_type>AllType</return_type>
       <definition>org.apache.asterix.external.library.AllTypesFactory
       </definition>
     </libraryFunction>
     <libraryFunction>
-      <function_type>SCALAR</function_type>
       <name>echoDelay</name>
-      <arguments>TweetMessageType</arguments>
+      <function_type>SCALAR</function_type>
+      <argument_type>TweetMessageType</argument_type>
       <return_type>TweetMessageType</return_type>
       <definition>org.apache.asterix.external.library.EchoDelayFactory
       </definition>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
index eb9631a..7dd1dc2 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
@@ -46,7 +46,7 @@ public class FunctionParser {
                     function.getLanguage());
         }
         String functionBody = function.getFunctionBody();
-        List<String> params = function.getParams();
+        List<String> arguments = function.getArguments();
         List<VarIdentifier> varIdentifiers = new ArrayList<VarIdentifier>();
 
         StringBuilder builder = new StringBuilder();
@@ -54,15 +54,15 @@ public class FunctionParser {
         builder.append(" declare function " + function.getName().split("@")[0]);
         builder.append("(");
         boolean first = true;
-        for (String param : params) {
-            VarIdentifier varId = new VarIdentifier(param);
+        for (String argument : arguments) {
+            VarIdentifier varId = new VarIdentifier(argument);
             varIdentifiers.add(varId);
             if (first) {
                 first = false;
             } else {
                 builder.append(",");
             }
-            builder.append(param);
+            builder.append(argument);
         }
         builder.append("){\n").append(functionBody).append("\n}");
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
index 4c0aa24..999aa7a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
@@ -46,7 +46,7 @@ public class FunctionParser {
                     Function.LANGUAGE_SQLPP, function.getLanguage());
         }
         String functionBody = function.getFunctionBody();
-        List<String> params = function.getParams();
+        List<String> params = function.getArguments();
 
         StringBuilder builder = new StringBuilder();
         builder.append(" use " + function.getDataverseName() + ";");

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
index b5721c0..128e704 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
@@ -35,17 +35,17 @@ public class Function implements IMetadataEntity<Function> {
     public static final String NOT_APPLICABLE = "N/A";
 
     private final FunctionSignature signature;
-    private final List<String> params;
     private final List<List<List<String>>> dependencies;
+    private final List<String> arguments;
     private final String body;
     private final String returnType;
     private final String language;
     private final String kind;
 
-    public Function(FunctionSignature signature, List<String> params, String returnType, String functionBody,
+    public Function(FunctionSignature signature, List<String> arguments, String returnType, String functionBody,
             String language, String functionKind, List<List<List<String>>> dependencies) {
         this.signature = signature;
-        this.params = params;
+        this.arguments = arguments;
         this.body = functionBody;
         this.returnType = returnType == null ? RETURNTYPE_VOID : returnType;
         this.language = language;
@@ -75,8 +75,8 @@ public class Function implements IMetadataEntity<Function> {
         return signature.getArity();
     }
 
-    public List<String> getParams() {
-        return params;
+    public List<String> getArguments() {
+        return arguments;
     }
 
     public List<List<List<String>>> getDependencies() {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index 120aa08..9085235 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -183,7 +183,7 @@ public class FunctionTupleTranslator extends AbstractTupleTranslator<Function> {
         ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
         listBuilder.reset((AOrderedListType) MetadataRecordTypes.FUNCTION_RECORDTYPE
                 .getFieldTypes()[MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_PARAM_LIST_FIELD_INDEX]);
-        for (String param : function.getParams()) {
+        for (String param : function.getArguments()) {
             itemValue.reset();
             aString.setValue(param);
             stringSerde.serialize(aString, itemValue.getDataOutput());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 3d80d0f..c2b7bb6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -73,11 +73,11 @@ public class ExternalFunctionCompilerUtil {
         FunctionIdentifier fid =
                 new FunctionIdentifier(function.getDataverseName(), function.getName(), function.getArity());
         IResultTypeComputer typeComputer = getResultTypeComputer(txnCtx, function);
-        List<IAType> arguments = new ArrayList<IAType>();
-        IAType returnType = null;
-        List<String> paramTypes = function.getParams();
-        for (String paramType : paramTypes) {
-            arguments.add(getTypeInfo(paramType, txnCtx, function));
+        List<IAType> arguments = new ArrayList<>();
+        IAType returnType;
+        List<String> argumentTypes = function.getArguments();
+        for (String argumentType : argumentTypes) {
+            arguments.add(getTypeInfo(argumentType, txnCtx, function));
         }
 
         returnType = getTypeInfo(function.getReturnType(), txnCtx, function);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
index 1bbfc43..af1bc7c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
@@ -60,7 +60,7 @@ public class ExternalFunctionInfo extends FunctionInfo implements IExternalFunct
     }
 
     @Override
-    public List<IAType> getParamList() {
+    public List<IAType> getArgumentList() {
         return argumentTypes;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
index d2d6339..3ccb66b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
@@ -33,7 +33,7 @@ public interface IExternalFunctionInfo extends IFunctionInfo {
 
     public String getFunctionBody();
 
-    public List<IAType> getParamList();
+    public List<IAType> getArgumentList();
 
     public String getLanguage();
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03ddd0cd/asterixdb/asterix-server/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm b/asterixdb/asterix-server/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
index ef563f1..20dc8c8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
@@ -2,7 +2,9 @@
 { "DataverseName": "externallibtest", "Name": "testlib#addHashTagsInPlace", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "org.apache.asterix.external.library.AddHashTagsInPlaceFactory", "Language": "JAVA", "Kind": "SCALAR" }
 { "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "org.apache.asterix.external.library.AllTypesFactory", "Language": "JAVA", "Kind": "SCALAR" }
 { "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "org.apache.asterix.external.library.EchoDelayFactory", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#fnameDetector", "Arity": "1", "Params": [ "InputRecordType" ], "ReturnType": "DetectResultType", "Definition": "org.apache.asterix.external.library.KeywordsDetectorFactory", "Language": "JAVA", "Kind": "SCALAR" }
 { "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "org.apache.asterix.external.library.CapitalFinderFactory", "Language": "JAVA", "Kind": "SCALAR" }
+{ "DataverseName": "externallibtest", "Name": "testlib#lnameDetector", "Arity": "1", "Params": [ "InputRecordType" ], "ReturnType": "DetectResultType", "Definition": "org.apache.asterix.external.library.KeywordsDetectorFactory", "Language": "JAVA", "Kind": "SCALAR" }
 { "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "org.apache.asterix.external.library.SumFactory", "Language": "JAVA", "Kind": "SCALAR" }
 { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "org.apache.asterix.external.library.ParseTweetFactory", "Language": "JAVA", "Kind": "SCALAR" }
 { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "org.apache.asterix.external.library.UpperCaseFactory", "Language": "JAVA", "Kind": "SCALAR" }