You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kh...@apache.org on 2016/10/06 00:18:45 UTC

[16/38] hive git commit: HIVE-14819: FunctionInfo for permanent functions shows TEMPORARY FunctionType (Jason Dere, reviewed by Sergey Shelukhin)

HIVE-14819: FunctionInfo for permanent functions shows TEMPORARY FunctionType (Jason Dere, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/74a6ff67
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/74a6ff67
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/74a6ff67

Branch: refs/heads/repl2
Commit: 74a6ff678f9312d946a7d55d73bf4a60127de763
Parents: 474425a
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Sep 29 13:35:17 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Sep 29 13:35:17 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionInfo.java       |  20 ++--
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   3 +-
 .../apache/hadoop/hive/ql/exec/Registry.java    |  96 ++++++++++++++----
 .../hadoop/hive/ql/exec/WindowFunctionInfo.java |   4 +-
 .../translator/SqlFunctionConverter.java        |   2 +-
 .../hive/ql/exec/TestFunctionRegistry.java      | 100 +++++++++++++++++++
 6 files changed, 194 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
index 30ba996..8014dab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
@@ -73,36 +73,36 @@ public class FunctionInfo {
     this.discarded = new AtomicBoolean(false);  // shared to all session functions
   }
 
-  public FunctionInfo(boolean isNative, String displayName,
+  public FunctionInfo(FunctionType functionType, String displayName,
       GenericUDF genericUDF, FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.genericUDF = genericUDF;
     this.isInternalTableFunction = false;
     this.resources = resources;
   }
 
-  public FunctionInfo(boolean isNative, String displayName,
+  public FunctionInfo(FunctionType functionType, String displayName,
       GenericUDAFResolver genericUDAFResolver, FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.genericUDAFResolver = genericUDAFResolver;
     this.isInternalTableFunction = false;
     this.resources = resources;
   }
 
-  public FunctionInfo(boolean isNative, String displayName,
+  public FunctionInfo(FunctionType functionType, String displayName,
       GenericUDTF genericUDTF, FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.genericUDTF = genericUDTF;
     this.isInternalTableFunction = false;
     this.resources = resources;
   }
 
-  public FunctionInfo(boolean isNative, String displayName, Class<? extends TableFunctionResolver> tFnCls,
+  public FunctionInfo(FunctionType functionType, String displayName, Class<? extends TableFunctionResolver> tFnCls,
       FunctionResource... resources) {
-    this.functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    this.functionType = functionType;
     this.displayName = displayName;
     this.tableFunctionResolver = tFnCls;
     PartitionTableFunctionDescription def = AnnotationUtils.getAnnotation(
@@ -263,6 +263,10 @@ public class FunctionInfo {
     }
   }
 
+  public FunctionType getFunctionType() {
+    return functionType;
+  }
+
   public static class FunctionResource {
     private final SessionState.ResourceType resourceType;
     private final String resourceURI;

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index de74c3e..b277f5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -1535,7 +1535,8 @@ public final class FunctionRegistry {
     }
 
     if (clazz != null) {
-      return system.isPermanentFunc(clazz);
+      // Use session registry - see Registry.isPermanentFunc()
+      return SessionState.getRegistryForWrite().isPermanentFunc(clazz);
     }
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
index 05926b5..c274933 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -104,31 +105,37 @@ public class Registry {
    * @return true if udfClass's type was recognized (so registration
    *         succeeded); false otherwise
    */
-  @SuppressWarnings("unchecked")
   public FunctionInfo registerFunction(
       String functionName, Class<?> udfClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerFunction(functionName, functionType, udfClass, resources);
+  }
+
+  @SuppressWarnings("unchecked")
+  private FunctionInfo registerFunction(
+      String functionName, FunctionType functionType, Class<?> udfClass, FunctionResource... resources) {
 
     FunctionUtils.UDFClassType udfClassType = FunctionUtils.getUDFClassType(udfClass);
     switch (udfClassType) {
       case UDF:
         return registerUDF(
-            functionName, (Class<? extends UDF>) udfClass, false, resources);
+            functionName, functionType, (Class<? extends UDF>) udfClass, false, functionName.toLowerCase(), resources);
       case GENERIC_UDF:
         return registerGenericUDF(
-            functionName, (Class<? extends GenericUDF>) udfClass, resources);
+            functionName, functionType, (Class<? extends GenericUDF>) udfClass, resources);
       case GENERIC_UDTF:
         return registerGenericUDTF(
-            functionName, (Class<? extends GenericUDTF>) udfClass, resources);
+            functionName, functionType, (Class<? extends GenericUDTF>) udfClass, resources);
       case UDAF:
         return registerUDAF(
-            functionName, (Class<? extends UDAF>) udfClass, resources);
+            functionName, functionType, (Class<? extends UDAF>) udfClass, resources);
       case GENERIC_UDAF_RESOLVER:
         return registerGenericUDAF(
-            functionName, (GenericUDAFResolver)
-            ReflectionUtil.newInstance(udfClass, null), resources);
+            functionName, functionType,
+            (GenericUDAFResolver) ReflectionUtil.newInstance(udfClass, null), resources);
       case TABLE_FUNCTION_RESOLVER:
         // native or not would be decided by annotation. need to evaluate that first
-        return registerTableFunction(functionName,
+        return registerTableFunction(functionName, functionType,
             (Class<? extends TableFunctionResolver>) udfClass, resources);
     }
     return null;
@@ -143,8 +150,15 @@ public class Registry {
   public FunctionInfo registerUDF(String functionName,
       Class<? extends UDF> UDFClass, boolean isOperator, String displayName,
       FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerUDF(functionName, functionType, UDFClass, isOperator, displayName);
+  }
+
+  private FunctionInfo registerUDF(String functionName, FunctionType functionType,
+      Class<? extends UDF> UDFClass, boolean isOperator, String displayName,
+      FunctionResource... resources) {
     validateClass(UDFClass, UDF.class);
-    FunctionInfo fI = new FunctionInfo(isNative, displayName,
+    FunctionInfo fI = new FunctionInfo(functionType, displayName,
         new GenericUDFBridge(displayName, isOperator, UDFClass.getName()), resources);
     addFunction(functionName, fI);
     return fI;
@@ -152,8 +166,14 @@ public class Registry {
 
   public FunctionInfo registerGenericUDF(String functionName,
       Class<? extends GenericUDF> genericUDFClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerGenericUDF(functionName, functionType, genericUDFClass, resources);
+  }
+
+  private FunctionInfo registerGenericUDF(String functionName, FunctionType functionType,
+      Class<? extends GenericUDF> genericUDFClass, FunctionResource... resources) {
     validateClass(genericUDFClass, GenericUDF.class);
-    FunctionInfo fI = new FunctionInfo(isNative, functionName,
+    FunctionInfo fI = new FunctionInfo(functionType, functionName,
         ReflectionUtil.newInstance(genericUDFClass, null), resources);
     addFunction(functionName, fI);
     return fI;
@@ -177,8 +197,14 @@ public class Registry {
 
   public FunctionInfo registerGenericUDTF(String functionName,
       Class<? extends GenericUDTF> genericUDTFClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerGenericUDTF(functionName, functionType, genericUDTFClass, resources);
+  }
+
+  private FunctionInfo registerGenericUDTF(String functionName, FunctionType functionType,
+      Class<? extends GenericUDTF> genericUDTFClass, FunctionResource... resources) {
     validateClass(genericUDTFClass, GenericUDTF.class);
-    FunctionInfo fI = new FunctionInfo(isNative, functionName,
+    FunctionInfo fI = new FunctionInfo(functionType, functionName,
         ReflectionUtil.newInstance(genericUDTFClass, null), resources);
     addFunction(functionName, fI);
     return fI;
@@ -186,8 +212,14 @@ public class Registry {
 
   public FunctionInfo registerGenericUDAF(String functionName,
       GenericUDAFResolver genericUDAFResolver, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerGenericUDAF(functionName, functionType, genericUDAFResolver, resources);
+  }
+
+  private FunctionInfo registerGenericUDAF(String functionName, FunctionType functionType,
+      GenericUDAFResolver genericUDAFResolver, FunctionResource... resources) {
     FunctionInfo function =
-        new WindowFunctionInfo(isNative, functionName, genericUDAFResolver, resources);
+        new WindowFunctionInfo(functionType, functionName, genericUDAFResolver, resources);
     addFunction(functionName, function);
     addFunction(WINDOW_FUNC_PREFIX + functionName, function);
     return function;
@@ -195,8 +227,14 @@ public class Registry {
 
   public FunctionInfo registerUDAF(String functionName,
       Class<? extends UDAF> udafClass, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerUDAF(functionName, functionType, udafClass, resources);
+  }
+
+  private FunctionInfo registerUDAF(String functionName, FunctionType functionType,
+      Class<? extends UDAF> udafClass, FunctionResource... resources) {
     validateClass(udafClass, UDAF.class);
-    FunctionInfo function = new WindowFunctionInfo(isNative, functionName,
+    FunctionInfo function = new WindowFunctionInfo(functionType, functionName,
         new GenericUDAFBridge(ReflectionUtil.newInstance(udafClass, null)), resources);
     addFunction(functionName, function);
     addFunction(WINDOW_FUNC_PREFIX + functionName, function);
@@ -205,8 +243,14 @@ public class Registry {
 
   public FunctionInfo registerTableFunction(String functionName,
       Class<? extends TableFunctionResolver> tFnCls, FunctionResource... resources) {
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    return registerTableFunction(functionName, functionType, tFnCls, resources);
+  }
+
+  private FunctionInfo registerTableFunction(String functionName, FunctionType functionType,
+      Class<? extends TableFunctionResolver> tFnCls, FunctionResource... resources) {
     validateClass(tFnCls, TableFunctionResolver.class);
-    FunctionInfo function = new FunctionInfo(isNative, functionName, tFnCls, resources);
+    FunctionInfo function = new FunctionInfo(functionType, functionName, tFnCls, resources);
     addFunction(functionName, function);
     return function;
   }
@@ -219,7 +263,7 @@ public class Registry {
   public FunctionInfo registerMacro(String macroName, ExprNodeDesc body,
       List<String> colNames, List<TypeInfo> colTypes, FunctionResource... resources) {
     GenericUDFMacro macro = new GenericUDFMacro(macroName, body, colNames, colTypes);
-    FunctionInfo fI = new FunctionInfo(isNative, macroName, macro, resources);
+    FunctionInfo fI = new FunctionInfo(FunctionType.TEMPORARY, macroName, macro, resources);
     addFunction(macroName, fI);
     return fI;
   }
@@ -252,7 +296,8 @@ public class Registry {
    * @param wFn
    */
   void registerWindowFunction(String name, GenericUDAFResolver wFn) {
-    addFunction(WINDOW_FUNC_PREFIX + name, new WindowFunctionInfo(isNative, name, wFn, null));
+    FunctionType functionType = isNative ? FunctionType.BUILTIN : FunctionType.TEMPORARY;
+    addFunction(WINDOW_FUNC_PREFIX + name, new WindowFunctionInfo(functionType, name, wFn, null));
   }
 
   private void validateClass(Class input, Class expected) {
@@ -310,6 +355,12 @@ public class Registry {
   }
 
   public boolean isPermanentFunc(Class<?> udfClass) {
+    // Note that permanent functions can only be properly checked from the session registry.
+    // If permanent functions are read from the metastore during Hive initialization,
+    // the JARs are not loaded for the UDFs during that time and so Hive is unable to instantiate
+    // the UDf classes to add to the persistent functions set.
+    // Once a permanent UDF has been referenced in a session its FunctionInfo should be registered
+    // in the session registry (and persistent set updated), so it can be looked up there.
     return udfClass != null && persistent.containsKey(udfClass);
   }
 
@@ -448,7 +499,10 @@ public class Registry {
   private void addFunction(String functionName, FunctionInfo function) {
     lock.lock();
     try {
-      if (isNative != function.isNative()) {
+      // Built-in functions shouldn't go in the session registry,
+      // and temp functions shouldn't go in the system registry.
+      // Persistent functions can be in either registry.
+      if ((!isNative && function.isBuiltIn()) || (isNative && !function.isNative())) {
         throw new RuntimeException("Function " + functionName + " is not for this registry");
       }
       functionName = functionName.toLowerCase();
@@ -463,7 +517,8 @@ public class Registry {
       mFunctions.put(functionName, function);
       if (function.isBuiltIn()) {
         builtIns.add(function.getFunctionClass());
-      } else if (function.isPersistent()) {
+      } else if (function.isPersistent() && !isNative) {
+        // System registry should not be used to check persistent functions - see isPermanentFunc()
         Class<?> functionClass = getPermanentUdfClass(function);
         Integer refCount = persistent.get(functionClass);
         persistent.put(functionClass, Integer.valueOf(refCount == null ? 1 : refCount + 1));
@@ -597,7 +652,10 @@ public class Registry {
       ClassLoader loader = Utilities.getSessionSpecifiedClassLoader();
       Class<?> udfClass = Class.forName(function.getClassName(), true, loader);
 
-      ret = FunctionRegistry.registerTemporaryUDF(qualifiedName, udfClass, resources);
+      // Make sure the FunctionInfo is listed as PERSISTENT (rather than TEMPORARY)
+      // when it is registered to the system registry.
+      ret = SessionState.getRegistryForWrite().registerFunction(
+          qualifiedName, FunctionType.PERSISTENT, udfClass, resources);
       if (ret == null) {
         LOG.error(function.getClassName() + " is not a valid UDF class and was not registered.");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
index a16d9e5..5e57ccd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
@@ -28,9 +28,9 @@ public class WindowFunctionInfo extends FunctionInfo {
   private final boolean pivotResult;
   private final boolean impliesOrder;
 
-  public WindowFunctionInfo(boolean isNative, String functionName,
+  public WindowFunctionInfo(FunctionType functionType, String functionName,
       GenericUDAFResolver resolver, FunctionResource[] resources) {
-    super(isNative, functionName, resolver, resources);
+    super(functionType, functionName, resolver, resources);
     WindowFunctionDescription def =
         AnnotationUtils.getAnnotation(resolver.getClass(), WindowFunctionDescription.class);
     supportsWindow = def == null ? true : def.supportsWindow();

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 911b86b..53e4a2a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -191,7 +191,7 @@ public class SqlFunctionConverter {
       throw new RuntimeException(e);
     }
     return new FunctionInfo(
-        fi.isNative(), fi.getDisplayName(), (GenericUDF) udf, fi.getResources());
+        fi.getFunctionType(), fi.getDisplayName(), (GenericUDF) udf, fi.getResources());
   }
 
   // TODO: 1) handle Agg Func Name translation 2) is it correct to add func

http://git-wip-us.apache.org/repos/asf/hive/blob/74a6ff67/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
index d2d5a1b..2192a89 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
@@ -27,7 +27,17 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFLn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentTimestamp;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -406,4 +416,94 @@ public class TestFunctionRegistry extends TestCase {
     Assert.assertTrue(FunctionRegistry.impliesOrder("lag"));
     Assert.assertFalse(FunctionRegistry.impliesOrder("min"));
   }
+
+  public void testRegisterTemporaryFunctions() throws Exception {
+    FunctionResource[] emptyResources = new FunctionResource[] {};
+
+    // UDF
+    FunctionRegistry.registerTemporaryUDF("tmp_ln", UDFLn.class, emptyResources);
+    FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("tmp_ln");
+    assertFalse(functionInfo.isNative());
+
+    // GenericUDF
+    FunctionRegistry.registerTemporaryUDF("tmp_concat", GenericUDFConcat.class, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("tmp_concat");
+    assertFalse(functionInfo.isNative());
+
+    // GenericUDAF
+    FunctionRegistry.registerTemporaryUDF("tmp_max",GenericUDAFMax.class, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("tmp_max");
+    assertFalse(functionInfo.isNative());
+    functionInfo = FunctionRegistry.getWindowFunctionInfo("tmp_max");
+    assertFalse(functionInfo.isNative());
+
+    // UDTF
+    FunctionRegistry.registerTemporaryUDF("tmp_explode", GenericUDTFExplode.class, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("tmp_explode");
+    assertFalse(functionInfo.isNative());
+  }
+
+  public void testRegisterPermanentFunction() throws Exception {
+    FunctionResource[] emptyResources = new FunctionResource[] {};
+
+    // UDF
+    FunctionRegistry.registerPermanentFunction("perm_ln", UDFLn.class.getName(), true, emptyResources);
+    FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("perm_ln");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_ln");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+
+    // GenericUDF
+    FunctionRegistry.registerPermanentFunction("default.perm_concat",
+        GenericUDFConcat.class.getName(), true, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_concat");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+
+    // GenericUDAF
+    FunctionRegistry.registerPermanentFunction("default.perm_max",
+        GenericUDAFMax.class.getName(), true, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_max");
+    assertTrue(functionInfo.isPersistent());
+    functionInfo = FunctionRegistry.getWindowFunctionInfo("default.perm_max");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+
+    // UDTF
+    FunctionRegistry.registerPermanentFunction("default.perm_explode",
+        GenericUDTFExplode.class.getName(), true, emptyResources);
+    functionInfo = FunctionRegistry.getFunctionInfo("default.perm_explode");
+    assertTrue(functionInfo.isPersistent());
+    assertTrue(functionInfo.isNative());
+    assertFalse(functionInfo.isBuiltIn());
+  }
+
+  public void testBuiltInFunction() throws Exception {
+    FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("ln");
+    assertTrue(functionInfo.isBuiltIn());
+    assertTrue(functionInfo.isNative());
+  }
+
+  public void testIsPermanentFunction() throws Exception {
+    // Setup exprNode
+    GenericUDF udf = new GenericUDFCurrentTimestamp();
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+    ExprNodeGenericFuncDesc fnExpr =
+        new ExprNodeGenericFuncDesc(TypeInfoFactory.timestampTypeInfo, udf, children);
+
+    assertFalse("Function not added as permanent yet", FunctionRegistry.isPermanentFunction(fnExpr));
+
+    // Now register as permanent function
+    FunctionResource[] emptyResources = new FunctionResource[] {};
+    FunctionRegistry.registerPermanentFunction("default.perm_current_timestamp",
+        GenericUDFCurrentTimestamp.class.getName(), true, emptyResources);
+
+    assertTrue("Function should now be recognized as permanent function", FunctionRegistry.isPermanentFunction(fnExpr));
+  }
 }