You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/02/20 00:50:34 UTC

svn commit: r1569985 [8/8] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/ itests/util/src/main/java/org/apache/hadoop/hive/ql/ metastore/if/ metastore/src/gen/thrift/gen-cpp/ metastore/src/gen/thrift/gen-javabean/org...

Added: hive/trunk/metastore/src/model/org/apache/hadoop/hive/metastore/model/MFunction.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/model/org/apache/hadoop/hive/metastore/model/MFunction.java?rev=1569985&view=auto
==============================================================================
--- hive/trunk/metastore/src/model/org/apache/hadoop/hive/metastore/model/MFunction.java (added)
+++ hive/trunk/metastore/src/model/org/apache/hadoop/hive/metastore/model/MFunction.java Wed Feb 19 23:50:32 2014
@@ -0,0 +1,107 @@
+/**
+ * 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.hadoop.hive.metastore.model;
+
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+
+public class MFunction {
+
+  private String functionName;
+  private MDatabase database;
+  private String className;
+  private String ownerName;
+  private String ownerType;
+  private int createTime;
+  private int functionType;
+
+  public MFunction() {
+  }
+
+  public MFunction(String functionName,
+      MDatabase database,
+      String className,
+      String ownerName,
+      String ownerType,
+      int createTime,
+      int functionType) {
+    this.setFunctionName(functionName);
+    this.setDatabase(database);
+    this.setFunctionType(functionType);
+    this.setClassName(className);
+    this.setOwnerName(ownerName);
+    this.setOwnerType(ownerType);
+    this.setCreateTime(createTime);
+  }
+
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  public void setFunctionName(String functionName) {
+    this.functionName = functionName;
+  }
+
+  public MDatabase getDatabase() {
+    return database;
+  }
+
+  public void setDatabase(MDatabase database) {
+    this.database = database;
+  }
+
+  public String getClassName() {
+    return className;
+  }
+
+  public void setClassName(String className) {
+    this.className = className;
+  }
+
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  public void setOwnerName(String owner) {
+    this.ownerName = owner;
+  }
+
+  public String getOwnerType() {
+    return ownerType;
+  }
+
+  public void setOwnerType(String ownerType) {
+    this.ownerType = ownerType;
+  }
+
+  public int getCreateTime() {
+    return createTime;
+  }
+
+  public void setCreateTime(int createTime) {
+    this.createTime = createTime;
+  }
+
+  public int getFunctionType() {
+    return functionType;
+  }
+
+  public void setFunctionType(int functionType) {
+    this.functionType = functionType;
+  }
+}

Modified: hive/trunk/metastore/src/model/package.jdo
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/model/package.jdo?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/metastore/src/model/package.jdo (original)
+++ hive/trunk/metastore/src/model/package.jdo Wed Feb 19 23:50:32 2014
@@ -917,6 +917,37 @@
       </field>
     </class>
 
+    <class name="MFunction" table="FUNCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="FUNC_ID"/>
+      </datastore-identity>
+      <index name="UniqueFunction" unique="true">
+        <column name="FUNC_NAME"/>
+        <column name="DB_ID"/>
+      </index>
+      <field name="functionName">
+        <column name="FUNC_NAME" length="128" jdbc-type="VARCHAR"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID"/>
+      </field>
+      <field name="functionType">
+        <column name="FUNC_TYPE" jdbc-type="integer"/>
+      </field>
+      <field name="className">
+        <column name="CLASS_NAME" length="4000" jdbc-type="VARCHAR"/>
+      </field>
+      <field name="ownerName">	
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR"/>
+      </field>
+       <field name="ownerType">
+        <column name="OWNER_TYPE" length="10" jdbc-type="VARCHAR"/>
+      </field>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer"/>
+      </field>
+    </class>
+
   </package>
 </jdo>
 

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java Wed Feb 19 23:50:32 2014
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configurab
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
@@ -662,4 +663,35 @@ public class DummyRawStoreControlledComm
       throws MetaException, NoSuchObjectException {
     objectStore.dropPartitions(dbName, tblName, partNames);
   }
+
+  public void createFunction(Function func) throws InvalidObjectException,
+      MetaException {
+    objectStore.createFunction(func);
+  }
+
+  @Override
+  public void alterFunction(String dbName, String funcName, Function newFunction)
+      throws InvalidObjectException, MetaException {
+    objectStore.alterFunction(dbName, funcName, newFunction);
+  }
+
+  @Override
+  public void dropFunction(String dbName, String funcName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException,
+      InvalidInputException {
+    objectStore.dropFunction(dbName, funcName);
+  }
+
+  @Override
+  public Function getFunction(String dbName, String funcName)
+      throws MetaException {
+    return objectStore.getFunction(dbName, funcName);
+  }
+
+  @Override
+  public List<String> getFunctions(String dbName, String pattern)
+      throws MetaException {
+    return objectStore.getFunctions(dbName, pattern);
+  }
+
 }

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java Wed Feb 19 23:50:32 2014
@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
@@ -690,6 +691,33 @@ public class DummyRawStoreForJdoConnecti
   @Override
   public void dropPartitions(String dbName, String tblName, List<String> partNames) {
   }
+
+  public void createFunction(Function func) throws InvalidObjectException,
+      MetaException {
+  }
+
+  @Override
+  public void alterFunction(String dbName, String funcName, Function newFunction)
+      throws InvalidObjectException, MetaException {
+  }
+
+  @Override
+  public void dropFunction(String dbName, String funcName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException,
+      InvalidInputException {
+  }
+
+  @Override
+  public Function getFunction(String dbName, String funcName)
+      throws MetaException {
+    return null;
+  }
+
+  @Override
+  public List<String> getFunctions(String dbName, String pattern)
+      throws MetaException {
+    return null;
+  }
 }
 
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java Wed Feb 19 23:50:32 2014
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import java.io.IOException;
 import java.io.InputStream;
 import java.lang.reflect.Method;
 import java.net.URL;
@@ -39,7 +40,12 @@ import javax.xml.parsers.DocumentBuilder
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.ql.exec.FunctionUtils.UDFClassType;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -519,9 +525,55 @@ public final class FunctionRegistry {
     }
   }
 
+  private static FunctionInfo getFunctionInfoFromMetastore(String functionName) {
+    FunctionInfo ret = null;
+  
+    try {
+      String dbName;
+      String fName;
+      if (FunctionUtils.isQualifiedFunctionName(functionName)) {
+        String[] parts = FunctionUtils.splitQualifiedFunctionName(functionName);
+        dbName = parts[0];
+        fName = parts[1];
+      } else {
+        // otherwise, qualify using current db
+        dbName = SessionState.get().getCurrentDatabase();
+        fName = functionName;
+      }
+
+      // Try looking up function in the metastore
+      HiveConf conf = SessionState.get().getConf();
+      Function func = Hive.get(conf).getFunction(dbName, fName);
+      if (func != null) {
+        // Found UDF in metastore - now add it to the function registry
+        // At this point we should add any relevant jars that would be needed for the UDf.
+        Class<?> udfClass = Class.forName(func.getClassName(), true, JavaUtils.getClassLoader());
+        if (registerTemporaryFunction(functionName, udfClass)) {
+          ret = mFunctions.get(functionName);
+        } else {
+          LOG.error(func.getClassName() + " is not a valid UDF class and was not registered.");
+        }
+      }
+    } catch (HiveException e) {
+      LOG.info("Unable to lookup UDF in metastore: " + e);
+    } catch (ClassNotFoundException e) {
+      // Lookup of UDf class failed
+      LOG.error("Unable to load UDF class: " + e);
+    }
+  
+    return ret;
+  }
+
   private static <T> T getQualifiedFunctionInfo(Map<String, T> mFunctions, String functionName) {
     T functionInfo =  mFunctions.get(functionName);
-    // Eventually this would check metastore for registered functions.
+    if (functionInfo == null) {
+      // Try looking up in metastore.
+      FunctionInfo fi = getFunctionInfoFromMetastore(functionName);
+      if (fi != null) {
+        // metastore lookup resulted in function getting added to mFunctions, try again
+        functionInfo = mFunctions.get(functionName);
+      }
+    }
     return functionInfo;
   }
 
@@ -554,7 +606,33 @@ public final class FunctionRegistry {
    * @return set of strings contains function names
    */
   public static Set<String> getFunctionNames() {
-    return mFunctions.keySet();
+    return getFunctionNames(true);
+  }
+
+  private static Set<String> getFunctionNames(boolean searchMetastore) {
+    Set<String> functionNames = mFunctions.keySet();
+    if (searchMetastore) {
+      functionNames = new HashSet<String>(functionNames);
+      try {
+        Hive db = getHive();
+        List<String> dbNames = db.getAllDatabases();
+
+        for (String dbName : dbNames) {
+          List<String> funcNames = db.getFunctions(dbName, "*");
+          for (String funcName : funcNames) {
+            functionNames.add(FunctionUtils.qualifyFunctionName(funcName, dbName));
+          }
+        }
+      } catch (Exception e) {
+        LOG.error(e);
+        // Continue on, we can still return the functions we've gotten to this point.
+      }
+    }
+    return functionNames;
+  }
+
+  public static Hive getHive() throws HiveException {
+    return Hive.get(SessionState.get().getConf());
   }
 
   /**
@@ -1593,29 +1671,38 @@ public final class FunctionRegistry {
   public static boolean registerTemporaryFunction(
     String functionName, Class<?> udfClass) {
 
-    if (UDF.class.isAssignableFrom(udfClass)) {
+    UDFClassType udfClassType = FunctionUtils.getUDFClassType(udfClass);
+    switch (udfClassType) {
+    case UDF:
       FunctionRegistry.registerTemporaryUDF(
         functionName, (Class<? extends UDF>) udfClass, false);
-    } else if (GenericUDF.class.isAssignableFrom(udfClass)) {
+      break;
+    case GENERIC_UDF:
       FunctionRegistry.registerTemporaryGenericUDF(
         functionName, (Class<? extends GenericUDF>) udfClass);
-    } else if (GenericUDTF.class.isAssignableFrom(udfClass)) {
+      break;
+    case GENERIC_UDTF:
       FunctionRegistry.registerTemporaryGenericUDTF(
         functionName, (Class<? extends GenericUDTF>) udfClass);
-    } else if (UDAF.class.isAssignableFrom(udfClass)) {
+      break;
+    case UDAF:
       FunctionRegistry.registerTemporaryUDAF(
         functionName, (Class<? extends UDAF>) udfClass);
-    } else if (GenericUDAFResolver.class.isAssignableFrom(udfClass)) {
+      break;
+    case GENERIC_UDAF_RESOLVER:
       FunctionRegistry.registerTemporaryGenericUDAF(
         functionName, (GenericUDAFResolver)
         ReflectionUtils.newInstance(udfClass, null));
-    } else if(TableFunctionResolver.class.isAssignableFrom(udfClass)) {
+      break;
+    case TABLE_FUNCTION_RESOLVER:
       FunctionRegistry.registerTableFunction(
         functionName, (Class<? extends TableFunctionResolver>)udfClass);
-    } else {
+      break;
+    default:
       return false;
     }
     return true;
+
   }
 
   /**

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java Wed Feb 19 23:50:32 2014
@@ -18,13 +18,22 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import static org.apache.hadoop.util.StringUtils.stringifyException;
+
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.FunctionUtils.FunctionType;
+import org.apache.hadoop.hive.ql.exec.FunctionUtils.UDFClassType;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.CreateFunctionDesc;
 import org.apache.hadoop.hive.ql.plan.DropFunctionDesc;
@@ -32,6 +41,7 @@ import org.apache.hadoop.hive.ql.plan.Cr
 import org.apache.hadoop.hive.ql.plan.DropMacroDesc;
 import org.apache.hadoop.hive.ql.plan.FunctionWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
@@ -62,12 +72,32 @@ public class FunctionTask extends Task<F
   public int execute(DriverContext driverContext) {
     CreateFunctionDesc createFunctionDesc = work.getCreateFunctionDesc();
     if (createFunctionDesc != null) {
-      return createFunction(createFunctionDesc);
+      if (createFunctionDesc.isTemp()) {
+        return createTemporaryFunction(createFunctionDesc);
+      } else {
+        try {
+          return createPermanentFunction(Hive.get(conf), createFunctionDesc);
+        } catch (Exception e) {
+          setException(e);
+          LOG.error(stringifyException(e));
+          return 1;
+        }
+      }
     }
 
     DropFunctionDesc dropFunctionDesc = work.getDropFunctionDesc();
     if (dropFunctionDesc != null) {
-      return dropFunction(dropFunctionDesc);
+      if (dropFunctionDesc.isTemp()) {
+        return dropTemporaryFunction(dropFunctionDesc);
+      } else {
+        try {
+          return dropPermanentFunction(Hive.get(conf), dropFunctionDesc);
+        } catch (Exception e) {
+          setException(e);
+          LOG.error(stringifyException(e));
+          return 1;
+        }
+      }
     }
 
     CreateMacroDesc createMacroDesc = work.getCreateMacroDesc();
@@ -82,7 +112,55 @@ public class FunctionTask extends Task<F
     return 0;
   }
 
-  private int createFunction(CreateFunctionDesc createFunctionDesc) {
+  private int createPermanentFunction(Hive db, CreateFunctionDesc createFunctionDesc)
+      throws HiveException, IOException {
+    String[] qualifiedNameParts = FunctionUtils.getQualifiedFunctionNameParts(
+        createFunctionDesc.getFunctionName());
+    String dbName = qualifiedNameParts[0];
+    String funcName = qualifiedNameParts[1];
+    String registeredName = FunctionUtils.qualifyFunctionName(funcName, dbName);
+    String className = createFunctionDesc.getClassName();
+    boolean addedToRegistry = false;
+
+    try {
+      // UDF class should exist
+      Class<?> udfClass = getUdfClass(createFunctionDesc);
+      if (FunctionUtils.getUDFClassType(udfClass) == UDFClassType.UNKNOWN) {
+        console.printError("FAILED: Class " + createFunctionDesc.getClassName()
+            + " does not implement UDF, GenericUDF, or UDAF");
+        return 1;
+      }
+
+      // TODO: There should be a registerPermanentFunction()
+      addedToRegistry = FunctionRegistry.registerTemporaryFunction(registeredName, udfClass);
+      if (!addedToRegistry) {
+        console.printError("Failed to register " + registeredName
+            + " using class " + createFunctionDesc.getClassName());
+        return 1;
+      }
+
+      // Add to metastore
+      Function func = new Function(
+          funcName,
+          dbName,
+          className,
+          SessionState.get().getUserName(),
+          PrincipalType.USER,
+          (int) (System.currentTimeMillis() / 1000),
+          org.apache.hadoop.hive.metastore.api.FunctionType.JAVA);
+      db.createFunction(func);
+      return 0;
+    } catch (ClassNotFoundException e) {
+      console.printError("FAILED: Class " + createFunctionDesc.getClassName() + " not found");
+      LOG.info("create function: " + StringUtils.stringifyException(e));
+      if (addedToRegistry) {
+        FunctionRegistry.unregisterTemporaryUDF(registeredName);
+      }
+      return 1;
+    }
+  }
+
+  private int createTemporaryFunction(CreateFunctionDesc createFunctionDesc) {
     try {
       Class<?> udfClass = getUdfClass(createFunctionDesc);
       boolean registered = FunctionRegistry.registerTemporaryFunction(
@@ -121,7 +199,27 @@ public class FunctionTask extends Task<F
     }
   }
 
-  private int dropFunction(DropFunctionDesc dropFunctionDesc) {
+  private int dropPermanentFunction(Hive db, DropFunctionDesc dropFunctionDesc) {
+    try {
+      String[] qualifiedNameParts = FunctionUtils.getQualifiedFunctionNameParts(
+          dropFunctionDesc.getFunctionName());
+      String dbName = qualifiedNameParts[0];
+      String funcName = qualifiedNameParts[1];
+
+      String registeredName = FunctionUtils.qualifyFunctionName(funcName, dbName);
+      // TODO: there should be a unregisterPermanentUDF()
+      FunctionRegistry.unregisterTemporaryUDF(registeredName);
+      db.dropFunction(dbName, funcName);
+
+      return 0;
+    } catch (Exception e) {
+      LOG.info("drop function: " + StringUtils.stringifyException(e));
+      console.printError("FAILED: error during drop function: " + StringUtils.stringifyException(e));
+      return 1;
+    }
+  }
+
+  private int dropTemporaryFunction(DropFunctionDesc dropFunctionDesc) {
     try {
       FunctionRegistry.unregisterTemporaryUDF(dropFunctionDesc
           .getFunctionName());

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionUtils.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionUtils.java Wed Feb 19 23:50:32 2014
@@ -19,6 +19,11 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
+import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver;
 
 public class FunctionUtils {
 
@@ -51,4 +56,55 @@ public class FunctionUtils {
     return names;
   }
 
+  public static String[] getQualifiedFunctionNameParts(String name) throws HiveException {
+    if (isQualifiedFunctionName(name)) {
+      return splitQualifiedFunctionName(name);
+    }
+    String dbName = SessionState.get().getCurrentDatabase();
+    return new String[] { dbName, name };
+  }
+
+  /**
+   * Function type, for permanent functions.
+   * Currently just JAVA, though we could support Groovy later on.
+   */
+  public enum FunctionType {
+    JAVA,
+  }
+
+  /**
+   * Enum type to describe what kind of UDF implementation class
+   */
+  public enum UDFClassType {
+    UNKNOWN,
+    UDF,
+    GENERIC_UDF,
+    GENERIC_UDTF,
+    UDAF,
+    GENERIC_UDAF_RESOLVER,
+    TABLE_FUNCTION_RESOLVER,
+  }
+
+  /**
+   * Determine the UDF class type of the class
+   * @param udfClass
+   * @return UDFClassType enum corresponding to the class type of the UDF
+   */
+  public static UDFClassType getUDFClassType(Class<?> udfClass) {
+    if (UDF.class.isAssignableFrom(udfClass)) {
+      return UDFClassType.UDF;
+    } else if (GenericUDF.class.isAssignableFrom(udfClass)) {
+      return UDFClassType.GENERIC_UDF;
+    } else if (GenericUDTF.class.isAssignableFrom(udfClass)) {
+      return UDFClassType.GENERIC_UDTF;
+    } else if (UDAF.class.isAssignableFrom(udfClass)) {
+      return UDFClassType.UDAF;
+    } else if (GenericUDAFResolver.class.isAssignableFrom(udfClass)) {
+      return UDFClassType.GENERIC_UDAF_RESOLVER;
+    } else if(TableFunctionResolver.class.isAssignableFrom(udfClass)) {
+      return UDFClassType.TABLE_FUNCTION_RESOLVER;
+    } else {
+      return UDFClassType.UNKNOWN;
+    }
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Wed Feb 19 23:50:32 2014
@@ -69,6 +69,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
@@ -2571,4 +2572,46 @@ private void constructOneLBLocationMap(F
     return qualifiedName.split("\\.");
   }
 
+  public void createFunction(Function func) throws HiveException {
+    try {
+      getMSC().createFunction(func);
+    } catch (TException te) {
+      throw new HiveException(te);
+    }
+  }
+
+  public void alterFunction(String dbName, String funcName, Function newFunction)
+      throws HiveException {
+    try {
+      getMSC().alterFunction(dbName, funcName, newFunction);
+    } catch (TException te) {
+      throw new HiveException(te);
+    }
+  }
+
+  public void dropFunction(String dbName, String funcName)
+      throws HiveException {
+    try {
+      getMSC().dropFunction(dbName, funcName);
+    } catch (TException te) {
+      throw new HiveException(te);
+    }
+  }
+
+  public Function getFunction(String dbName, String funcName) throws HiveException {
+    try {
+      return getMSC().getFunction(dbName, funcName);
+    } catch (TException te) {
+      throw new HiveException(te);
+    }
+  }
+
+  public List<String> getFunctions(String dbName, String pattern) throws HiveException {
+    try {
+      return getMSC().getFunctions(dbName, pattern);
+    } catch (TException te) {
+      throw new HiveException(te);
+    }
+  }
+
 };

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java Wed Feb 19 23:50:32 2014
@@ -21,13 +21,17 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.FunctionUtils;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.CreateFunctionDesc;
 import org.apache.hadoop.hive.ql.plan.DropFunctionDesc;
 import org.apache.hadoop.hive.ql.plan.FunctionWork;
