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 2014/02/14 11:53:39 UTC

svn commit: r1568236 [4/25] - in /hive/branches/tez: ./ bin/ cli/src/java/org/apache/hadoop/hive/cli/ cli/src/test/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/common/type/ common/src/java/org/apache/hadoop/hive/conf/ conf/ data/c...

Modified: hive/branches/tez/hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java (original)
+++ hive/branches/tez/hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java Fri Feb 14 10:53:31 2014
@@ -345,6 +345,7 @@ public class HCatCreateTableDesc {
     private String fileFormat;
     private String location;
     private String storageHandler;
+    private String serDe;
     private Map<String, String> tblProps;
     private boolean ifNotExists;
     private String dbName;
@@ -472,6 +473,17 @@ public class HCatCreateTableDesc {
     }
 
     /**
+     * SerDe.
+     *
+     * @param serDe the SerDe implementation's class-name.
+     * @return the builder
+     */
+    public Builder serDe(String serDe) {
+        this.serDe = serDe;
+        return this;
+    }
+
+    /**
      * Builds the HCatCreateTableDesc.
      *
      * @return HCatCreateTableDesc
@@ -494,7 +506,7 @@ public class HCatCreateTableDesc {
       desc.location = this.location;
       desc.tblProps = this.tblProps;
       desc.sortCols = this.sortCols;
-      desc.serde = null;
+      desc.serde = this.serDe;
       if (!StringUtils.isEmpty(fileFormat)) {
         desc.fileFormat = fileFormat;
         if ("SequenceFile".equalsIgnoreCase(fileFormat)) {

Modified: hive/branches/tez/hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java (original)
+++ hive/branches/tez/hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java Fri Feb 14 10:53:31 2014
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.io.orc.
 import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
+import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler;
 import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
@@ -51,6 +52,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertArrayEquals;
@@ -179,6 +181,24 @@ public class TestHCatClient {
       OrcSerde.class.getName()));
     assertTrue(table1.getCols().equals(cols));
 
+    // Check that serDe settings stick.
+    String tableFour = "table4";
+    String nonDefaultSerDe = "com.my.custom.SerDe";
+    HCatCreateTableDesc tableDesc4 = 
+        HCatCreateTableDesc.create(db, tableFour, cols).serDe(nonDefaultSerDe).build();
+    client.createTable(tableDesc4);
+    HCatTable table4 = client.getTable(db, tableFour);
+    assertEquals("SerDe libraries don't match!", nonDefaultSerDe, table4.getSerdeLib());
+    client.dropTable(db, tableFour, true);
+
+    // Check that serDe settings don't stick when a storageHandler is used.
+    tableDesc4 = HCatCreateTableDesc.create(db, tableFour, cols).
+        serDe(nonDefaultSerDe).storageHandler(DefaultStorageHandler.class.getName()).build();
+    client.createTable(tableDesc4);
+    table4 = client.getTable(db, tableFour);
+    assertNotSame("SerDe libraries shouldn't have matched!", nonDefaultSerDe, table4.getSerdeLib());
+    client.dropTable(db, tableFour, true);
+
     client.close();
   }
 

Modified: hive/branches/tez/hcatalog/webhcat/svr/src/main/bin/webhcat_config.sh
URL: http://svn.apache.org/viewvc/hive/branches/tez/hcatalog/webhcat/svr/src/main/bin/webhcat_config.sh?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/hcatalog/webhcat/svr/src/main/bin/webhcat_config.sh (original)
+++ hive/branches/tez/hcatalog/webhcat/svr/src/main/bin/webhcat_config.sh Fri Feb 14 10:53:31 2014
@@ -111,6 +111,9 @@ if [ ! -d ${TEMPLETON_HOME}/share/webhca
     die "TEMPLETON_HOME=${TEMPLETON_HOME} is invalid";
 fi
 
+source $WEBHCAT_PREFIX/bin/common.sh
+find_hadoop_home
+
 #====================================
 #determine where hadoop is
 #====================================

Modified: hive/branches/tez/itests/hcatalog-unit/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hcatalog-unit/pom.xml?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hcatalog-unit/pom.xml (original)
+++ hive/branches/tez/itests/hcatalog-unit/pom.xml Fri Feb 14 10:53:31 2014
@@ -192,6 +192,14 @@
           <version>${pig.version}</version>
           <scope>test</scope>
         </dependency>
+        <dependency>
+          <!--this should be automatically brought in by Pig, it's not in Pig 0.12 due to a bug
+              in Pig which requires it This is fixed in Pig's pom file in ASF trunk (pig 13)-->
+          <groupId>joda-time</groupId>
+          <artifactId>joda-time</artifactId>
+          <version>2.2</version>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
    <profile>
@@ -332,6 +340,14 @@
           <classifier>h2</classifier>
           <scope>test</scope>
         </dependency>
+        <dependency>
+          <!--this should be automatically brought in by Pig, it's not in Pig 0.12 due to a bug
+              in Pig which requires it This is fixed in Pig's pom file in ASF trunk (pig 13)-->
+          <groupId>joda-time</groupId>
+          <artifactId>joda-time</artifactId>
+          <version>2.2</version>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
   </profiles>

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java Fri Feb 14 10:53:31 2014
@@ -1147,24 +1147,16 @@ public class TestJdbcDriver extends Test
 
     ResultSet res = stmt.getResultSet();
     assertTrue(res.next());
-    assertEquals("database", res.getString(1));
-    assertEquals("default", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("table", res.getString(1));
+    assertEquals("default", res.getString(1));
     assertEquals(dataTypeTableName, res.getString(2));
-    assertTrue(res.next());
-    assertEquals("principalName", res.getString(1));
-    assertEquals("hive_test_user", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("principalType", res.getString(1));
-    assertEquals("USER", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("privilege", res.getString(1));
-    assertEquals("Select", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("grantTime", res.getString(1));
-    assertTrue(res.next());
-    assertEquals("grantor", res.getString(1));
+    assertEquals("", res.getString(3));     // partition
+    assertEquals("", res.getString(4));     // column
+    assertEquals("hive_test_user", res.getString(5));
+    assertEquals("USER", res.getString(6));
+    assertEquals("Select", res.getString(7));
+    assertEquals(false, res.getBoolean(8)); // grant option
+    assertEquals(-1, res.getLong(9));
+    assertNotNull(res.getString(10));       // grantor
     assertFalse(res.next());
     res.close();
   }
@@ -1177,9 +1169,9 @@ public class TestJdbcDriver extends Test
 
     ResultSet res = stmt.getResultSet();
     assertTrue(res.next());
-    assertEquals("role1", res.getString(1));
-    assertTrue(res.next());
     assertEquals("PUBLIC", res.getString(1));
+    assertTrue(res.next());
+    assertEquals("role1", res.getString(1));
     res.close();
   }
 }

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java Fri Feb 14 10:53:31 2014
@@ -2498,8 +2498,8 @@ public abstract class TestHiveMetaStore 
    */
   private void updateTableNameInDB(String oldTableName, String newTableName) throws SQLException {
     String connectionStr = HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTORECONNECTURLKEY);
-    int interval= HiveConf.getIntVar(hiveConf, HiveConf.ConfVars.METASTOREINTERVAL);
-    int attempts = HiveConf.getIntVar(hiveConf, HiveConf.ConfVars.METASTOREATTEMPTS);
+    int interval= 1;
+    int attempts = 1;
 
 
     Utilities.SQLCommand<Void> execUpdate = new Utilities.SQLCommand<Void>() {

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java Fri Feb 14 10:53:31 2014
@@ -1947,40 +1947,29 @@ public class TestJdbcDriver2 {
     }
   }
 
+  @Test
   public void testShowGrant() throws SQLException {
     Statement stmt = con.createStatement();
     stmt.execute("grant select on table " + dataTypeTableName + " to user hive_test_user");
     stmt.execute("show grant user hive_test_user on table " + dataTypeTableName);
 
     ResultSet res = stmt.getResultSet();
-    ResultSetMetaData metaData = res.getMetaData();
-
-    assertEquals("property", metaData.getColumnName(1));
-    assertEquals("value", metaData.getColumnName(2));
-
-    assertTrue(res.next());
-    assertEquals("database", res.getString(1));
-    assertEquals("default", res.getString(2));
     assertTrue(res.next());
-    assertEquals("table", res.getString(1));
+    assertEquals("default", res.getString(1));
     assertEquals(dataTypeTableName, res.getString(2));
-    assertTrue(res.next());
-    assertEquals("principalName", res.getString(1));
-    assertEquals("hive_test_user", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("principalType", res.getString(1));
-    assertEquals("USER", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("privilege", res.getString(1));
-    assertEquals("Select", res.getString(2));
-    assertTrue(res.next());
-    assertEquals("grantTime", res.getString(1));
-    assertTrue(res.next());
-    assertEquals("grantor", res.getString(1));
+    assertEquals("", res.getString(3));     // partition
+    assertEquals("", res.getString(4));     // column
+    assertEquals("hive_test_user", res.getString(5));
+    assertEquals("USER", res.getString(6));
+    assertEquals("Select", res.getString(7));
+    assertEquals(false, res.getBoolean(8)); // grant option
+    assertEquals(-1, res.getLong(9));
+    assertNotNull(res.getString(10));       // grantor
     assertFalse(res.next());
     res.close();
   }
 
+  @Test
   public void testShowRoleGrant() throws SQLException {
     Statement stmt = con.createStatement();
     stmt.execute("create role role1");
@@ -1989,8 +1978,9 @@ public class TestJdbcDriver2 {
 
     ResultSet res = stmt.getResultSet();
     assertTrue(res.next());
+    assertEquals("PUBLIC", res.getString(1));
+    assertTrue(res.next());
     assertEquals("role1", res.getString(1));
-    assertFalse(res.next());
     res.close();
   }
 }

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java Fri Feb 14 10:53:31 2014
@@ -28,6 +28,8 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.Statement;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -52,7 +54,8 @@ import org.junit.Test;
       String dataFileDir = conf.get("test.data.files").replace('\\', '/')
           .replace("c:", "");
       dataFilePath = new Path(dataFileDir, "kv1.txt");
-      miniHS2.start();
+      Map<String, String> confOverlay = new HashMap<String, String>();
+      miniHS2.start(confOverlay);
     }
 
     @Before

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java Fri Feb 14 10:53:31 2014
@@ -26,12 +26,13 @@ import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
-//import org.apache.hive.service.miniHS2.MiniHS2;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -43,11 +44,17 @@ public class TestSSL {
   private static final String KEY_STORE_PASSWORD = "HiveJdbc";
   private static final String JAVA_TRUST_STORE_PROP = "javax.net.ssl.trustStore";
   private static final String JAVA_TRUST_STORE_PASS_PROP = "javax.net.ssl.trustStorePassword";
+  private static final String HS2_BINARY_MODE = "binary";
+  private static final String HS2_HTTP_MODE = "http";
+  private static final String HS2_HTTP_ENDPOINT = "cliservice";
+  private static final String HS2_BINARY_AUTH_MODE = "NONE";
+  private static final String HS2_HTTP_AUTH_MODE = "NOSASL";
 
   private MiniHS2 miniHS2 = null;
   private static HiveConf conf = new HiveConf();
   private Connection hs2Conn = null;
   private String dataFileDir = conf.get("test.data.files");
+  private Map<String, String> confOverlay;
 
   @BeforeClass
   public static void beforeTest() throws Exception {
@@ -62,6 +69,7 @@ public class TestSSL {
     }
     dataFileDir = dataFileDir.replace('\\', '/').replace("c:", "");
     miniHS2 = new MiniHS2(conf);
+    confOverlay = new HashMap<String, String>();
   }
 
   @After
@@ -82,7 +90,10 @@ public class TestSSL {
    */
   @Test
   public void testInvalidConfig() throws Exception {
-    miniHS2.start();
+    clearSslConfOverlay(confOverlay);
+    // Test in binary mode
+    setBinaryConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
     DriverManager.setLoginTimeout(4);
     try {
       hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() + ";ssl=true;sslTrustStore=" +
@@ -104,7 +115,25 @@ public class TestSSL {
       // expected error
       assertEquals("08S01", e.getSQLState().trim());
     }
+    miniHS2.stop();
 
+    // Test in http mode with ssl properties specified in url
+    System.clearProperty(JAVA_TRUST_STORE_PROP);
+    System.clearProperty(JAVA_TRUST_STORE_PASS_PROP);
+    setHttpConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
+    try {
+      hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() +
+          ";ssl=true;sslTrustStore=" + dataFileDir + File.separator +
+          TRUST_STORE_NAME + ";trustStorePassword=" + KEY_STORE_PASSWORD +
+          "?hive.server2.transport.mode=" + HS2_HTTP_MODE +
+          ";hive.server2.thrift.http.path=" + HS2_HTTP_ENDPOINT,
+          System.getProperty("user.name"), "bar");
+      fail("SSL connection should fail with NON-SSL server");
+    } catch (SQLException e) {
+      // expected error
+      assertEquals("08S01", e.getSQLState().trim());
+    }
   }
 
   /***
@@ -113,9 +142,11 @@ public class TestSSL {
    */
   @Test
   public void testConnectionMismatch() throws Exception {
-    miniHS2.setConfProperty(ConfVars.HIVE_SERVER2_USE_SSL.varname, "true");
-    miniHS2.setConfProperty(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname, "");
-    miniHS2.start();
+    setSslConfOverlay(confOverlay);
+    // Test in binary mode
+    setBinaryConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
+    // Start HS2 with SSL
     try {
       hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL(), System.getProperty("user.name"), "bar");
       fail("NON SSL connection should fail with SSL server");
@@ -132,6 +163,23 @@ public class TestSSL {
       // expected error
       assertEquals("08S01", e.getSQLState().trim());
     }
+    miniHS2.stop();
+
+    // Test in http mode
+    setHttpConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
+    try {
+      hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() +
+          ";ssl=false;sslTrustStore=" + dataFileDir + File.separator +
+          TRUST_STORE_NAME + ";trustStorePassword=" + KEY_STORE_PASSWORD +
+          "?hive.server2.transport.mode=" + HS2_HTTP_MODE +
+          ";hive.server2.thrift.http.path=" + HS2_HTTP_ENDPOINT,
+          System.getProperty("user.name"), "bar");
+      fail("NON SSL connection should fail with SSL server");
+    } catch (SQLException e) {
+      // expected error
+      assertEquals("08S01", e.getSQLState().trim());
+    }
 
   }
 
@@ -141,14 +189,29 @@ public class TestSSL {
    */
   @Test
   public void testSSLConnectionWithURL() throws Exception {
+    setSslConfOverlay(confOverlay);
+    // Test in binary mode
+    setBinaryConfOverlay(confOverlay);
     // Start HS2 with SSL
-    startSslSever();
+    miniHS2.start(confOverlay);
 
     // make SSL connection
     hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() + ";ssl=true;sslTrustStore=" +
         dataFileDir + File.separator + TRUST_STORE_NAME + ";trustStorePassword=" +
         KEY_STORE_PASSWORD, System.getProperty("user.name"), "bar");
+    hs2Conn.close();
+    miniHS2.stop();
 
+    // Test in http mode
+    setHttpConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
+    // make SSL connection
+    hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() +
+        ";ssl=true;sslTrustStore=" + dataFileDir + File.separator +
+        TRUST_STORE_NAME + ";trustStorePassword=" + KEY_STORE_PASSWORD +
+        "?hive.server2.transport.mode=" + HS2_HTTP_MODE +
+        ";hive.server2.thrift.http.path=" + HS2_HTTP_ENDPOINT,
+        System.getProperty("user.name"), "bar");
     hs2Conn.close();
   }
 
@@ -158,15 +221,28 @@ public class TestSSL {
    */
   @Test
   public void testSSLConnectionWithProperty() throws Exception {
+    setSslConfOverlay(confOverlay);
+    // Test in binary mode
+    setBinaryConfOverlay(confOverlay);
     // Start HS2 with SSL
-    startSslSever();
+    miniHS2.start(confOverlay);
 
     System.setProperty(JAVA_TRUST_STORE_PROP, dataFileDir + File.separator + TRUST_STORE_NAME );
     System.setProperty(JAVA_TRUST_STORE_PASS_PROP, KEY_STORE_PASSWORD);
     // make SSL connection
     hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() + ";ssl=true",
         System.getProperty("user.name"), "bar");
+    hs2Conn.close();
+    miniHS2.stop();
 
+    // Test in http mode
+    setHttpConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
+    // make SSL connection
+    hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() +
+        ";ssl=true;" + "?hive.server2.transport.mode=" + HS2_HTTP_MODE +
+        ";hive.server2.thrift.http.path=" + HS2_HTTP_ENDPOINT,
+        System.getProperty("user.name"), "bar");
     hs2Conn.close();
   }
 
