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/03/23 18:01:23 UTC

svn commit: r1580537 - in /hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec: CommonFunctionInfo.java FunctionInfo.java FunctionRegistry.java WindowFunctionInfo.java

Author: hashutosh
Date: Sun Mar 23 17:01:23 2014
New Revision: 1580537

URL: http://svn.apache.org/r1580537
Log:
HIVE-6672 : JARs loaded by permanent functions don't work properly with HiveServer2 (Jason Dere via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java
Modified:
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java

Added: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java?rev=1580537&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java (added)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java Sun Mar 23 17:01:23 2014
@@ -0,0 +1,25 @@
+/**
+ * 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.ql.exec;
+
+/**
+ * Interface for common functionality between FunctionInfo/WindowFunctionInfo
+ */
+public interface CommonFunctionInfo {
+  Class<?> getFunctionClass();
+}

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java?rev=1580537&r1=1580536&r2=1580537&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java Sun Mar 23 17:01:23 2014
@@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.udf.ptf
  * FunctionInfo.
  *
  */
-public class FunctionInfo {
+public class FunctionInfo implements CommonFunctionInfo {
 
   private final boolean isNative;
 

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java?rev=1580537&r1=1580536&r2=1580537&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java Sun Mar 23 17:01:23 2014
@@ -571,7 +571,8 @@ public final class FunctionRegistry {
     return ret;
   }
 
-  private static <T> T getQualifiedFunctionInfo(Map<String, T> mFunctions, String functionName) {
+  private static <T extends CommonFunctionInfo> T getQualifiedFunctionInfo(
+      Map<String, T> mFunctions, String functionName) {
     T functionInfo =  mFunctions.get(functionName);
     if (functionInfo == null) {
       // Try looking up in metastore.
@@ -581,10 +582,55 @@ public final class FunctionRegistry {
         functionInfo = mFunctions.get(functionName);
       }
     }
+
+    // HIVE-6672: In HiveServer2 the JARs for this UDF may have been loaded by a different thread,
+    // and the current thread may not be able to resolve the UDF. Test for this condition
+    // and if necessary load the JARs in this thread.
+    if (functionInfo != null) {
+      loadFunctionResourcesIfNecessary(functionName, functionInfo);
+    }
+    
     return functionInfo;
   }
 
-  private static <T> T getFunctionInfo(Map<String, T> mFunctions, String functionName) {
+  private static void checkFunctionClass(CommonFunctionInfo cfi) throws ClassNotFoundException {
+    // This call will fail for non-generic UDFs using GenericUDFBridge
+    Class<?> udfClass = cfi.getFunctionClass();
+    // Even if we have a reference to the class (which will be the case for GenericUDFs),
+    // the classloader may not be able to resolve the class, which would mean reflection-based
+    // methods would fail such as for plan deserialization. Make sure this works too.
+    Class.forName(udfClass.getName(), true, JavaUtils.getClassLoader());
+  }
+
+  private static void loadFunctionResourcesIfNecessary(String functionName, CommonFunctionInfo cfi) {
+    try {
+      // Check if the necessary JARs have been loaded for this function.
+      checkFunctionClass(cfi);
+    } catch (Exception e) {
+      // Unable to resolve the UDF with the classloader.
+      // Look up the function in the metastore and load any resources.
+      LOG.debug("Attempting to reload resources for " + functionName);
+      try {
+        String[] parts = FunctionUtils.getQualifiedFunctionNameParts(functionName);
+        HiveConf conf = SessionState.get().getConf();
+        Function func = Hive.get(conf).getFunction(parts[0], parts[1]);
+        if (func != null) {
+          FunctionTask.addFunctionResources(func.getResourceUris());
+          // Check again now that we've loaded the resources in this thread.
+          checkFunctionClass(cfi);
+        } else {
+          // Couldn't find the function .. just rethrow the original error
+          LOG.error("Unable to reload resources for " + functionName);
+          throw e;
+        }
+      } catch (Exception err) {
+        throw new RuntimeException(err);
+      }
+    }
+  }
+
+  private static <T extends CommonFunctionInfo> T getFunctionInfo(
+      Map<String, T> mFunctions, String functionName) {
     functionName = functionName.toLowerCase();
     T functionInfo = null;
     if (FunctionUtils.isQualifiedFunctionName(functionName)) {

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java?rev=1580537&r1=1580536&r2=1580537&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java Sun Mar 23 17:01:23 2014
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.exec;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver;
 
 @SuppressWarnings("deprecation")
-public class WindowFunctionInfo
+public class WindowFunctionInfo implements CommonFunctionInfo
 {
 	boolean supportsWindow = true;
 	boolean pivotResult = false;
@@ -59,4 +59,9 @@ public class WindowFunctionInfo
 	{
 		return fInfo;
 	}
+
+  @Override
+  public Class<?> getFunctionClass() {
+    return getfInfo().getFunctionClass();
+  }
 }