You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2014/03/29 01:09:09 UTC

svn commit: r1582922 - in /hive/trunk: itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java

Author: jdere
Date: Sat Mar 29 00:09:08 2014
New Revision: 1582922

URL: http://svn.apache.org/r1582922
Log:
HIVE-6744: Permanent UDF lookup fails when current DB has uppercase letters (jdere, reviewed by thejas)

Modified:
    hive/trunk/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java

Modified: hive/trunk/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java?rev=1582922&r1=1582921&r2=1582922&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java (original)
+++ hive/trunk/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java Sat Mar 29 00:09:08 2014
@@ -283,6 +283,14 @@ public class MiniHS2 extends AbstractHiv
     return driverName;
   }
 
+  public MiniMrShim getMR() {
+    return mr;
+  }
+
+  public MiniDFSShim getDFS() {
+    return dfs;
+  }
+
   private void waitForStartup() throws Exception {
     int waitTime = 0;
     long startupTimeout = 1000L * 1000L;

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java?rev=1582922&r1=1582921&r2=1582922&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniMr.java Sat Mar 29 00:09:08 2014
@@ -20,6 +20,7 @@ package org.apache.hive.jdbc;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 import java.util.Map;
 import java.util.HashMap;
@@ -28,6 +29,8 @@ import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -52,6 +55,7 @@ public class TestJdbcWithMiniMr {
    }
 
   private static MiniHS2 miniHS2 = null;
+  private static HiveConf conf;
   private static Path dataFilePath;
   private static String  dbName = "mrTestDb";
   private Connection hs2Conn = null;
@@ -60,7 +64,7 @@ public class TestJdbcWithMiniMr {
   @BeforeClass
   public static void beforeTest() throws Exception {
     Class.forName(MiniHS2.getJdbcDriverName());
-    HiveConf conf = new HiveConf();
+    conf = new HiveConf();
     conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
     String dataFileDir = conf.get("test.data.files").replace('\\', '/')
         .replace("c:", "");
@@ -145,6 +149,80 @@ public class TestJdbcWithMiniMr {
     testKvQuery(tableName, queryStr, resultVal);
   }
 
+  @Test
+  public void testPermFunc() throws Exception {
+
+    // This test assumes the hive-contrib JAR has been built as part of the Hive build.
+    // Also dependent on the UDFExampleAdd class within that JAR.
+    String udfClassName = "org.apache.hadoop.hive.contrib.udf.example.UDFExampleAdd";
+    String mvnRepo = System.getProperty("maven.local.repository");
+    String hiveVersion = System.getProperty("hive.version");
+    String jarFileName = "hive-contrib-" + hiveVersion + ".jar";
+    String[] pathParts = {
+        "org", "apache", "hive",
+        "hive-contrib", hiveVersion, jarFileName
+    };
+
+    // Create path to hive-contrib JAR on local filesystem
+    Path contribJarPath = new Path(mvnRepo);
+    for (String pathPart : pathParts) {
+      contribJarPath = new Path(contribJarPath, pathPart);
+    }
+    FileSystem localFs = FileSystem.getLocal(conf);
+    assertTrue("Hive contrib JAR exists at " + contribJarPath, localFs.exists(contribJarPath));
+
+    String hdfsJarPathStr = "hdfs:///" + jarFileName;
+    Path hdfsJarPath = new Path(hdfsJarPathStr);
+
+    // Copy JAR to DFS
+    FileSystem dfs = miniHS2.getDFS().getFileSystem();
+    dfs.copyFromLocalFile(contribJarPath, hdfsJarPath);
+    assertTrue("Verify contrib JAR copied to HDFS at " + hdfsJarPath, dfs.exists(hdfsJarPath));
+
+    // Register function
+    String queryStr = "CREATE FUNCTION example_add AS '" + udfClassName + "'"
+        + " USING JAR '" + hdfsJarPathStr + "'";
+    stmt.execute(queryStr);
+
+    // Call describe
+    ResultSet res;
+    res = stmt.executeQuery("DESCRIBE FUNCTION " + dbName + ".example_add");
+    checkForNotExist(res);
+
+    // Use UDF in query
+    String tableName = "testTab3";
+    setupKv1Tabs(tableName);
+    res = stmt.executeQuery("SELECT EXAMPLE_ADD(1, 2) FROM " + tableName + " LIMIT 1");
+    assertTrue("query has results", res.next());
+    assertEquals(3, res.getInt(1));
+    assertFalse("no more results", res.next());
+
+    // A new connection should be able to call describe/use function without issue
+    Connection conn2 = DriverManager.getConnection(miniHS2.getJdbcURL(dbName),
+        System.getProperty("user.name"), "bar");
+    Statement stmt2 = conn2.createStatement();
+    stmt2.execute("USE " + dbName);
+    res = stmt2.executeQuery("DESCRIBE FUNCTION " + dbName + ".example_add");
+    checkForNotExist(res);
+
+    res = stmt2.executeQuery("SELECT " + dbName + ".example_add(1, 1) FROM " + tableName + " LIMIT 1");
+    assertTrue("query has results", res.next());
+    assertEquals(2, res.getInt(1));
+    assertFalse("no more results", res.next());
+
+    stmt.execute("DROP TABLE " + tableName);
+  }
+
+  private void checkForNotExist(ResultSet res) throws Exception {
+    int numRows = 0;
+    while (res.next()) {
+      numRows++;
+      String strVal = res.getString(1);
+      assertEquals("Should not find 'not exist'", -1, strVal.toLowerCase().indexOf("not exist"));
+    }
+    assertTrue("Rows returned from describe function", numRows > 0);
+  }
+
   /**
    * Verify if the given property contains the expected value
    * @param propertyName

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=1582922&r1=1582921&r2=1582922&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 Sat Mar 29 00:09:08 2014
@@ -538,7 +538,7 @@ public final class FunctionRegistry {
         fName = parts[1];
       } else {
         // otherwise, qualify using current db
-        dbName = SessionState.get().getCurrentDatabase();
+        dbName = SessionState.get().getCurrentDatabase().toLowerCase();
         fName = functionName;
       }
 
@@ -645,7 +645,7 @@ public final class FunctionRegistry {
       functionInfo =  mFunctions.get(functionName);
       if (functionInfo == null && !FunctionUtils.isQualifiedFunctionName(functionName)) {
         String qualifiedName = FunctionUtils.qualifyFunctionName(functionName,
-            SessionState.get().getCurrentDatabase());
+            SessionState.get().getCurrentDatabase().toLowerCase());
         functionInfo = getQualifiedFunctionInfo(mFunctions, qualifiedName);
       }
     }