@@ -176,46 +252,111 @@ public class TestSSL {
    */
   @Test
   public void testSSLFetch() throws Exception {
+    setSslConfOverlay(confOverlay);
+    // Test in binary mode
+    setBinaryConfOverlay(confOverlay);
     // Start HS2 with SSL
-    startSslSever();
+    miniHS2.start(confOverlay);
+
+    String tableName = "sslTab";
+    Path dataFilePath = new Path(dataFileDir, "kv1.txt");
 
     // make SSL connection
     hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() + ";ssl=true;sslTrustStore=" +
         dataFileDir + File.separator + TRUST_STORE_NAME + ";trustStorePassword=" +
         KEY_STORE_PASSWORD, System.getProperty("user.name"), "bar");
 
-    String tableName = "sslTab";
+    // Set up test data
+    setupTestTableWithData(tableName, dataFilePath, hs2Conn);
+
     Statement stmt = hs2Conn.createStatement();
-    Path dataFilePath = new Path(dataFileDir, "kv1.txt");
+    ResultSet res = stmt.executeQuery("SELECT * FROM " + tableName);
+    int rowCount = 0;
+    while (res.next()) {
+      ++rowCount;
+      assertEquals("val_" + res.getInt(1), res.getString(2));
+    }
+    // read result over SSL
+    assertEquals(500, rowCount);
 
-    stmt.execute("set hive.support.concurrency = false");
+    hs2Conn.close();
+  }
 
