You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2013/07/29 23:08:19 UTC

svn commit: r1508202 [4/48] - in /hive/branches/tez: ./ beeline/src/java/org/apache/hive/beeline/ cli/src/java/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/common/metrics/ common/src/java/org/apache/hadoop/hive/conf/ common/src/te...

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDataSource.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDataSource.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDataSource.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDataSource.java Mon Jul 29 21:08:03 2013
@@ -21,6 +21,8 @@ package org.apache.hive.jdbc;
 import java.io.PrintWriter;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.logging.Logger;
 
 import javax.sql.DataSource;
 
@@ -84,6 +86,11 @@ public class HiveDataSource implements D
     throw new SQLException("Method not supported");
   }
 
+  public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+    // JDK 1.7
+    throw new SQLFeatureNotSupportedException("Method not supported");
+  }
+
   /*
    * (non-Javadoc)
    * 

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java Mon Jul 29 21:08:03 2013
@@ -144,6 +144,17 @@ public class HiveDatabaseMetaData implem
     throw new SQLException("Method not supported");
   }
 
+  public ResultSet getPseudoColumns(String catalog, String schemaPattern,
+      String tableNamePattern, String columnNamePattern) throws SQLException {
+    // JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
+  public boolean generatedKeyAlwaysReturned() throws SQLException {
+    // JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
   /**
    * Convert a pattern containing JDBC catalog search wildcards into
    * Java regex patterns.
@@ -709,6 +720,16 @@ public class HiveDatabaseMetaData implem
       public boolean next() throws SQLException {
         return false;
       }
+
+      public <T> T getObject(String columnLabel, Class<T> type) throws SQLException {
+        // JDK 1.7
+        throw new SQLException("Method not supported");
+      }
+
+      public <T> T getObject(int columnIndex, Class<T> type) throws SQLException {
+        // JDK 1.7
+        throw new SQLException("Method not supported");
+        }
     };
   }
 

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java Mon Jul 29 21:08:03 2013
@@ -24,9 +24,11 @@ import java.sql.Connection;
 import java.sql.Driver;
 import java.sql.DriverPropertyInfo;
 import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
 import java.util.Properties;
 import java.util.jar.Attributes;
 import java.util.jar.Manifest;
+import java.util.logging.Logger;
 import java.util.regex.Pattern;
 /**
  * HiveDriver.
@@ -171,6 +173,11 @@ public class HiveDriver implements Drive
     return HiveDriver.getMinorDriverVersion();
   }
 
+  public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+    // JDK 1.7
+    throw new SQLFeatureNotSupportedException("Method not supported");
+  }
+
   public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException {
     if (info == null) {
       info = new Properties();

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java Mon Jul 29 21:08:03 2013
@@ -43,10 +43,10 @@ import java.util.Calendar;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TExecuteStatementReq;
 import org.apache.hive.service.cli.thrift.TExecuteStatementResp;
 import org.apache.hive.service.cli.thrift.TOperationHandle;
-import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TSessionHandle;
 
 /**
@@ -510,8 +510,7 @@ public class HivePreparedStatement imple
    */
 
   public void setDate(int parameterIndex, Date x) throws SQLException {
-    // TODO Auto-generated method stub
-    throw new SQLException("Method not supported");
+    this.parameters.put(parameterIndex, x.toString());
   }
 
   /*
@@ -858,6 +857,11 @@ public class HivePreparedStatement imple
      warningChain=null;
   }
 
+  public void closeOnCompletion() throws SQLException {
+    // JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
   /**
    *  Closes the prepared statement.
    *
@@ -1154,6 +1158,11 @@ public class HivePreparedStatement imple
     return isClosed;
   }
 
+  public boolean isCloseOnCompletion() throws SQLException {
+    //JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
   /*
    * (non-Javadoc)
    *

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java Mon Jul 29 21:08:03 2013
@@ -274,4 +274,13 @@ public class HiveQueryResultSet extends 
     return fetchSize;
   }
 
+  public <T> T getObject(String columnLabel, Class<T> type)  throws SQLException {
+    //JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
+  public <T> T getObject(int columnIndex, Class<T> type)  throws SQLException {
+    //JDK 1.7
+    throw new SQLException("Method not supported");
+  }
 }

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveResultSetMetaData.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveResultSetMetaData.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveResultSetMetaData.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveResultSetMetaData.java Mon Jul 29 21:08:03 2013
@@ -82,14 +82,7 @@ public class HiveResultSetMetaData imple
   }
 
   public String getColumnTypeName(int column) throws SQLException {
-    if (columnTypes == null) {
-      throw new SQLException(
-          "Could not determine column type name for ResultSet");
-    }
-
-    if (column < 1 || column > columnTypes.size()) {
-      throw new SQLException("Invalid column value: " + column);
-    }
+    validateColumnType(column);
 
     // we need to convert the Hive type to the SQL type name
     // TODO: this would be better handled in an enum
@@ -112,10 +105,14 @@ public class HiveResultSetMetaData imple
       return serdeConstants.BIGINT_TYPE_NAME;
     } else if ("timestamp".equalsIgnoreCase(type)) {
       return serdeConstants.TIMESTAMP_TYPE_NAME;
+    } else if ("date".equalsIgnoreCase(type)) {
+      return serdeConstants.DATE_TYPE_NAME;
     } else if ("decimal".equalsIgnoreCase(type)) {
       return serdeConstants.DECIMAL_TYPE_NAME;
     } else if ("binary".equalsIgnoreCase(type)) {
       return serdeConstants.BINARY_TYPE_NAME;
+    } else if ("void".equalsIgnoreCase(type)) {
+      return serdeConstants.VOID_TYPE_NAME;
     } else if (type.startsWith("map<")) {
       return serdeConstants.STRING_TYPE_NAME;
     } else if (type.startsWith("array<")) {
@@ -153,7 +150,17 @@ public class HiveResultSetMetaData imple
   }
 
   public boolean isCaseSensitive(int column) throws SQLException {
-    throw new SQLException("Method not supported");
+    validateColumnType(column);
+
+    // we need to convert the Hive type to the SQL type name
+    // TODO: this would be better handled in an enum
+    String type = columnTypes.get(column - 1);
+
+    if("string".equalsIgnoreCase(type)) {
+      return true;
+    } else {
+      return false;
+    }
   }
 
   public boolean isCurrency(int column) throws SQLException {
@@ -194,4 +201,14 @@ public class HiveResultSetMetaData imple
     throw new SQLException("Method not supported");
   }
 
+  protected void validateColumnType(int column) throws SQLException {
+    if (columnTypes == null) {
+      throw new SQLException(
+          "Could not determine column type name for ResultSet");
+    }
+
+    if (column < 1 || column > columnTypes.size()) {
+      throw new SQLException("Invalid column value: " + column);
+    }    
+  }
 }

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java Mon Jul 29 21:08:03 2013
@@ -163,6 +163,11 @@ public class HiveStatement implements ja
     isClosed = true;
   }
 
+  public void closeOnCompletion() throws SQLException {
+    // JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -450,6 +455,11 @@ public class HiveStatement implements ja
     return isClosed;
   }
 
+  public boolean isCloseOnCompletion() throws SQLException {
+    // JDK 1.7
+    throw new SQLException("Method not supported");
+  }
+
   /*
    * (non-Javadoc)
    *

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/JdbcColumn.java Mon Jul 29 21:08:03 2013
@@ -75,6 +75,8 @@ public class JdbcColumn {
     case Types.INTEGER:
     case Types.BIGINT:
       return columnPrecision(columnType) + 1; // allow +/-
+    case Types.DATE:
+      return 10;
     case Types.TIMESTAMP:
       return columnPrecision(columnType);
 
@@ -110,6 +112,8 @@ public class JdbcColumn {
       return 7;
     case Types.DOUBLE:
       return 15;
+    case Types.DATE:
+      return 10;
     case Types.TIMESTAMP:
       return 29;
     case Types.DECIMAL:
@@ -128,6 +132,7 @@ public class JdbcColumn {
     case Types.SMALLINT:
     case Types.INTEGER:
     case Types.BIGINT:
+    case Types.DATE:
       return 0;
     case Types.FLOAT:
       return 7;

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/Utils.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/Utils.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/Utils.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/Utils.java Mon Jul 29 21:08:03 2013
@@ -128,6 +128,8 @@ public class Utils {
       return Types.INTEGER;
     } else if ("bigint".equalsIgnoreCase(type)) {
       return Types.BIGINT;
+    } else if ("date".equalsIgnoreCase(type)) {
+      return Types.DATE;
     } else if ("timestamp".equalsIgnoreCase(type)) {
       return Types.TIMESTAMP;
     } else if ("decimal".equalsIgnoreCase(type)) {
@@ -190,8 +192,19 @@ public class Utils {
       connParams.setEmbeddedMode(true);
       return connParams;
     }
+
     URI jdbcURI = URI.create(uri.substring(URI_JDBC_PREFIX.length()));
 
+    //Check to prevent unintentional use of embedded mode. A missing "/" can
+    // to separate the 'path' portion of URI can result in this.
+    //The missing "/" common typo while using secure mode, eg of such url -
+    // jdbc:hive2://localhost:10000;principal=hive/HiveServer2Host@YOUR-REALM.COM
+    if((jdbcURI.getAuthority() != null) && (jdbcURI.getHost()==null)){
+       throw new IllegalArgumentException("Bad URL format. Hostname not found "
+           + " in authority part of the url: " + jdbcURI.getAuthority()
+           + ". Are you missing a '/' after the hostname ?");
+    }
+
     connParams.setHost(jdbcURI.getHost());
     if (connParams.getHost() == null) {
       connParams.setEmbeddedMode(true);

Modified: hive/branches/tez/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java (original)
+++ hive/branches/tez/jdbc/src/test/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java Mon Jul 29 21:08:03 2013
@@ -150,7 +150,9 @@ public class TestJdbcDriver extends Test
         + " c15 struct<r:int,s:struct<a:int,b:string>>,"
         + " c16 array<struct<m:map<string,string>,n:int>>,"
         + " c17 timestamp, "
-        + " c18 decimal) comment'" + dataTypeTableComment
+        + " c18 decimal,"
+        + " c19 binary,"
+        + " c20 date) comment'" + dataTypeTableComment
             +"' partitioned by (dt STRING)");
     assertFalse(res.next());
 
@@ -226,6 +228,7 @@ public class TestJdbcDriver extends Test
         + tableName
         + " where   'not?param?not?param' <> 'not_param??not_param' and ?=? "
         + " and 1=? and 2=? and 3.0=? and 4.0=? and 'test\\'string\"'=? and 5=? and ?=? "
+        + " and date '2012-01-01' = date ?"
         + " ) t  select '2011-03-25' ddate,'China',true bv, 10 num limit 10";
 
      ///////////////////////////////////////////////
@@ -296,7 +299,7 @@ public class TestJdbcDriver extends Test
     assertNotNull(
         "Execute the invalid setted sql statement should throw exception",
         expectedException);
-    
+
     // setObject to the yet unknown type java.util.Date
     expectedException = null;
     try {
@@ -326,6 +329,7 @@ public class TestJdbcDriver extends Test
     ps.setObject(8, 5L); //setLong
     ps.setObject(9, (byte) 1); //setByte
     ps.setObject(10, (byte) 1); //setByte
+    ps.setString(11, "2012-01-01"); //setString
 
     ps.setMaxRows(2);
     return ps;
@@ -345,6 +349,7 @@ public class TestJdbcDriver extends Test
     ps.setLong(8, 5L); //setLong
     ps.setByte(9, (byte) 1); //setByte
     ps.setByte(10, (byte) 1); //setByte
+    ps.setString(11, "2012-01-01"); //setString
 
     ps.setMaxRows(2);
     return ps;
@@ -392,6 +397,17 @@ public class TestJdbcDriver extends Test
     doTestSelectAll(tableName, 100, 20);
   }
 
+  public void testNullType() throws Exception {
+    Statement stmt = con.createStatement();
+    try {
+      ResultSet res = stmt.executeQuery("select null from " + dataTypeTableName);
+      assertTrue(res.next());
+      assertNull(res.getObject(1));
+    } finally {
+      stmt.close();
+    }
+  }
+
   public void testDataTypes() throws Exception {
     Statement stmt = con.createStatement();
 
@@ -427,6 +443,8 @@ public class TestJdbcDriver extends Test
     assertEquals(null, res.getString(17));
     assertEquals(null, res.getTimestamp(17));
     assertEquals(null, res.getBigDecimal(18));
+    assertEquals(null, res.getString(20));
+    assertEquals(null, res.getDate(20));
 
     // row 3
     assertTrue(res.next());
@@ -449,6 +467,8 @@ public class TestJdbcDriver extends Test
     assertEquals("2012-04-22 09:00:00.123456789", res.getString(17));
     assertEquals("2012-04-22 09:00:00.123456789", res.getTimestamp(17).toString());
     assertEquals("123456789.0123456", res.getBigDecimal(18).toString());
+    assertEquals("2013-01-01", res.getString(20));
+    assertEquals("2013-01-01", res.getDate(20).toString());
 
     // test getBoolean rules on non-boolean columns
     assertEquals(true, res.getBoolean(1));
@@ -839,13 +859,14 @@ public class TestJdbcDriver extends Test
 
     ResultSet res = stmt.executeQuery(
         "select c1, c2, c3, c4, c5 as a, c6, c7, c8, c9, c10, c11, c12, " +
-        "c1*2, sentences(null, null, null) as b, c17, c18 from " + dataTypeTableName + " limit 1");
+        "c1*2, sentences(null, null, null) as b, c17, c18, c20 from " + dataTypeTableName +
+        " limit 1");
     ResultSetMetaData meta = res.getMetaData();
 
     ResultSet colRS = con.getMetaData().getColumns(null, null,
         dataTypeTableName.toLowerCase(), null);
 
-    assertEquals(16, meta.getColumnCount());
+    assertEquals(17, meta.getColumnCount());
 
     assertTrue(colRS.next());
 
@@ -1055,6 +1076,13 @@ public class TestJdbcDriver extends Test
     assertEquals(Integer.MAX_VALUE, meta.getPrecision(16));
     assertEquals(Integer.MAX_VALUE, meta.getScale(16));
 
+    assertEquals("c20", meta.getColumnName(17));
+    assertEquals(Types.DATE, meta.getColumnType(17));
+    assertEquals("date", meta.getColumnTypeName(17));
+    assertEquals(10, meta.getColumnDisplaySize(17));
+    assertEquals(10, meta.getPrecision(17));
+    assertEquals(0, meta.getScale(17));
+
     for (int i = 1; i <= meta.getColumnCount(); i++) {
       assertFalse(meta.isAutoIncrement(i));
       assertFalse(meta.isCurrency(i));

Modified: hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/branches/tez/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java Mon Jul 29 21:08:03 2013
@@ -150,7 +150,8 @@ public class TestJdbcDriver2 extends Tes
         + " c16 array<struct<m:map<string,string>,n:int>>,"
         + " c17 timestamp, "
         + " c18 decimal, "
-        + " c19 binary) comment'" + dataTypeTableComment
+        + " c19 binary, "
+        + " c20 date) comment'" + dataTypeTableComment
             +"' partitioned by (dt STRING)");
 
     stmt.execute("load data local inpath '"
@@ -195,6 +196,24 @@ public class TestJdbcDriver2 extends Tes
         expectedException);
   }
 
+  public void testBadURL() throws Exception {
+    checkBadUrl("jdbc:hive2://localhost:10000;principal=test");
+    checkBadUrl("jdbc:hive2://localhost:10000;" +
+    		"principal=hive/HiveServer2Host@YOUR-REALM.COM");
+    checkBadUrl("jdbc:hive2://localhost:10000test");
+  }
+
+
+  private void checkBadUrl(String url) throws SQLException {
+    try{
+      DriverManager.getConnection(url, "", "");
+      fail("should have thrown IllegalArgumentException but did not ");
+    }catch(IllegalArgumentException i){
+      assertTrue(i.getMessage().contains("Bad URL format. Hostname not found "
+          + " in authority part of the url"));
+    }
+  }
+
   public void testDataTypes2() throws Exception {
     Statement stmt = con.createStatement();
 
@@ -260,6 +279,7 @@ public class TestJdbcDriver2 extends Tes
         + tableName
         + " where   'not?param?not?param' <> 'not_param??not_param' and ?=? "
         + " and 1=? and 2=? and 3.0=? and 4.0=? and 'test\\'string\"'=? and 5=? and ?=? "
+        + " and date '2012-01-01' = date ?"
         + " ) t  select '2011-03-25' ddate,'China',true bv, 10 num limit 10";
 
      ///////////////////////////////////////////////
@@ -279,6 +299,7 @@ public class TestJdbcDriver2 extends Tes
       ps.setLong(8, 5L);
       ps.setByte(9, (byte) 1);
       ps.setByte(10, (byte) 1);
+      ps.setString(11, "2012-01-01");
 
       ps.setMaxRows(2);
 
@@ -383,6 +404,17 @@ public class TestJdbcDriver2 extends Tes
     doTestSelectAll(tableName, 100, 20);
   }
 
+  public void testNullType() throws Exception {
+    Statement stmt = con.createStatement();
+    try {
+      ResultSet res = stmt.executeQuery("select null from " + dataTypeTableName);
+      assertTrue(res.next());
+      assertNull(res.getObject(1));
+    } finally {
+      stmt.close();
+    }
+  }
+
   public void testDataTypes() throws Exception {
     Statement stmt = con.createStatement();
 
@@ -416,6 +448,8 @@ public class TestJdbcDriver2 extends Tes
     assertEquals(null, res.getString(17));
     assertEquals(null, res.getString(18));
     assertEquals(null, res.getString(19));
+    assertEquals(null, res.getString(20));
+    assertEquals(null, res.getDate(20));
 
     // row 2
     assertTrue(res.next());
@@ -439,6 +473,8 @@ public class TestJdbcDriver2 extends Tes
     assertEquals(null, res.getTimestamp(17));
     assertEquals(null, res.getBigDecimal(18));
     assertEquals(null, res.getString(19));
+    assertEquals(null, res.getString(20));
+    assertEquals(null, res.getDate(20));
 
     // row 3
     assertTrue(res.next());
@@ -462,11 +498,19 @@ public class TestJdbcDriver2 extends Tes
     assertEquals("2012-04-22 09:00:00.123456789", res.getTimestamp(17).toString());
     assertEquals("123456789.0123456", res.getBigDecimal(18).toString());
     assertEquals("abcd", res.getString(19));
+    assertEquals("2013-01-01", res.getString(20));
+    assertEquals("2013-01-01", res.getDate(20).toString());
 
     // test getBoolean rules on non-boolean columns
     assertEquals(true, res.getBoolean(1));
     assertEquals(true, res.getBoolean(4));
 
+    // test case sensitivity
+    assertFalse(meta.isCaseSensitive(1));
+    assertFalse(meta.isCaseSensitive(2));
+    assertFalse(meta.isCaseSensitive(3));
+    assertTrue(meta.isCaseSensitive(4));
+
     // no more rows
     assertFalse(res.next());
   }
@@ -864,13 +908,14 @@ public class TestJdbcDriver2 extends Tes
 
     ResultSet res = stmt.executeQuery(
         "select c1, c2, c3, c4, c5 as a, c6, c7, c8, c9, c10, c11, c12, " +
-        "c1*2, sentences(null, null, null) as b, c17, c18 from " + dataTypeTableName + " limit 1");
+        "c1*2, sentences(null, null, null) as b, c17, c18, c20 from " + dataTypeTableName +
+        " limit 1");
     ResultSetMetaData meta = res.getMetaData();
 
     ResultSet colRS = con.getMetaData().getColumns(null, null,
         dataTypeTableName.toLowerCase(), null);
 
-    assertEquals(16, meta.getColumnCount());
+    assertEquals(17, meta.getColumnCount());
 
     assertTrue(colRS.next());
 
@@ -1071,6 +1116,13 @@ public class TestJdbcDriver2 extends Tes
     assertEquals(Integer.MAX_VALUE, meta.getPrecision(16));
     assertEquals(Integer.MAX_VALUE, meta.getScale(16));
 
+    assertEquals("c20", meta.getColumnName(17));
+    assertEquals(Types.DATE, meta.getColumnType(17));
+    assertEquals("date", meta.getColumnTypeName(17));
+    assertEquals(10, meta.getColumnDisplaySize(17));
+    assertEquals(10, meta.getPrecision(17));
+    assertEquals(0, meta.getScale(17));
+
     for (int i = 1; i <= meta.getColumnCount(); i++) {
       assertFalse(meta.isAutoIncrement(i));
       assertFalse(meta.isCurrency(i));
@@ -1283,4 +1335,6 @@ public class TestJdbcDriver2 extends Tes
     assertNull(conn);
   }
 
+
+
 }

Modified: hive/branches/tez/metastore/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/build.xml?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/build.xml (original)
+++ hive/branches/tez/metastore/build.xml Mon Jul 29 21:08:03 2013
@@ -81,7 +81,7 @@
   <target name="model-enhance" depends="model-compile" unless="enhanceModel.notRequired" >
     <echo message="Project: ${ant.project.name}"/>
     <taskdef name="datanucleusenhancer"
-                classname="org.datanucleus.enhancer.tools.EnhancerTask">
+                classname="org.datanucleus.enhancer.EnhancerTask">
        <classpath refid="classpath"/>
    </taskdef>
 

Modified: hive/branches/tez/metastore/ivy.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/ivy.xml?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/ivy.xml (original)
+++ hive/branches/tez/metastore/ivy.xml Mon Jul 29 21:08:03 2013
@@ -31,20 +31,18 @@
     <dependency org="org.antlr" name="ST4" rev="${ST4.version}" transitive="false"/><!-- manually added (antlr dep), bad POM -->
     <dependency org="org.apache.hive" name="hive-serde" rev="${version}"
                 conf="compile->default" />
-    <dependency org="commons-dbcp" name="commons-dbcp" rev="${commons-dbcp.version}">
-      <exclude module="commons-pool" />
-      <exclude org="org.apache.geronimo.specs" module="geronimo-jta_1.1_spec"/>
-    </dependency>
+    <dependency org="com.jolbox" name="bonecp" rev="${BoneCP.version}"/>
     <dependency org="commons-pool" name="commons-pool" rev="${commons-pool.version}"/>
-    <dependency org="org.datanucleus" name="datanucleus-connectionpool" rev="${datanucleus-connectionpool.version}"
-                transitive="false"/>
-     <dependency org="org.datanucleus" name="datanucleus-core" rev="${datanucleus-core.version}"
-                transitive="false"/>
-    <dependency org="org.datanucleus" name="datanucleus-enhancer" rev="${datanucleus-enhancer.version}"
+    <dependency org="org.datanucleus" name="datanucleus-api-jdo" rev="${datanucleus-api-jdo.version}">
+        <exclude org="javax.jdo" module="jdo2-api"/>
+        <exclude org="junit" module="junit"/>
+        <exclude org="log4j" module="log4j"/>
+    </dependency>
+    <dependency org="org.datanucleus" name="datanucleus-core" rev="${datanucleus-core.version}"
                 transitive="false"/>
     <dependency org="org.datanucleus" name="datanucleus-rdbms" rev="${datanucleus-rdbms.version}"
                 transitive="false"/>
-    <dependency org="javax.jdo" name="jdo2-api" rev="${jdo-api.version}"
+    <dependency org="javax.jdo" name="jdo-api" rev="${jdo-api.version}"
                 transitive="false"/>
     <dependency org="org.apache.derby" name="derby" rev="${derby.version}"/>
     <dependency org="asm" name="asm" rev="${asm.version}"/>

Modified: hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java (original)
+++ hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java Mon Jul 29 21:08:03 2013
@@ -125,7 +125,7 @@ public class SkewedInfo implements org.a
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))));
     tmpMap.put(_Fields.SKEWED_COL_VALUE_LOCATION_MAPS, new org.apache.thrift.meta_data.FieldMetaData("skewedColValueLocationMaps", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SkewedValueList.class),
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SkewedValueList.class), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SkewedInfo.class, metaDataMap);

Modified: hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedValueList.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedValueList.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedValueList.java (original)
+++ hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedValueList.java Mon Jul 29 21:08:03 2013
@@ -106,8 +106,8 @@ public class SkewedValueList implements 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.SKEWED_VALUE_LIST, new org.apache.thrift.meta_data.FieldMetaData("skewedValueList", org.apache.thrift.TFieldRequirementType.DEFAULT,
-        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+    tmpMap.put(_Fields.SKEWED_VALUE_LIST, new org.apache.thrift.meta_data.FieldMetaData("skewedValueList", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SkewedValueList.class, metaDataMap);
@@ -339,7 +339,7 @@ public class SkewedValueList implements 
       while (true)
       {
         schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (schemeField.id) {
@@ -357,7 +357,7 @@ public class SkewedValueList implements 
                 iprot.readListEnd();
               }
               struct.setSkewedValueListIsSet(true);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;

Modified: hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java (original)
+++ hive/branches/tez/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java Mon Jul 29 21:08:03 2013
@@ -47102,17 +47102,17 @@ public class ThriftHiveMetastore {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.PARTITION_SPECS, new org.apache.thrift.meta_data.FieldMetaData("partitionSpecs", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+      tmpMap.put(_Fields.PARTITION_SPECS, new org.apache.thrift.meta_data.FieldMetaData("partitionSpecs", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-      tmpMap.put(_Fields.SOURCE_DB, new org.apache.thrift.meta_data.FieldMetaData("source_db", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SOURCE_DB, new org.apache.thrift.meta_data.FieldMetaData("source_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.SOURCE_TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("source_table_name", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SOURCE_TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("source_table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.DEST_DB, new org.apache.thrift.meta_data.FieldMetaData("dest_db", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.DEST_DB, new org.apache.thrift.meta_data.FieldMetaData("dest_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.DEST_TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("dest_table_name", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.DEST_TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("dest_table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(exchange_partition_args.class, metaDataMap);
@@ -47945,9 +47945,9 @@ public class ThriftHiveMetastore {
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.O4, new org.apache.thrift.meta_data.FieldMetaData("o4", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(exchange_partition_result.class, metaDataMap);
@@ -48459,7 +48459,7 @@ public class ThriftHiveMetastore {
         while (true)
         {
           schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
             break;
           }
           switch (schemeField.id) {
@@ -48468,7 +48468,7 @@ public class ThriftHiveMetastore {
                 struct.success = new Partition();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -48477,7 +48477,7 @@ public class ThriftHiveMetastore {
                 struct.o1 = new MetaException();
                 struct.o1.read(iprot);
                 struct.setO1IsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -48486,7 +48486,7 @@ public class ThriftHiveMetastore {
                 struct.o2 = new NoSuchObjectException();
                 struct.o2.read(iprot);
                 struct.setO2IsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -48495,7 +48495,7 @@ public class ThriftHiveMetastore {
                 struct.o3 = new InvalidObjectException();
                 struct.o3.read(iprot);
                 struct.setO3IsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -48504,7 +48504,7 @@ public class ThriftHiveMetastore {
                 struct.o4 = new InvalidInputException();
                 struct.o4.read(iprot);
                 struct.setO4IsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -48726,17 +48726,17 @@ public class ThriftHiveMetastore {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.PART_VALS, new org.apache.thrift.meta_data.FieldMetaData("part_vals", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.PART_VALS, new org.apache.thrift.meta_data.FieldMetaData("part_vals", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-      tmpMap.put(_Fields.USER_NAME, new org.apache.thrift.meta_data.FieldMetaData("user_name", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.USER_NAME, new org.apache.thrift.meta_data.FieldMetaData("user_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.GROUP_NAMES, new org.apache.thrift.meta_data.FieldMetaData("group_names", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.GROUP_NAMES, new org.apache.thrift.meta_data.FieldMetaData("group_names", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_partition_with_auth_args.class, metaDataMap);
@@ -49283,7 +49283,7 @@ public class ThriftHiveMetastore {
         while (true)
         {
           schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
             break;
           }
           switch (schemeField.id) {
@@ -49291,7 +49291,7 @@ public class ThriftHiveMetastore {
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
                 struct.db_name = iprot.readString();
                 struct.setDb_nameIsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -49299,7 +49299,7 @@ public class ThriftHiveMetastore {
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
                 struct.tbl_name = iprot.readString();
                 struct.setTbl_nameIsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -49317,7 +49317,7 @@ public class ThriftHiveMetastore {
                   iprot.readListEnd();
                 }
                 struct.setPart_valsIsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -49325,7 +49325,7 @@ public class ThriftHiveMetastore {
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
                 struct.user_name = iprot.readString();
                 struct.setUser_nameIsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -49343,7 +49343,7 @@ public class ThriftHiveMetastore {
                   iprot.readListEnd();
                 }
                 struct.setGroup_namesIsSet(true);
-              } else {
+              } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
@@ -49594,11 +49594,11 @@ public class ThriftHiveMetastore {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Partition.class)));
-      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_partition_with_auth_result.class, metaDataMap);

Modified: hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Mon Jul 29 21:08:03 2013
@@ -4374,10 +4374,11 @@ public class HiveMetaStore extends Thrif
             conf.getVar(HiveConf.ConfVars.METASTORE_KERBEROS_KEYTAB_FILE),
             conf.getVar(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL));
         // start delegation token manager
-        saslServer.startDelegationTokenSecretManager(conf);
+        HMSHandler hmsHandler = new HMSHandler("new db based metaserver", conf);
+        saslServer.startDelegationTokenSecretManager(conf, hmsHandler);
         transFactory = saslServer.createTransportFactory();
-        processor = saslServer.wrapProcessor(new ThriftHiveMetastore.Processor<IHMSHandler>(
-            newHMSHandler("new db based metaserver", conf)));
+        processor = saslServer.wrapProcessor(
+          new ThriftHiveMetastore.Processor<HMSHandler>(hmsHandler));
         LOG.info("Starting DB backed MetaStore Server in Secure Mode");
       } else {
         // we are in unsecure mode.

Modified: hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java (original)
+++ hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java Mon Jul 29 21:08:03 2013
@@ -43,6 +43,7 @@ import javax.jdo.PersistenceManagerFacto
 import javax.jdo.Query;
 import javax.jdo.Transaction;
 import javax.jdo.datastore.DataStoreCache;
+import javax.jdo.identity.IntIdentity;
 
 import org.antlr.runtime.CharStream;
 import org.antlr.runtime.CommonTokenStream;
@@ -96,9 +97,11 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.model.MColumnDescriptor;
 import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
 import org.apache.hadoop.hive.metastore.model.MDatabase;
+import org.apache.hadoop.hive.metastore.model.MDelegationToken;
 import org.apache.hadoop.hive.metastore.model.MFieldSchema;
 import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege;
 import org.apache.hadoop.hive.metastore.model.MIndex;
+import org.apache.hadoop.hive.metastore.model.MMasterKey;
 import org.apache.hadoop.hive.metastore.model.MOrder;
 import org.apache.hadoop.hive.metastore.model.MPartition;
 import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege;
@@ -258,7 +261,7 @@ public class ObjectStore implements RawS
     return prop;
   }
 
-  private static PersistenceManagerFactory getPMF() {
+  private static synchronized PersistenceManagerFactory getPMF() {
     if (pmf == null) {
       pmf = JDOHelper.getPersistenceManagerFactory(prop);
       DataStoreCache dsc = pmf.getDataStoreCache();
@@ -5269,4 +5272,204 @@ public class ObjectStore implements RawS
     return delCnt;
   }
 
+  private MDelegationToken getTokenFrom(String tokenId) {
+    Query query = pm.newQuery(MDelegationToken.class, "tokenIdentifier == tokenId");
+    query.declareParameters("java.lang.String tokenId");
+    query.setUnique(true);
+    return (MDelegationToken)query.execute(tokenId);
+  }
+
+  @Override
+  public boolean addToken(String tokenId, String delegationToken) {
+
+    LOG.debug("Begin executing addToken");
+    boolean committed = false;
+    MDelegationToken token;
+    try{
+      openTransaction();
+      token = getTokenFrom(tokenId);
+      if (token == null) {
+        // add Token, only if it already doesn't exist
+        pm.makePersistent(new MDelegationToken(tokenId, delegationToken));
+      }
+      committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing addToken with status : " + committed);
+    return committed && (token == null);
+  }
+
+  @Override
+  public boolean removeToken(String tokenId) {
+
+    LOG.debug("Begin executing removeToken");
+    boolean committed = false;
+    MDelegationToken token;
+    try{
+      openTransaction();
+      token = getTokenFrom(tokenId);
+      if (null != token) {
+        pm.deletePersistent(token);
+      }
+      committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing removeToken with status : " + committed);
+    return committed && (token != null);
+  }
+
+  @Override
+  public String getToken(String tokenId) {
+
+    LOG.debug("Begin executing getToken");
+    boolean committed = false;
+    MDelegationToken token;
+    try{
+      openTransaction();
+      token = getTokenFrom(tokenId);
+      if (null != token) {
+        pm.retrieve(token);
+      }
+      committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing getToken with status : " + committed);
+    return (null == token) ? null : token.getTokenStr();
+  }
+
+  @Override
+  public List<String> getAllTokenIdentifiers() {
+
+    LOG.debug("Begin executing getAllTokenIdentifiers");
+    boolean committed = false;
+    List<MDelegationToken> tokens;
+    try{
+      openTransaction();
+      Query query = pm.newQuery(MDelegationToken.class);
+      tokens = (List<MDelegationToken>) query.execute();
+      pm.retrieveAll(tokens);
+      committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing getAllTokenIdentifers with status : " + committed);
+    List<String> tokenIdents = new ArrayList<String>(tokens.size());
+
+    for (MDelegationToken token : tokens) {
+      tokenIdents.add(token.getTokenIdentifier());
+    }
+    return tokenIdents;
+  }
+
+  @Override
+  public int addMasterKey(String key) throws MetaException{
+    LOG.debug("Begin executing addMasterKey");
+    boolean committed = false;
+    MMasterKey masterKey = new MMasterKey(key);
+    try{
+      openTransaction();
+      pm.makePersistent(masterKey);
+      committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing addMasterKey with status : " + committed);
+    if (committed) {
+      return ((IntIdentity)pm.getObjectId(masterKey)).getKey();
+    } else {
+      throw new MetaException("Failed to add master key.");
+    }
+  }
+
+  @Override
+  public void updateMasterKey(Integer id, String key) throws NoSuchObjectException, MetaException {
+    LOG.debug("Begin executing updateMasterKey");
+    boolean committed = false;
+    MMasterKey masterKey;
+    try{
+    openTransaction();
+    Query query = pm.newQuery(MMasterKey.class, "keyId == id");
+    query.declareParameters("java.lang.Integer id");
+    query.setUnique(true);
+    masterKey = (MMasterKey)query.execute(id);
+    if (null != masterKey) {
+      masterKey.setMasterKey(key);
+    }
+    committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing updateMasterKey with status : " + committed);
+    if (null == masterKey) {
+      throw new NoSuchObjectException("No key found with keyId: " + id);
+    }
+    if (!committed) {
+      throw new MetaException("Though key is found, failed to update it. " + id);
+    }
+  }
+
+  @Override
+  public boolean removeMasterKey(Integer id) {
+    LOG.debug("Begin executing removeMasterKey");
+    boolean success = false;
+    MMasterKey masterKey;
+    try{
+    openTransaction();
+    Query query = pm.newQuery(MMasterKey.class, "keyId == id");
+    query.declareParameters("java.lang.Integer id");
+    query.setUnique(true);
+    masterKey = (MMasterKey)query.execute(id);
+    if (null != masterKey) {
+      pm.deletePersistent(masterKey);
+    }
+    success = commitTransaction();
+    } finally {
+      if(!success) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing removeMasterKey with status : " + success);
+    return (null != masterKey) && success;
+  }
+
+  @Override
+  public String[] getMasterKeys() {
+    LOG.debug("Begin executing getMasterKeys");
+    boolean committed = false;
+    List<MMasterKey> keys;
+    try{
+      openTransaction();
+      Query query = pm.newQuery(MMasterKey.class);
+      keys = (List<MMasterKey>) query.execute();
+      pm.retrieveAll(keys);
+      committed = commitTransaction();
+    } finally {
+      if(!committed) {
+        rollbackTransaction();
+      }
+    }
+    LOG.debug("Done executing getMasterKeys with status : " + committed);
+    String[] masterKeys = new String[keys.size()];
+
+    for (int i = 0; i < keys.size(); i++) {
+      masterKeys[i] = keys.get(i).getMasterKey();
+    }
+    return masterKeys;
+  }
+
 }

Modified: hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java (original)
+++ hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java Mon Jul 29 21:08:03 2013
@@ -419,6 +419,21 @@ public interface RawStore extends Config
 
  public abstract long cleanupEvents();
 
+ public abstract boolean addToken(String tokenIdentifier, String delegationToken);
 
+ public abstract boolean removeToken(String tokenIdentifier);
+
+ public abstract String getToken(String tokenIdentifier);
+
+ public abstract List<String> getAllTokenIdentifiers();
+
+ public abstract int addMasterKey(String key) throws MetaException;
+
+ public abstract void updateMasterKey(Integer seqNo, String key)
+     throws NoSuchObjectException, MetaException;
+
+ public abstract boolean removeMasterKey(Integer keySeq);
+
+ public abstract String[] getMasterKeys();
 
 }

Modified: hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java (original)
+++ hive/branches/tez/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java Mon Jul 29 21:08:03 2013
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.common.cla
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -126,6 +127,12 @@ public class RetryingHMSHandler implemen
           // Due to reflection, the jdo exception is wrapped in
           // invocationTargetException
           caughtException = e.getCause();
+        } else if (e.getCause() instanceof NoSuchObjectException) {
+          String methodName = method.getName();
+          if (!methodName.startsWith("get_table") && !methodName.startsWith("get_partition")) {
+            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
+          }
+          throw e.getCause();
         } else if (e.getCause() instanceof MetaException && e.getCause().getCause() != null
             && e.getCause().getCause() instanceof javax.jdo.JDOException) {
           // The JDOException may be wrapped further in a MetaException

Modified: hive/branches/tez/metastore/src/model/package.jdo
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/model/package.jdo?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/model/package.jdo (original)
+++ hive/branches/tez/metastore/src/model/package.jdo Mon Jul 29 21:08:03 2013
@@ -91,10 +91,13 @@
           <embedded>
             <field name="name">
               <column name="FIELD_NAME" length="128" jdbc-type="VARCHAR"/>
-              </field>
+            </field>
             <field name="type">
               <column name="FIELD_TYPE" length="767" jdbc-type="VARCHAR"  allows-null="false"/>
             </field>
+            <field name="comment" >
+              <column name="COMMENT" length="256" jdbc-type="VARCHAR" allows-null="true"/>
+            </field>
           </embedded>
         </element>
       </field>
@@ -228,6 +231,9 @@
             <field name="type">
               <column name="TYPE_NAME" length="4000" jdbc-type="VARCHAR"  allows-null="false"/>
             </field>
+            <field name="comment">
+              <column name="COMMENT" length="256" jdbc-type="VARCHAR" allows-null="true"/>
+            </field>
           </embedded>
         </element>
       </field>
@@ -262,7 +268,9 @@
       <field name="outputFormat">
         <column name="OUTPUT_FORMAT" length="4000" jdbc-type="VARCHAR"/>
       </field>
-      <field name="isCompressed"/>
+      <field name="isCompressed">
+        <column name="IS_COMPRESSED"/>
+      </field>
       <field name="isStoredAsSubDirectories">
         <column name="IS_STOREDASSUBDIRECTORIES"/>
       </field>
@@ -754,6 +762,30 @@
       </field>
 
     </class>
+    
+    <class name="MMasterKey" table="MASTER_KEYS" identity-type="application" detachable="true">
+
+      <field name="keyId" primary-key="true" value-strategy="identity">
+        <column name="KEY_ID" jdbc-type="integer" />
+      </field>
+        
+      <field name="masterKey">
+        <column name="MASTER_KEY" length="767" jdbc-type="VARCHAR" />
+      </field>  
+      
+    </class>
+
+    <class name="MDelegationToken" table="DELEGATION_TOKENS" identity-type="application" detachable="true">
+
+      <field name="tokenIdentifier" primary-key="true">
+        <column name="TOKEN_IDENT" length="767" jdbc-type="VARCHAR" />
+      </field>  
+
+      <field name="tokenStr">
+        <column name="TOKEN" length="767" jdbc-type="VARCHAR" />
+      </field>
+            
+    </class>    
 
     <class name="MTableColumnStatistics" table="TAB_COL_STATS" identity-type="datastore" detachable="true">
       <datastore-identity>
@@ -870,3 +902,4 @@
 
   </package>
 </jdo>
+

Modified: hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java (original)
+++ hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java Mon Jul 29 21:08:03 2013
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.metastore;
 
 import java.util.List;
+import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configurable;
@@ -527,4 +528,36 @@ public class DummyRawStoreControlledComm
       InvalidInputException {
     return objectStore.updatePartitionColumnStatistics(statsObj, partVals);
   }
+
+  public boolean addToken(String tokenIdentifier, String delegationToken) {
+    return false;
+  }
+ 
+  public boolean removeToken(String tokenIdentifier) {
+    return false;
+  }
+ 
+  public String getToken(String tokenIdentifier) {
+    return "";
+  }
+
+  public List<String> getAllTokenIdentifiers() {
+    return new ArrayList<String>();
+  }
+
+  public int addMasterKey(String key) throws MetaException {
+    return -1;
+  }
+
+  public void updateMasterKey(Integer seqNo, String key)
+    throws NoSuchObjectException, MetaException {}
+
+  public boolean removeMasterKey(Integer keySeq) {
+    return false;
+  }
+
+  public String[] getMasterKeys() {
+    return new String[0];
+  }
+
 }

Modified: hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java (original)
+++ hive/branches/tez/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java Mon Jul 29 21:08:03 2013
@@ -515,6 +515,45 @@ public class DummyRawStoreForJdoConnecti
   }
 
   @Override
+  public boolean addToken(String tokenIdentifier, String delegationToken) {
+    return false;
+  }
+
+  @Override
+  public boolean removeToken(String tokenIdentifier) {
+    return false;
+  }
+
+  @Override
+  public String getToken(String tokenIdentifier) {
+    return null;
+  }
+
+  @Override
+  public List<String> getAllTokenIdentifiers() {
+    return null;
+  }
+
+  @Override
+  public int addMasterKey(String key) {
+    return 0;
+  }
+
+  @Override
+  public void updateMasterKey(Integer seqNo, String key) {
+  }
+
+  @Override
+  public boolean removeMasterKey(Integer keySeq) {
+    return false;
+  }
+
+  @Override
+  public String[] getMasterKeys() {
+    return null;
+  }
+
+  @Override
   public ColumnStatistics getTableColumnStatistics(String dbName, String tableName, String colName)
       throws MetaException, NoSuchObjectException {
     return null;

Modified: hive/branches/tez/ql/build.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/build.xml?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/ql/build.xml (original)
+++ hive/branches/tez/ql/build.xml Mon Jul 29 21:08:03 2013
@@ -82,19 +82,24 @@
               logDirectory="${test.log.dir}/clientpositive"
               hadoopVersion="${hadoopVersion}"/>
 
-    <qtestgen hiveRootDirectory="${hive.root}"
-              outputDirectory="${test.build.src}/org/apache/hive/beeline/util" 
-              templatePath="${ql.test.template.dir}" template="TestBeeLineDriver.vm" 
-              queryDirectory="${ql.test.query.clientpositive.dir}" 
-              queryFile="${qfile}"
-              excludeQueryFile="${ql.test.beelinepositive.exclude}"
-              queryFileRegex="${qfile_regex}"
-              clusterMode="${clustermode}"
-              runDisabled="${run_disabled}"
-              resultsDirectory="${ql.test.results.beelinepositive.dir}" className="TestBeeLineDriver"
-              logFile="${test.log.dir}/testbeelinedrivergen.log"
-              logDirectory="${test.log.dir}/beelinepositive"
-              hadoopVersion="${hadoopVersion}" />
+    <if>
+      <matches string="${iterate.hive.all}" pattern="beeline"/>
+      <then>
+        <qtestgen hiveRootDirectory="${hive.root}"
+                  outputDirectory="${test.build.src}/org/apache/hive/beeline/util" 
+                  templatePath="${ql.test.template.dir}" template="TestBeeLineDriver.vm" 
+                  queryDirectory="${ql.test.query.clientpositive.dir}" 
+                  queryFile="${qfile}"
+                  excludeQueryFile="${ql.test.beelinepositive.exclude}"
+                  queryFileRegex="${qfile_regex}"
+                  clusterMode="${clustermode}"
+                  runDisabled="${run_disabled}"
+                  resultsDirectory="${ql.test.results.beelinepositive.dir}" className="TestBeeLineDriver"
+                  logFile="${test.log.dir}/testbeelinedrivergen.log"
+                  logDirectory="${test.log.dir}/beelinepositive"
+                  hadoopVersion="${hadoopVersion}" />
+      </then>
+    </if>
 
     <if>
       <not>

Modified: hive/branches/tez/ql/if/queryplan.thrift
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/if/queryplan.thrift?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/ql/if/queryplan.thrift (original)
+++ hive/branches/tez/ql/if/queryplan.thrift Mon Jul 29 21:08:03 2013
@@ -54,6 +54,8 @@ enum OperatorType {
   HASHTABLESINK,
   HASHTABLEDUMMY,
   PTF,
+  MUX,
+  DEMUX,
 }
 
 struct Operator {

Modified: hive/branches/tez/ql/ivy.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/ivy.xml?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/ql/ivy.xml (original)
+++ hive/branches/tez/ql/ivy.xml Mon Jul 29 21:08:03 2013
@@ -34,6 +34,12 @@
                 conf="test->default" transitive="false"/>
     <dependency org="org.apache.hive" name="hive-testutils" rev="${version}"
                 conf="test->default" transitive="false"/>
+    <dependency org="org.apache.hbase" name="hbase" rev="${hbase.version}"
+                conf="test->default" transitive="false">
+      <artifact name="hbase" type="jar"/>
+      <artifact name="hbase" type="test-jar" ext="jar"
+                m:classifier="tests"/>
+    </dependency>
     <dependency org="com.google.protobuf" name="protobuf-java" 
                 rev="${protobuf.version}" transitive="false"/>
     <dependency org="org.iq80.snappy" name="snappy"