+import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
  * FunctionSemanticAnalyzer.
@@ -54,30 +58,40 @@ public class FunctionSemanticAnalyzer ex
   }
 
   private void analyzeCreateFunction(ASTNode ast) throws SemanticException {
-    String functionName = ast.getChild(0).getText();
+    // ^(TOK_CREATEFUNCTION identifier StringLiteral ({isTempFunction}? => TOK_TEMPORARY))
+    String functionName = ast.getChild(0).getText().toLowerCase();
+    boolean isTemporaryFunction = (ast.getFirstChildWithType(HiveParser.TOK_TEMPORARY) != null);
     String className = unescapeSQLString(ast.getChild(1).getText());
 
     // Temp functions are not allowed to have qualified names.
-    if (FunctionUtils.isQualifiedFunctionName(functionName)) {
+    if (isTemporaryFunction && FunctionUtils.isQualifiedFunctionName(functionName)) {
       throw new SemanticException("Temporary function cannot be created with a qualified name.");
     }
 
-    CreateFunctionDesc desc = new CreateFunctionDesc(functionName, className);
+    CreateFunctionDesc desc = new CreateFunctionDesc(functionName, isTemporaryFunction, className);
     rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
   }
 
   private void analyzeDropFunction(ASTNode ast) throws SemanticException {
+    // ^(TOK_DROPFUNCTION identifier ifExists? $temp?)
     String functionName = ast.getChild(0).getText();
     boolean ifExists = (ast.getFirstChildWithType(HiveParser.TOK_IFEXISTS) != null);
     // we want to signal an error if the function doesn't exist and we're
     // configured not to ignore this
     boolean throwException =
       !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
-    if (throwException && FunctionRegistry.getFunctionInfo(functionName) == null) {
-      throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(functionName));
+
+    if (FunctionRegistry.getFunctionInfo(functionName) == null) {
+      if (throwException) {
+        throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(functionName));
+      } else {
+        // Fail silently
+        return;
+      }
     }
 
-    DropFunctionDesc desc = new DropFunctionDesc(functionName);
+    boolean isTemporaryFunction = (ast.getFirstChildWithType(HiveParser.TOK_TEMPORARY) != null);
+    DropFunctionDesc desc = new DropFunctionDesc(functionName, isTemporaryFunction);
     rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g Wed Feb 19 23:50:32 2014
@@ -215,6 +215,7 @@ TOK_CREATEFUNCTION;
 TOK_DROPFUNCTION;
 TOK_CREATEMACRO;
 TOK_DROPMACRO;
+TOK_TEMPORARY;
 TOK_CREATEVIEW;
 TOK_DROPVIEW;
 TOK_ALTERVIEW_AS;
@@ -1496,15 +1497,17 @@ metastoreCheck
 createFunctionStatement
 @init { pushMsg("create function statement", state); }
 @after { popMsg(state); }
-    : KW_CREATE KW_TEMPORARY KW_FUNCTION functionIdentifier KW_AS StringLiteral
-    -> ^(TOK_CREATEFUNCTION functionIdentifier StringLiteral)
+    : KW_CREATE (temp=KW_TEMPORARY)? KW_FUNCTION functionIdentifier KW_AS StringLiteral
+    -> {$temp != null}? ^(TOK_CREATEFUNCTION functionIdentifier StringLiteral TOK_TEMPORARY)
+    ->                  ^(TOK_CREATEFUNCTION functionIdentifier StringLiteral)
     ;
 
 dropFunctionStatement
-@init { pushMsg("drop temporary function statement", state); }
+@init { pushMsg("drop function statement", state); }
 @after { popMsg(state); }
-    : KW_DROP KW_TEMPORARY KW_FUNCTION ifExists? functionIdentifier
-    -> ^(TOK_DROPFUNCTION functionIdentifier ifExists?)
+    : KW_DROP (temp=KW_TEMPORARY)? KW_FUNCTION ifExists? functionIdentifier
+    -> {$temp != null}? ^(TOK_DROPFUNCTION functionIdentifier ifExists? TOK_TEMPORARY)
+    ->                  ^(TOK_DROPFUNCTION functionIdentifier ifExists?)
     ;
 
 createMacroStatement

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java Wed Feb 19 23:50:32 2014
@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
@@ -40,10 +41,13 @@ import org.apache.hadoop.hive.ql.exec.Ta
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.CreateMacroDesc;
 import org.apache.hadoop.hive.ql.plan.DropMacroDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.FunctionWork;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
@@ -144,6 +148,12 @@ public class MacroSemanticAnalyzer exten
     // configured not to ignore this
     boolean throwException =
       !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
+
+    // Temp macros are not allowed to have qualified names.
+    if (FunctionUtils.isQualifiedFunctionName(functionName)) {
+      throw new SemanticException("Temporary macro name cannot be a qualified name.");
+    }
+
     if (throwException && FunctionRegistry.getFunctionInfo(functionName) == null) {
       throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(functionName));
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateFunctionDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateFunctionDesc.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateFunctionDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateFunctionDesc.java Wed Feb 19 23:50:32 2014
@@ -30,6 +30,7 @@ public class CreateFunctionDesc implemen
 
   private String functionName;
   private String className;
+  private boolean isTemp;
 
   /**
    * For serialization only.
@@ -37,8 +38,9 @@ public class CreateFunctionDesc implemen
   public CreateFunctionDesc() {
   }
   
-  public CreateFunctionDesc(String functionName, String className) {
+  public CreateFunctionDesc(String functionName, boolean isTemp, String className) {
     this.functionName = functionName;
+    this.isTemp = isTemp;
     this.className = className;
   }
 
@@ -60,4 +62,12 @@ public class CreateFunctionDesc implemen
     this.className = className;
   }
 
+  public boolean isTemp() {
+    return isTemp;
+  }
+
+  public void setTemp(boolean isTemp) {
+    this.isTemp = isTemp;
+  }
+
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropFunctionDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropFunctionDesc.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropFunctionDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropFunctionDesc.java Wed Feb 19 23:50:32 2014
@@ -29,6 +29,7 @@ public class DropFunctionDesc implements
   private static final long serialVersionUID = 1L;
 
   private String functionName;
+  private boolean isTemp;
 
   /**
    * For serialization only.
@@ -36,8 +37,9 @@ public class DropFunctionDesc implements
   public DropFunctionDesc() {
   }
   
-  public DropFunctionDesc(String functionName) {
+  public DropFunctionDesc(String functionName, boolean isTemp) {
     this.functionName = functionName;
+    this.isTemp = isTemp;
   }
 
   @Explain(displayName = "name")
@@ -49,4 +51,12 @@ public class DropFunctionDesc implements
     this.functionName = functionName;
   }
 
+  public boolean isTemp() {
+    return isTemp;
+  }
+
+  public void setTemp(boolean isTemp) {
+    this.isTemp = isTemp;
+  }
+
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java Wed Feb 19 23:50:32 2014
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec;
 
 import java.lang.reflect.Method;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -27,6 +28,8 @@ import junit.framework.Assert;
 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.session.SessionState;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -73,6 +76,7 @@ public class TestFunctionRegistry extend
     varchar5 = TypeInfoFactory.getPrimitiveTypeInfo("varchar(5)");
     char10 = TypeInfoFactory.getPrimitiveTypeInfo("char(10)");
     char5 = TypeInfoFactory.getPrimitiveTypeInfo("char(5)");
+    SessionState.start(new HiveConf());
   }
 
   private void implicit(TypeInfo a, TypeInfo b, boolean convertible) {

Added: hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q Wed Feb 19 23:50:32 2014
@@ -0,0 +1 @@
+create function default.badfunc as 'my.nonexistent.class';

Added: hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q Wed Feb 19 23:50:32 2014
@@ -0,0 +1 @@
+create function nonexistentdb.badfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper';

Added: hive/trunk/ql/src/test/queries/clientnegative/create_function_nonudf_class.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/create_function_nonudf_class.q?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/create_function_nonudf_class.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/create_function_nonudf_class.q Wed Feb 19 23:50:32 2014
@@ -0,0 +1 @@
+create function default.badfunc as 'java.lang.String';

Added: hive/trunk/ql/src/test/queries/clientnegative/drop_func_nonexistent.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/drop_func_nonexistent.q?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/drop_func_nonexistent.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/drop_func_nonexistent.q Wed Feb 19 23:50:32 2014
@@ -0,0 +1,3 @@
+set hive.exec.drop.ignorenonexistent=false;
+-- Can't use DROP FUNCTION if the function doesn't exist and IF EXISTS isn't specified
+drop function nonexistent_function;

Added: hive/trunk/ql/src/test/queries/clientpositive/create_func1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/create_func1.q?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/create_func1.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/create_func1.q Wed Feb 19 23:50:32 2014
@@ -0,0 +1,30 @@
+
+-- qtest_get_java_boolean should already be created during test initialization
+select qtest_get_java_boolean('true'), qtest_get_java_boolean('false') from src limit 1;
+
+create database mydb;
+create function mydb.func1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper';
+
+show functions mydb.func1;
+
+select mydb.func1('abc') from src limit 1;
+
+drop function mydb.func1;
+
+-- function should now be gone
+show functions mydb.func1;
+
+-- To test function name resolution
+create function mydb.qtest_get_java_boolean as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper';
+
+use default;
+-- unqualified function should resolve to one in default db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1;
+
+use mydb;
+-- unqualified function should resolve to one in mydb db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1;
+
+drop function mydb.qtest_get_java_boolean;
+
+drop database mydb cascade;

Added: hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out Wed Feb 19 23:50:32 2014
@@ -0,0 +1,4 @@
+PREHOOK: query: create function default.badfunc as 'my.nonexistent.class'
+PREHOOK: type: CREATEFUNCTION
+FAILED: Class my.nonexistent.class not found
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask

Added: hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_db.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_db.q.out?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_db.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/create_function_nonexistent_db.q.out Wed Feb 19 23:50:32 2014
@@ -0,0 +1,3 @@
+PREHOOK: query: create function nonexistentdb.badfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'
+PREHOOK: type: CREATEFUNCTION
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask. NoSuchObjectException(message:There is no database named nonexistentdb)

Added: hive/trunk/ql/src/test/results/clientnegative/create_function_nonudf_class.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/create_function_nonudf_class.q.out?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/create_function_nonudf_class.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/create_function_nonudf_class.q.out Wed Feb 19 23:50:32 2014
@@ -0,0 +1,4 @@
+PREHOOK: query: create function default.badfunc as 'java.lang.String'
+PREHOOK: type: CREATEFUNCTION
+FAILED: Class java.lang.String does not implement UDF, GenericUDF, or UDAF
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask

Modified: hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty.q.out?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty.q.out Wed Feb 19 23:50:32 2014
@@ -26,4 +26,4 @@ PREHOOK: query: DROP DATABASE test_db
 PREHOOK: type: DROPDATABASE
 PREHOOK: Input: database:test_db
 PREHOOK: Output: database:test_db
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Database test_db is not empty)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Database test_db is not empty. One or more tables exist.)

Modified: hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty_restrict.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty_restrict.q.out?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty_restrict.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/database_drop_not_empty_restrict.q.out Wed Feb 19 23:50:32 2014
@@ -26,4 +26,4 @@ PREHOOK: query: DROP DATABASE db_drop_no
 PREHOOK: type: DROPDATABASE
 PREHOOK: Input: database:db_drop_non_empty_restrict
 PREHOOK: Output: database:db_drop_non_empty_restrict
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Database db_drop_non_empty_restrict is not empty)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. InvalidOperationException(message:Database db_drop_non_empty_restrict is not empty. One or more tables exist.)

Added: hive/trunk/ql/src/test/results/clientnegative/drop_func_nonexistent.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/drop_func_nonexistent.q.out?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/drop_func_nonexistent.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/drop_func_nonexistent.q.out Wed Feb 19 23:50:32 2014
@@ -0,0 +1 @@
+FAILED: SemanticException [Error 10011]: Invalid function nonexistent_function

Added: hive/trunk/ql/src/test/results/clientpositive/create_func1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/create_func1.q.out?rev=1569985&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/create_func1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/create_func1.q.out Wed Feb 19 23:50:32 2014
@@ -0,0 +1,91 @@
+PREHOOK: query: -- qtest_get_java_boolean should already be created during test initialization
+select qtest_get_java_boolean('true'), qtest_get_java_boolean('false') from src limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- qtest_get_java_boolean should already be created during test initialization
+select qtest_get_java_boolean('true'), qtest_get_java_boolean('false') from src limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+true	false
+PREHOOK: query: create database mydb
+PREHOOK: type: CREATEDATABASE
+POSTHOOK: query: create database mydb
+POSTHOOK: type: CREATEDATABASE
+PREHOOK: query: create function mydb.func1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'
+PREHOOK: type: CREATEFUNCTION
+POSTHOOK: query: create function mydb.func1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'
+POSTHOOK: type: CREATEFUNCTION
+PREHOOK: query: show functions mydb.func1
+PREHOOK: type: SHOWFUNCTIONS
+POSTHOOK: query: show functions mydb.func1
+POSTHOOK: type: SHOWFUNCTIONS
+mydb.func1
+PREHOOK: query: select mydb.func1('abc') from src limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select mydb.func1('abc') from src limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+ABC
+PREHOOK: query: drop function mydb.func1
+PREHOOK: type: DROPFUNCTION
+POSTHOOK: query: drop function mydb.func1
+POSTHOOK: type: DROPFUNCTION
+PREHOOK: query: -- function should now be gone
+show functions mydb.func1
+PREHOOK: type: SHOWFUNCTIONS
+POSTHOOK: query: -- function should now be gone
+show functions mydb.func1
+POSTHOOK: type: SHOWFUNCTIONS
+PREHOOK: query: -- To test function name resolution
+create function mydb.qtest_get_java_boolean as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'
+PREHOOK: type: CREATEFUNCTION
+POSTHOOK: query: -- To test function name resolution
+create function mydb.qtest_get_java_boolean as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'
+POSTHOOK: type: CREATEFUNCTION
+PREHOOK: query: use default
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: use default
+POSTHOOK: type: SWITCHDATABASE
+PREHOOK: query: -- unqualified function should resolve to one in default db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- unqualified function should resolve to one in default db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+NULL	NULL	ABC
+PREHOOK: query: use mydb
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: use mydb
+POSTHOOK: type: SWITCHDATABASE
+PREHOOK: query: -- unqualified function should resolve to one in mydb db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: -- unqualified function should resolve to one in mydb db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+ABC	NULL	ABC
+PREHOOK: query: drop function mydb.qtest_get_java_boolean
+PREHOOK: type: DROPFUNCTION
+POSTHOOK: query: drop function mydb.qtest_get_java_boolean
+POSTHOOK: type: DROPFUNCTION
+PREHOOK: query: drop database mydb cascade
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:mydb
+PREHOOK: Output: database:mydb
+POSTHOOK: query: drop database mydb cascade
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:mydb
+POSTHOOK: Output: database:mydb

Modified: hive/trunk/ql/src/test/results/clientpositive/show_functions.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/show_functions.q.out?rev=1569985&r1=1569984&r2=1569985&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/show_functions.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/show_functions.q.out Wed Feb 19 23:50:32 2014
@@ -57,6 +57,7 @@ datediff
 day
 dayofmonth
 decode
+default.qtest_get_java_boolean
 degrees
 dense_rank
 div