-    stmt.execute("drop table if exists " + tableName);
-    stmt.execute("create table " + tableName
-        + " (under_col int comment 'the under column', value string)");
+  /**
+   * Start HS2 in Http mode with SSL enabled, open a SSL connection and fetch data
+   * @throws Exception
+   */
+  @Test
+  public void testSSLFetchHttp() throws Exception {
+    setSslConfOverlay(confOverlay);
+    // Test in http mode
+    setHttpConfOverlay(confOverlay);
+    miniHS2.start(confOverlay);
 
-    // load data
-    stmt.execute("load data local inpath '"
-        + dataFilePath.toString() + "' into table " + tableName);
+    String tableName = "sslTab";
+    Path dataFilePath = new Path(dataFileDir, "kv1.txt");
 
+    // make SSL connection
+    hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL() +
+        ";ssl=true;sslTrustStore=" + dataFileDir + File.separator +
+        TRUST_STORE_NAME + ";trustStorePassword=" + KEY_STORE_PASSWORD +
+        "?hive.server2.transport.mode=" + HS2_HTTP_MODE +
+        ";hive.server2.thrift.http.path=" + HS2_HTTP_ENDPOINT,
+        System.getProperty("user.name"), "bar");
+
+    // Set up test data
+    setupTestTableWithData(tableName, dataFilePath, hs2Conn);
+    Statement stmt = hs2Conn.createStatement();
     ResultSet res = stmt.executeQuery("SELECT * FROM " + tableName);
     int rowCount = 0;
     while (res.next()) {
       ++rowCount;
       assertEquals("val_" + res.getInt(1), res.getString(2));
     }
-
     // read result over SSL
     assertEquals(500, rowCount);
+
+    hs2Conn.close();
   }
 
