You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pv...@apache.org on 2020/08/04 07:35:43 UTC

[hive] branch master updated: HIVE-20441: NPE in GenericUDF when hive.allow.udf.load.on.demand is set to true (Zhihua Deng reviewed by Peter Vary)

This is an automated email from the ASF dual-hosted git repository.

pvary pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 34ed983  HIVE-20441: NPE in GenericUDF when hive.allow.udf.load.on.demand is set to true (Zhihua Deng reviewed by Peter Vary)
34ed983 is described below

commit 34ed9834abaf06537a127cbf6f12560ea7a4418a
Author: dengzh <de...@gmail.com>
AuthorDate: Tue Aug 4 15:35:33 2020 +0800

    HIVE-20441: NPE in GenericUDF when hive.allow.udf.load.on.demand is set to true (Zhihua Deng reviewed by Peter Vary)
    
    Closes (#1242)
---
 .../org/apache/hadoop/hive/ql/exec/Registry.java   |  5 +-
 .../parse/type/TestExprProcessorGetFuncExpr.java   | 81 ++++++++++++++++++++++
 2 files changed, 84 insertions(+), 2 deletions(-)

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 6ceea2f..1f6ae08 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
@@ -293,9 +293,10 @@ public class Registry {
     if (registerToSession) {
       String qualifiedName = FunctionUtils.qualifyFunctionName(
           functionName, SessionState.get().getCurrentDatabase().toLowerCase());
-      if (registerToSessionRegistry(qualifiedName, function) != null) {
+      FunctionInfo newFunction = registerToSessionRegistry(qualifiedName, function);
+      if (newFunction != null) {
         addFunction(functionName, function);
-        return function;
+        return newFunction;
       }
     } else {
         addFunction(functionName, function);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprProcessorGetFuncExpr.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprProcessorGetFuncExpr.java
new file mode 100644
index 0000000..f944f91
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprProcessorGetFuncExpr.java
@@ -0,0 +1,81 @@
+/*
+ * 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.parse.type;
+
+import org.antlr.runtime.CommonToken;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+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.serde2.typeinfo.TypeInfoFactory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Check the genericUDF field of FunctionInfo returned on demand.
+ *
+ */
+public class TestExprProcessorGetFuncExpr {
+
+  @Before
+  public void setUp() throws Exception {
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_ALLOW_UDF_LOAD_ON_DEMAND, true);
+    SessionState sessionState = new SessionState(hiveConf, System.getProperty("user.name"));
+    SessionState.setCurrentSessionState(sessionState);
+    Function function = new Function("myupper", sessionState.getCurrentDatabase(),
+        "org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper", sessionState.getUserName(),
+        PrincipalType.USER, (int) (System.currentTimeMillis() / 1000), FunctionType.JAVA, null);
+    Hive.get().createFunction(function);
+  }
+
+  @Test
+  public void testLookupFunctionOnDemand() throws Exception {
+    TypeCheckProcFactory.DefaultExprProcessor defaultExprProcessor =
+        ExprNodeTypeCheck.getExprNodeDefaultExprProcessor();
+    ASTNode funcExpr = new ASTNode(new CommonToken(HiveParser.TOK_FUNCTION, "TOK_FUNCTION"));
+    funcExpr.addChild(new ASTNode(new CommonToken(HiveParser.Identifier, "myupper")));
+    funcExpr.addChild(new ASTNode(new CommonToken(HiveParser.StringLiteral, "test")));
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+    children.add(new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, "test"));
+    // getXpathOrFuncExprNodeDesc cannot access from outside package
+    ExprNodeDesc exprNodeDesc = (ExprNodeDesc) defaultExprProcessor.
+        getXpathOrFuncExprNodeDesc(funcExpr, true, children, new TypeCheckCtx(null));
+    Assert.assertNotNull(exprNodeDesc);
+    Assert.assertNotNull(((ExprNodeGenericFuncDesc)exprNodeDesc).getGenericUDF());
+  }
+
+  @After
+  public void tearDown() {
+    Hive.closeCurrent();
+    SessionState.detachSession();
+  }
+}