-  private void startSslSever () throws Exception {
-    miniHS2.setConfProperty(ConfVars.HIVE_SERVER2_USE_SSL.varname, "true");
-    miniHS2.setConfProperty(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname,
+  private void setupTestTableWithData(String tableName, Path dataFilePath,
+      Connection hs2Conn) throws Exception {
+    Statement stmt = hs2Conn.createStatement();
+    stmt.execute("set hive.support.concurrency = false");
+
+    stmt.execute("drop table if exists " + tableName);
+    stmt.execute("create table " + tableName
+        + " (under_col int comment 'the under column', value string)");
+
+    // load data
+    stmt.execute("load data local inpath '"
+        + dataFilePath.toString() + "' into table " + tableName);
+    stmt.close();
+  }
+
+  private void setSslConfOverlay(Map<String, String> confOverlay) {
+    confOverlay.put(ConfVars.HIVE_SERVER2_USE_SSL.varname, "true");
+    confOverlay.put(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname,
         dataFileDir + File.separator +  KEY_STORE_NAME);
-    miniHS2.setConfProperty(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname,
+    confOverlay.put(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname,
         KEY_STORE_PASSWORD);
-    miniHS2.start();
   }
 
+  private void clearSslConfOverlay(Map<String, String> confOverlay) {
+    confOverlay.put(ConfVars.HIVE_SERVER2_USE_SSL.varname, "false");
+  }
+
+  // Currently http mode works with server in NOSASL auth mode & doesn't support doAs
+  private void setHttpConfOverlay(Map<String, String> confOverlay) {
+    confOverlay.put(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, HS2_HTTP_MODE);
+    confOverlay.put(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH.varname, HS2_HTTP_ENDPOINT);
+    confOverlay.put(ConfVars.HIVE_SERVER2_AUTHENTICATION.varname,  HS2_HTTP_AUTH_MODE);
+    confOverlay.put(ConfVars.HIVE_SERVER2_ENABLE_DOAS.varname, "false");
+  }
+
+  private void setBinaryConfOverlay(Map<String, String> confOverlay) {
+    confOverlay.put(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, HS2_BINARY_MODE);
+    confOverlay.put(ConfVars.HIVE_SERVER2_AUTHENTICATION.varname,  HS2_BINARY_AUTH_MODE);
+    confOverlay.put(ConfVars.HIVE_SERVER2_ENABLE_DOAS.varname, "true");
+  }
 }

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java Fri Feb 14 10:53:31 2014
@@ -20,6 +20,7 @@ package org.apache.hive.jdbc.miniHS2;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Map;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -33,32 +34,40 @@ import org.apache.hive.service.cli.CLISe
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
+import org.apache.hive.service.cli.thrift.ThriftHttpCLIService;
 import org.apache.hive.service.server.HiveServer2;
 
 import com.google.common.io.Files;
 
-public class MiniHS2 extends AbstarctHiveService {
+public class MiniHS2 extends AbstractHiveService {
   private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
   private HiveServer2 hiveServer2 = null;
   private final File baseDir;
   private static final AtomicLong hs2Counter = new AtomicLong();
+  private static final String HS2_BINARY_MODE = "binary";
+  private static final String HS2_HTTP_MODE = "http";
 
   public MiniHS2(HiveConf hiveConf) throws IOException {
-    super(hiveConf, "localhost", MetaStoreUtils.findFreePort());
+    super(hiveConf, "localhost", MetaStoreUtils.findFreePort(), MetaStoreUtils.findFreePort());
     baseDir =  Files.createTempDir();
     setWareHouseDir("file://" + baseDir.getPath() + File.separator + "warehouse");
     String metaStoreURL =  "jdbc:derby:" + baseDir.getAbsolutePath() + File.separator + "test_metastore-" +
         hs2Counter.incrementAndGet() + ";create=true";
-
     System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, metaStoreURL);
     hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, metaStoreURL);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
     hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, getHost());
-    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, getPort());
+    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, getBinaryPort());
+    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, getHttpPort());
     HiveMetaStore.HMSHandler.resetDefaultDBFlag();
   }
 
-  public void start() throws Exception {
+  public void start(Map<String, String> confOverlay) throws Exception {
     hiveServer2 = new HiveServer2();
+    // Set confOverlay parameters
+    for (Map.Entry<String, String> entry : confOverlay.entrySet()) {
+      setConfProperty(entry.getKey(), entry.getValue());
+    }
     hiveServer2.init(getHiveConf());
     hiveServer2.start();
     waitForStartup();
@@ -80,14 +89,23 @@ public class MiniHS2 extends AbstarctHiv
   public CLIServiceClient getServiceClientInternal() {
     for (Service service : hiveServer2.getServices()) {
       if (service instanceof ThriftBinaryCLIService) {
-        return new ThriftCLIServiceClient((ThriftBinaryCLIService)service);
+        return new ThriftCLIServiceClient((ThriftBinaryCLIService) service);
+      }
+      if (service instanceof ThriftHttpCLIService) {
+        return new ThriftCLIServiceClient((ThriftHttpCLIService) service);
       }
     }
-    throw new IllegalStateException("HS2 not running Thrift service");
+    throw new IllegalStateException("HiveServer2 not running Thrift service");
   }
 
   public String getJdbcURL() {
-    return "jdbc:hive2://" + getHost() + ":" + getPort() + "/default";
+    String transportMode = getConfProperty(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
+    if(transportMode != null && (transportMode.equalsIgnoreCase(HS2_HTTP_MODE))) {
+      return "jdbc:hive2://" + getHost() + ":" + getHttpPort() + "/default";
+    }
+    else {
+      return "jdbc:hive2://" + getHost() + ":" + getBinaryPort() + "/default";
+    }
   }
 
   public static String getJdbcDriverName() {
@@ -103,7 +121,7 @@ public class MiniHS2 extends AbstarctHiv
       Thread.sleep(500L);
       waitTime += 500L;
       if (waitTime > startupTimeout) {
-        throw new TimeoutException("Couldn't access new HiveServer: " + getJdbcURL());
+        throw new TimeoutException("Couldn't access new HiveServer2: " + getJdbcURL());
       }
       try {
         sessionHandle = hs2Client.openSession("foo", "bar");

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java Fri Feb 14 10:53:31 2014
@@ -47,12 +47,12 @@ public class TestHiveServer2 {
 
   @Before
   public void setUp() throws Exception {
-    miniHS2.start();
     confOverlay = new HashMap<String, String>();
+    miniHS2.start(confOverlay);
   }
 
   @After
-  public void tearDown() {
+  public void tearDown() throws Exception {
     miniHS2.stop();
   }
 

Modified: hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java (original)
+++ hive/branches/tez/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java Fri Feb 14 10:53:31 2014
@@ -35,11 +35,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
-*
-* TestThriftHttpCLIService.
-* This tests ThriftCLIService started in http mode.
-*
-*/
+ *
+ * TestThriftHttpCLIService.
+ * This tests ThriftCLIService started in http mode.
+ *
+ */
 
 public class TestThriftHttpCLIService extends ThriftCLIServiceTest {
 

Modified: hive/branches/tez/itests/qtest/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/qtest/pom.xml?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/qtest/pom.xml (original)
+++ hive/branches/tez/itests/qtest/pom.xml Fri Feb 14 10:53:31 2014
@@ -36,10 +36,10 @@
     <run_disabled>false</run_disabled>
     <clustermode></clustermode>
     <execute.beeline.tests>false</execute.beeline.tests>
-    <minimr.query.files>stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q</minimr.query.files>
+    <minimr.query.files>stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q,index_bitmap3.q,ql_rewrite_gbtoidx.q,index_bitmap_auto.q</minimr.query.files>
     <minimr.query.negative.files>cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q,file_with_header_footer_negative.q</minimr.query.negative.files>
     <minitez.query.files>tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q</minitez.query.files>
-    <minitez.query.files.shared>join0.q,join1.q,auto_join0.q,auto_join1.q,bucket2.q,bucket3.q,bucket4.q,count.q,create_merge_compressed.q,cross_join.q,ctas.q,custom_input_output_format.q,disable_merge_for_bucketing.q,enforce_order.q,filter_join_breaktask.q,filter_join_breaktask2.q,groupby1.q,groupby2.q,groupby3.q,having.q,insert1.q,insert_into1.q,insert_into2.q,leftsemijoin.q,limit_pushdown.q,load_dyn_part1.q,load_dyn_part2.q,load_dyn_part3.q,mapjoin_mapjoin.q,mapreduce1.q,mapreduce2.q,merge1.q,merge2.q,metadata_only_queries.q,sample1.q,subquery_in.q,subquery_exists.q,vectorization_15.q,ptf.q</minitez.query.files.shared>
+    <minitez.query.files.shared>join0.q,join1.q,auto_join0.q,auto_join1.q,bucket2.q,bucket3.q,bucket4.q,count.q,create_merge_compressed.q,cross_join.q,ctas.q,custom_input_output_format.q,disable_merge_for_bucketing.q,enforce_order.q,filter_join_breaktask.q,filter_join_breaktask2.q,groupby1.q,groupby2.q,groupby3.q,having.q,insert1.q,insert_into1.q,insert_into2.q,leftsemijoin.q,limit_pushdown.q,load_dyn_part1.q,load_dyn_part2.q,load_dyn_part3.q,mapjoin_mapjoin.q,mapreduce1.q,mapreduce2.q,merge1.q,merge2.q,metadata_only_queries.q,sample1.q,subquery_in.q,subquery_exists.q,vectorization_15.q,ptf.q,stats_counter.q,stats_noscan_1.q,stats_counter_partitioned.q</minitez.query.files.shared>
     <beeline.positive.exclude>add_part_exist.q,alter1.q,alter2.q,alter4.q,alter5.q,alter_rename_partition.q,alter_rename_partition_authorization.q,archive.q,archive_corrupt.q,archive_multi.q,archive_mr_1806.q,archive_multi_mr_1806.q,authorization_1.q,authorization_2.q,authorization_4.q,authorization_5.q,authorization_6.q,authorization_7.q,ba_table1.q,ba_table2.q,ba_table3.q,ba_table_udfs.q,binary_table_bincolserde.q,binary_table_colserde.q,cluster.q,columnarserde_create_shortcut.q,combine2.q,constant_prop.q,create_nested_type.q,create_or_replace_view.q,create_struct_table.q,create_union_table.q,database.q,database_location.q,database_properties.q,ddltime.q,describe_database_json.q,drop_database_removes_partition_dirs.q,escape1.q,escape2.q,exim_00_nonpart_empty.q,exim_01_nonpart.q,exim_02_00_part_empty.q,exim_02_part.q,exim_03_nonpart_over_compat.q,exim_04_all_part.q,exim_04_evolved_parts.q,exim_05_some_part.q,exim_06_one_part.q,exim_07_all_part_over_nonoverlap.q,exim_08_nonpart_rena
 me.q,exim_09_part_spec_nonoverlap.q,exim_10_external_managed.q,exim_11_managed_external.q,exim_12_external_location.q,exim_13_managed_location.q,exim_14_managed_location_over_existing.q,exim_15_external_part.q,exim_16_part_external.q,exim_17_part_managed.q,exim_18_part_external.q,exim_19_00_part_external_location.q,exim_19_part_external_location.q,exim_20_part_managed_location.q,exim_21_export_authsuccess.q,exim_22_import_exist_authsuccess.q,exim_23_import_part_authsuccess.q,exim_24_import_nonexist_authsuccess.q,global_limit.q,groupby_complex_types.q,groupby_complex_types_multi_single_reducer.q,index_auth.q,index_auto.q,index_auto_empty.q,index_bitmap.q,index_bitmap1.q,index_bitmap2.q,index_bitmap3.q,index_bitmap_auto.q,index_bitmap_rc.q,index_compact.q,index_compact_1.q,index_compact_2.q,index_compact_3.q,index_stale_partitioned.q,init_file.q,input16.q,input16_cc.q,input46.q,input_columnarserde.q,input_dynamicserde.q,input_lazyserde.q,input_testxpath3.q,input_testxpath4.q,insert2_o
 verwrite_partitions.q,insertexternal1.q,join_thrift.q,lateral_view.q,load_binary_data.q,load_exist_part_authsuccess.q,load_nonpart_authsuccess.q,load_part_authsuccess.q,loadpart_err.q,lock1.q,lock2.q,lock3.q,lock4.q,merge_dynamic_partition.q,multi_insert.q,multi_insert_move_tasks_share_dependencies.q,null_column.q,ppd_clusterby.q,query_with_semi.q,rename_column.q,sample6.q,sample_islocalmode_hook.q,set_processor_namespaces.q,show_tables.q,source.q,split_sample.q,str_to_map.q,transform1.q,udaf_collect_set.q,udaf_context_ngrams.q,udaf_histogram_numeric.q,udaf_ngrams.q,udaf_percentile_approx.q,udf_array.q,udf_bitmap_and.q,udf_bitmap_or.q,udf_explode.q,udf_format_number.q,udf_map.q,udf_map_keys.q,udf_map_values.q,udf_max.q,udf_min.q,udf_named_struct.q,udf_percentile.q,udf_printf.q,udf_sentences.q,udf_sort_array.q,udf_split.q,udf_struct.q,udf_substr.q,udf_translate.q,udf_union.q,udf_xpath.q,udtf_stack.q,view.q,virtual_column.q</beeline.positive.exclude>
   </properties>
 

Modified: hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java Fri Feb 14 10:53:31 2014
@@ -267,7 +267,7 @@ public class QTestUtil {
                   (new Path(dfsUriString,
                             "/build/ql/test/data/warehouse/")).toString());
     }
-    
+
     // Windows paths should be converted after MiniMrShim.setupConfiguration()
     // since setupConfiguration may overwrite configuration values.
     if (Shell.WINDOWS) {
@@ -330,12 +330,12 @@ public class QTestUtil {
     }
   }
 
-  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, String hadoopVer) 
+  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, String hadoopVer)
     throws Exception {
     this(outDir, logDir, clusterType, null, hadoopVer);
   }
 
-  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, 
+  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
       String confDir, String hadoopVer)
     throws Exception {
     this.outDir = outDir;
@@ -589,7 +589,7 @@ public class QTestUtil {
 
     List<String> roleNames = db.getAllRoleNames();
       for (String roleName : roleNames) {
-        if (!"PUBLIC".equals(roleName)) {
+        if (!"PUBLIC".equalsIgnoreCase(roleName) && !"ADMIN".equalsIgnoreCase(roleName)) {
           db.dropRole(roleName);
         }
     }
@@ -1376,7 +1376,7 @@ public class QTestUtil {
   }
 
   private static int executeCmd(Collection<String> args, String outFile, String errFile) throws Exception {
-    String[] cmdArray = (String[]) args.toArray(new String[args.size()]);
+    String[] cmdArray = args.toArray(new String[args.size()]);
     return executeCmd(cmdArray, outFile, errFile);
   }
 
@@ -1520,6 +1520,7 @@ public class QTestUtil {
       this.fname = fname;
     }
 
+    @Override
     public void run() {
       try {
         // assumption is that environment has already been cleaned once globally

Modified: hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java (original)
+++ hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java Fri Feb 14 10:53:31 2014
@@ -22,11 +22,12 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.session.SessionState;
 
 public class DummyAuthenticator implements HiveAuthenticationProvider {
 
-  private List<String> groupNames;
-  private String userName;
+  private final List<String> groupNames;
+  private final String userName;
   private Configuration conf;
 
   public DummyAuthenticator() {
@@ -56,8 +57,14 @@ public class DummyAuthenticator implemen
     this.conf = conf;
   }
 
+  @Override
   public Configuration getConf() {
     return this.conf;
   }
 
+  @Override
+  public void setSessionState(SessionState ss) {
+    //no op
+  }
+
 }

Modified: hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java (original)
+++ hive/branches/tez/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java Fri Feb 14 10:53:31 2014
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
  *
@@ -80,7 +81,7 @@ public class InjectableDummyAuthenticato
   @Override
   public void setConf(Configuration config) {
     try {
-      hmap = (HiveMetastoreAuthenticationProvider) hmapClass.newInstance();
+      hmap = hmapClass.newInstance();
     } catch (InstantiationException e) {
       throw new RuntimeException("Whoops, could not create an Authenticator of class " +
           hmapClass.getName());
@@ -102,4 +103,9 @@ public class InjectableDummyAuthenticato
     hmap.destroy();
   }
 
+  @Override
+  public void setSessionState(SessionState arg0) {
+    //no-op
+  }
+
 }

Modified: hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java (original)
+++ hive/branches/tez/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java Fri Feb 14 10:53:31 2014
@@ -18,6 +18,8 @@
 
 package org.apache.hive.jdbc;
 
+import java.io.FileInputStream;
+import java.security.KeyStore;
 import java.sql.Array;
 import java.sql.Blob;
 import java.sql.CallableStatement;
@@ -44,9 +46,12 @@ import java.util.Properties;
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
+import javax.net.ssl.SSLContext;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.auth.HiveAuthFactory;
@@ -60,7 +65,11 @@ import org.apache.hive.service.cli.thrif
 import org.apache.hive.service.cli.thrift.TOpenSessionResp;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
 import org.apache.hive.service.cli.thrift.TSessionHandle;
-import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLContexts;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.transport.THttpClient;
@@ -72,6 +81,7 @@ import org.apache.thrift.transport.TTran
  *
  */
 public class HiveConnection implements java.sql.Connection {
+  public static final Log LOG = LogFactory.getLog(HiveConnection.class.getName());
   private static final String HIVE_AUTH_TYPE= "auth";
   private static final String HIVE_AUTH_QOP = "sasl.qop";
   private static final String HIVE_AUTH_SIMPLE = "noSasl";
@@ -85,6 +95,9 @@ public class HiveConnection implements j
   private static final String HIVE_SSL_TRUST_STORE_PASSWORD = "trustStorePassword";
   private static final String HIVE_VAR_PREFIX = "hivevar:";
   private static final String HIVE_CONF_PREFIX = "hiveconf:";
+  // Currently supports JKS keystore format
+  // See HIVE-6286 (Add support for PKCS12 keystore format)
+  private static final String HIVE_SSL_TRUST_STORE_TYPE = "JKS";
 
   private final String jdbcURI;
   private final String host;
@@ -166,7 +179,7 @@ public class HiveConnection implements j
     transport = isHttpTransportMode() ? createHttpTransport() : createBinaryTransport();
     try {
       if (!transport.isOpen()) {
-        transport.open(); 
+        transport.open();
       }
     } catch (TTransportException e) {
       throw new SQLException("Could not open connection to "
@@ -175,9 +188,11 @@ public class HiveConnection implements j
   }
 
   private TTransport createHttpTransport() throws SQLException {
+    CloseableHttpClient httpClient;
     // http path should begin with "/"
     String httpPath;
-    httpPath = hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH.varname);
+    httpPath = hiveConfMap.get(
+        HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH.varname);
     if(httpPath == null) {
       httpPath = "/";
     }
@@ -185,12 +200,15 @@ public class HiveConnection implements j
       httpPath = "/" + httpPath;
     }
 
-    DefaultHttpClient httpClient = new DefaultHttpClient();
-    String httpUrl = hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname) +
-        "://" + host + ":" + port + httpPath;
-    httpClient.addRequestInterceptor(
-        new HttpBasicAuthInterceptor(getUserName(), getPasswd())
-        );
+    boolean useSsl = "true".equalsIgnoreCase(sessConfMap.get(HIVE_USE_SSL));
+
+    // Create an http client from the configs
+    httpClient = getHttpClient(useSsl);
+
+    // Create the http/https url
+    // JDBC driver will set up an https url if ssl is enabled, otherwise http
+    String schemeName = useSsl ? "https" : "http";
+    String httpUrl = schemeName +  "://" + host + ":" + port + httpPath;
     try {
       transport = new THttpClient(httpUrl, httpClient);
     }
@@ -202,6 +220,45 @@ public class HiveConnection implements j
     return transport;
   }
 
+  private CloseableHttpClient getHttpClient(Boolean useSsl) throws SQLException {
+    // Add an interceptor to pass username/password in the header
+    // for basic preemtive http authentication at the server
+    // In https mode, the entire information is encrypted
+    HttpRequestInterceptor authInterceptor = new HttpBasicAuthInterceptor(
+        getUserName(), getPasswd());
+    if (useSsl) {
+      String sslTrustStorePath = sessConfMap.get(HIVE_SSL_TRUST_STORE);
+      String sslTrustStorePassword = sessConfMap.get(
+          HIVE_SSL_TRUST_STORE_PASSWORD);
+      KeyStore sslTrustStore;
+      SSLContext sslContext;
+      if (sslTrustStorePath == null || sslTrustStorePath.isEmpty()) {
+        // Create a default client context based on standard JSSE trust material
+        sslContext = SSLContexts.createDefault();
+      } else {
+        // Pick trust store config from the given path
+        try {
+          sslTrustStore = KeyStore.getInstance(HIVE_SSL_TRUST_STORE_TYPE);
+          sslTrustStore.load(new FileInputStream(sslTrustStorePath),
+              sslTrustStorePassword.toCharArray());
+          sslContext = SSLContexts.custom().loadTrustMaterial(
+              sslTrustStore).build();
+        }
+        catch (Exception e) {
+          String msg =  "Could not create an https connection to " +
+              jdbcURI + ". " + e.getMessage();
+          throw new SQLException(msg, " 08S01", e);
+        }
+      }
+      return HttpClients.custom().setHostnameVerifier(SSLConnectionSocketFactory.ALLOW_ALL_HOSTNAME_VERIFIER).setSslcontext(
+          sslContext).addInterceptorFirst(authInterceptor).build();
+    }
+    else {
+      // Create a plain http client
+      return HttpClients.custom().addInterceptorFirst(authInterceptor).build();
+    }
+  }
+
   private TTransport createBinaryTransport() throws SQLException {
     try {
       // handle secure connection if specified
@@ -214,8 +271,8 @@ public class HiveConnection implements j
             try {
               saslQOP = SaslQOP.fromString(sessConfMap.get(HIVE_AUTH_QOP));
             } catch (IllegalArgumentException e) {
-              throw new SQLException("Invalid " + HIVE_AUTH_QOP + " parameter. " + e.getMessage(),
-                  "42000", e);
+              throw new SQLException("Invalid " + HIVE_AUTH_QOP +
+                  " parameter. " + e.getMessage(), "42000", e);
             }
           }
           saslProps.put(Sasl.QOP, saslQOP.toString());
@@ -264,8 +321,7 @@ public class HiveConnection implements j
   private boolean isHttpTransportMode() {
     String transportMode =
         hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
-    if(transportMode != null && (transportMode.equalsIgnoreCase("http") ||
-        transportMode.equalsIgnoreCase("https"))) {
+    if(transportMode != null && (transportMode.equalsIgnoreCase("http"))) {
       return true;
     }
     return false;
@@ -288,7 +344,7 @@ public class HiveConnection implements j
       protocol = openResp.getServerProtocolVersion();
       sessHandle = openResp.getSessionHandle();
     } catch (TException e) {
-      e.printStackTrace();
+      LOG.error("Error opening session", e);
       throw new SQLException("Could not establish connection to "
           + jdbcURI + ": " + e.getMessage(), " 08S01", e);
     }
@@ -370,6 +426,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#clearWarnings()
    */
 
+  @Override
   public void clearWarnings() throws SQLException {
     warningChain = null;
   }
@@ -380,6 +437,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#close()
    */
 
+  @Override
   public void close() throws SQLException {
     if (!isClosed) {
       TCloseSessionReq closeReq = new TCloseSessionReq(sessHandle);
@@ -402,6 +460,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#commit()
    */
 
+  @Override
   public void commit() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -414,6 +473,7 @@ public class HiveConnection implements j
    * java.lang.Object[])
    */
 
+  @Override
   public Array createArrayOf(String arg0, Object[] arg1) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -425,6 +485,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createBlob()
    */
 
+  @Override
   public Blob createBlob() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -436,6 +497,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createClob()
    */
 
+  @Override
   public Clob createClob() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -447,6 +509,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createNClob()
    */
 
+  @Override
   public NClob createNClob() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -458,6 +521,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createSQLXML()
    */
 
+  @Override
   public SQLXML createSQLXML() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -471,6 +535,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createStatement()
    */
 
+  @Override
   public Statement createStatement() throws SQLException {
     if (isClosed) {
       throw new SQLException("Can't create Statement, connection is closed");
@@ -484,6 +549,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createStatement(int, int)
    */
 
+  @Override
   public Statement createStatement(int resultSetType, int resultSetConcurrency)
       throws SQLException {
     if (resultSetConcurrency != ResultSet.CONCUR_READ_ONLY) {
@@ -504,6 +570,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createStatement(int, int, int)
    */
 
+  @Override
   public Statement createStatement(int resultSetType, int resultSetConcurrency,
       int resultSetHoldability) throws SQLException {
     // TODO Auto-generated method stub
@@ -516,6 +583,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#createStruct(java.lang.String, java.lang.Object[])
    */
 
+  @Override
   public Struct createStruct(String typeName, Object[] attributes)
       throws SQLException {
     // TODO Auto-generated method stub
@@ -528,6 +596,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getAutoCommit()
    */
 
+  @Override
   public boolean getAutoCommit() throws SQLException {
     return true;
   }
@@ -538,6 +607,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getCatalog()
    */
 
+  @Override
   public String getCatalog() throws SQLException {
     return "";
   }
@@ -548,6 +618,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getClientInfo()
    */
 
+  @Override
   public Properties getClientInfo() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -559,6 +630,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getClientInfo(java.lang.String)
    */
 
+  @Override
   public String getClientInfo(String name) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -570,6 +642,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getHoldability()
    */
 
+  @Override
   public int getHoldability() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -581,6 +654,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getMetaData()
    */
 
+  @Override
   public DatabaseMetaData getMetaData() throws SQLException {
     if (isClosed) {
       throw new SQLException("Connection is closed");
@@ -605,6 +679,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getTransactionIsolation()
    */
 
+  @Override
   public int getTransactionIsolation() throws SQLException {
     return Connection.TRANSACTION_NONE;
   }
@@ -615,6 +690,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getTypeMap()
    */
 
+  @Override
   public Map<String, Class<?>> getTypeMap() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -626,6 +702,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#getWarnings()
    */
 
+  @Override
   public SQLWarning getWarnings() throws SQLException {
     return warningChain;
   }
@@ -636,6 +713,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#isClosed()
    */
 
+  @Override
   public boolean isClosed() throws SQLException {
     return isClosed;
   }
@@ -646,6 +724,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#isReadOnly()
    */
 
+  @Override
   public boolean isReadOnly() throws SQLException {
     return false;
   }
@@ -656,6 +735,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#isValid(int)
    */
 
+  @Override
   public boolean isValid(int timeout) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -667,6 +747,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#nativeSQL(java.lang.String)
    */
 
+  @Override
   public String nativeSQL(String sql) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -678,6 +759,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareCall(java.lang.String)
    */
 
+  @Override
   public CallableStatement prepareCall(String sql) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -689,6 +771,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareCall(java.lang.String, int, int)
    */
 
+  @Override
   public CallableStatement prepareCall(String sql, int resultSetType,
       int resultSetConcurrency) throws SQLException {
     // TODO Auto-generated method stub
@@ -701,6 +784,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareCall(java.lang.String, int, int, int)
    */
 
+  @Override
   public CallableStatement prepareCall(String sql, int resultSetType,
       int resultSetConcurrency, int resultSetHoldability) throws SQLException {
     // TODO Auto-generated method stub
@@ -713,6 +797,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareStatement(java.lang.String)
    */
 
+  @Override
   public PreparedStatement prepareStatement(String sql) throws SQLException {
     return new HivePreparedStatement(this, client, sessHandle, sql);
   }
@@ -723,6 +808,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareStatement(java.lang.String, int)
    */
 
+  @Override
   public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys)
       throws SQLException {
     return new HivePreparedStatement(this, client, sessHandle, sql);
@@ -734,6 +820,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareStatement(java.lang.String, int[])
    */
 
+  @Override
   public PreparedStatement prepareStatement(String sql, int[] columnIndexes)
       throws SQLException {
     // TODO Auto-generated method stub
@@ -747,6 +834,7 @@ public class HiveConnection implements j
    * java.lang.String[])
    */
 
+  @Override
   public PreparedStatement prepareStatement(String sql, String[] columnNames)
       throws SQLException {
     // TODO Auto-generated method stub
@@ -759,6 +847,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareStatement(java.lang.String, int, int)
    */
 
+  @Override
   public PreparedStatement prepareStatement(String sql, int resultSetType,
       int resultSetConcurrency) throws SQLException {
     return new HivePreparedStatement(this, client, sessHandle, sql);
@@ -770,6 +859,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#prepareStatement(java.lang.String, int, int, int)
    */
 
+  @Override
   public PreparedStatement prepareStatement(String sql, int resultSetType,
       int resultSetConcurrency, int resultSetHoldability) throws SQLException {
     // TODO Auto-generated method stub
@@ -782,6 +872,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#releaseSavepoint(java.sql.Savepoint)
    */
 
+  @Override
   public void releaseSavepoint(Savepoint savepoint) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -793,6 +884,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#rollback()
    */
 
+  @Override
   public void rollback() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -804,6 +896,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#rollback(java.sql.Savepoint)
    */
 
+  @Override
   public void rollback(Savepoint savepoint) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -815,6 +908,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setAutoCommit(boolean)
    */
 
+  @Override
   public void setAutoCommit(boolean autoCommit) throws SQLException {
     if (autoCommit) {
       throw new SQLException("enabling autocommit is not supported");
@@ -827,6 +921,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setCatalog(java.lang.String)
    */
 
+  @Override
   public void setCatalog(String catalog) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -838,6 +933,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setClientInfo(java.util.Properties)
    */
 
+  @Override
   public void setClientInfo(Properties properties)
       throws SQLClientInfoException {
     // TODO Auto-generated method stub
@@ -850,6 +946,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setClientInfo(java.lang.String, java.lang.String)
    */
 
+  @Override
   public void setClientInfo(String name, String value)
       throws SQLClientInfoException {
     // TODO Auto-generated method stub
@@ -862,6 +959,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setHoldability(int)
    */
 
+  @Override
   public void setHoldability(int holdability) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -878,6 +976,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setReadOnly(boolean)
    */
 
+  @Override
   public void setReadOnly(boolean readOnly) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -889,6 +988,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setSavepoint()
    */
 
+  @Override
   public Savepoint setSavepoint() throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -900,6 +1000,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setSavepoint(java.lang.String)
    */
 
+  @Override
   public Savepoint setSavepoint(String name) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -916,6 +1017,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setTransactionIsolation(int)
    */
 
+  @Override
   public void setTransactionIsolation(int level) throws SQLException {
     // TODO: throw an exception?
   }
@@ -926,6 +1028,7 @@ public class HiveConnection implements j
    * @see java.sql.Connection#setTypeMap(java.util.Map)
    */
 
+  @Override
   public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -937,6 +1040,7 @@ public class HiveConnection implements j
    * @see java.sql.Wrapper#isWrapperFor(java.lang.Class)
    */
 
+  @Override
   public boolean isWrapperFor(Class<?> iface) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");
@@ -948,6 +1052,7 @@ public class HiveConnection implements j
    * @see java.sql.Wrapper#unwrap(java.lang.Class)
    */
 
+  @Override
   public <T> T unwrap(Class<T> iface) throws SQLException {
     // TODO Auto-generated method stub
     throw new SQLException("Method not supported");

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=1568236&r1=1568235&r2=1568236&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 Fri Feb 14 10:53:31 2014
@@ -258,11 +258,11 @@ public class HiveDatabaseMetaData implem
   }
 
   public int getDatabaseMajorVersion() throws SQLException {
-    return Utils.getVersionPart(getDatabaseProductVersion(), 1);
+    return Utils.getVersionPart(getDatabaseProductVersion(), 0);
   }
 
   public int getDatabaseMinorVersion() throws SQLException {
-    return Utils.getVersionPart(getDatabaseProductVersion(), 2);
+    return Utils.getVersionPart(getDatabaseProductVersion(), 1);
   }
 
   public String getDatabaseProductName() throws SQLException {

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=1568236&r1=1568235&r2=1568236&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 Fri Feb 14 10:53:31 2014
@@ -236,7 +236,6 @@ public class HiveQueryResultSet extends 
    * Retrieve schema from the server
    */
   private void retrieveSchema() throws SQLException {
-    System.err.println("[HiveQueryResultSet/next] 0");
     try {
       TGetResultSetMetadataReq metadataReq = new TGetResultSetMetadataReq(stmtHandle);
       // TODO need session handle

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=1568236&r1=1568235&r2=1568236&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 Fri Feb 14 10:53:31 2014
@@ -234,8 +234,8 @@ public class Utils {
    * @param fullVersion
    *          version string.
    * @param tokenPosition
-   *          position of version string to get starting at 1. eg, for a X.x.xxx
-   *          string, 1 will return the major version, 2 will return minor
+   *          position of version string to get starting at 0. eg, for a X.x.xxx
+   *          string, 0 will return the major version, 1 will return minor
    *          version.
    * @return version part, or -1 if version string was malformed.
    */

Modified: hive/branches/tez/metastore/if/hive_metastore.thrift
URL: http://svn.apache.org/viewvc/hive/branches/tez/metastore/if/hive_metastore.thrift?rev=1568236&r1=1568235&r2=1568236&view=diff
==============================================================================
--- hive/branches/tez/metastore/if/hive_metastore.thrift (original)
+++ hive/branches/tez/metastore/if/hive_metastore.thrift Fri Feb 14 10:53:31 2014
@@ -107,6 +107,11 @@ struct Role {
   1: string roleName,
   2: i32 createTime,
   3: string ownerName,
+  4: optional string principalName,
+  5: optional string principalType,
+  6: optional bool grantOption,
+  7: optional i32 grantTime,
+  8: optional string grantor
 }
 
 // namespace for tables
@@ -321,6 +326,34 @@ struct AddPartitionsRequest {
   5: optional bool needResult=true
 }
 
+// Return type for drop_partitions_req
+struct DropPartitionsResult {
+  1: optional list<Partition> partitions,
+}
+
+struct DropPartitionsExpr {
+  1: required binary expr;
+  2: optional i32 partArchiveLevel;
+}
+
+union RequestPartsSpec {
+  1: list<string> names;
+  2: list<DropPartitionsExpr> exprs;
+}
+
+// Request type for drop_partitions_req
+// TODO: we might want to add "bestEffort" flag; where a subset can fail
+struct DropPartitionsRequest {
+  1: required string dbName,
+  2: required string tblName,
+  3: required RequestPartsSpec parts,
+  4: optional bool deleteData,
+  5: optional bool ifExists=true, // currently verified on client
+  6: optional bool ignoreProtection,
+  7: optional EnvironmentContext environmentContext,
+  8: optional bool needResult=true
+}
+
 exception MetaException {
   1: string message
 }
@@ -500,6 +533,9 @@ service ThriftHiveMetastore extends fb30
   bool drop_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name,
       3:string part_name, 4:bool deleteData, 5:EnvironmentContext environment_context)
                        throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  DropPartitionsResult drop_partitions_req(1: DropPartitionsRequest req)
+                       throws(1:NoSuchObjectException o1, 2:MetaException o2)
+
   Partition get_partition(1:string db_name, 2:string tbl_name, 3:list<string> part_vals)
                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
   Partition exchange_partition(1:map<string, string> partitionSpecs, 2:string source_db,