You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2012/11/23 05:59:10 UTC

svn commit: r1412751 - in /hive/branches/branch-0.10/ql/src: java/org/apache/hadoop/hive/ql/metadata/Hive.java test/org/apache/hadoop/hive/ql/metadata/TestHive.java

Author: hashutosh
Date: Fri Nov 23 04:59:10 2012
New Revision: 1412751

URL: http://svn.apache.org/viewvc?rev=1412751&view=rev
Log:
HIVE-3722 : Create index fails on CLI using remote metastore (Kevin Wilfong via Ashutosh Chauhan)

Modified:
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java

Modified: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1412751&r1=1412750&r2=1412751&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Fri Nov 23 04:59:10 2012
@@ -56,7 +56,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
@@ -72,6 +71,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.index.HiveIndexHandler;
 import org.apache.hadoop.hive.ql.session.CreateTableAutomaticGrant;
@@ -739,7 +739,9 @@ public class Hive {
 
       Index indexDesc = new Index(indexName, indexHandlerClass, dbName, tableName, time, time, indexTblName,
           storageDescriptor, params, deferredRebuild);
-      indexDesc.getParameters().put("comment", indexComment);
+      if (indexComment != null) {
+        indexDesc.getParameters().put("comment", indexComment);
+      }
 
       if (idxProps != null)
       {

Modified: hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=1412751&r1=1412750&r2=1412751&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Fri Nov 23 04:59:10 2012
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import junit.framework.TestCase;
 
@@ -33,6 +34,10 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.index.HiveIndex;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -49,8 +54,8 @@ import org.apache.thrift.protocol.TBinar
  *
  */
 public class TestHive extends TestCase {
-  private Hive hm;
-  private HiveConf hiveConf;
+  protected Hive hm;
+  protected HiveConf hiveConf;
 
   @Override
   protected void setUp() throws Exception {
@@ -61,7 +66,7 @@ public class TestHive extends TestCase {
     } catch (Exception e) {
       System.err.println(StringUtils.stringifyException(e));
       System.err
-          .println("Unable to initialize Hive Metastore using configruation: \n "
+          .println("Unable to initialize Hive Metastore using configuration: \n "
           + hiveConf);
       throw e;
     }
@@ -145,33 +150,7 @@ public class TestHive extends TestCase {
       }
 
       // get table
-      Table ft = null;
-      Warehouse wh = new Warehouse(hiveConf);
-      try {
-        ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
-        ft.checkValidity();
-        assertEquals("Table names didn't match for table: " + tableName, tbl
-            .getTableName(), ft.getTableName());
-        assertEquals("Table owners didn't match for table: " + tableName, tbl
-            .getOwner(), ft.getOwner());
-        assertEquals("Table retention didn't match for table: " + tableName,
-            tbl.getRetention(), ft.getRetention());
-        assertEquals("Data location is not set correctly",
-            wh.getTablePath(hm.getDatabase(DEFAULT_DATABASE_NAME), tableName).toString(),
-            ft.getDataLocation().toString());
-        // now that URI is set correctly, set the original table's uri and then
-        // compare the two tables
-        tbl.setDataLocation(ft.getDataLocation());
-        assertTrue("Tables doesn't match: " + tableName, ft.getTTable().equals(
-            tbl.getTTable()));
-        assertEquals("Serde is not set correctly", tbl.getDeserializer()
-            .getClass().getName(), ft.getDeserializer().getClass().getName());
-        assertEquals("SerializationLib is not set correctly", tbl
-            .getSerializationLib(), LazySimpleSerDe.class.getName());
-      } catch (HiveException e) {
-        e.printStackTrace();
-        assertTrue("Unable to fetch table correctly: " + tableName, false);
-      }
+      validateTable(tbl, tableName);
 
       try {
         hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, true,
@@ -218,34 +197,7 @@ public class TestHive extends TestCase {
         assertTrue("Unable to create table: " + tableName, false);
       }
       // get table
-      Warehouse wh = new Warehouse(hiveConf);
-      Table ft = null;
-      try {
-        ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
-        assertNotNull("Unable to fetch table", ft);
-        ft.checkValidity();
-        assertEquals("Table names didn't match for table: " + tableName, tbl
-            .getTableName(), ft.getTableName());
-        assertEquals("Table owners didn't match for table: " + tableName, tbl
-            .getOwner(), ft.getOwner());
-        assertEquals("Table retention didn't match for table: " + tableName,
-            tbl.getRetention(), ft.getRetention());
-        assertEquals("Data location is not set correctly",
-            wh.getTablePath(hm.getDatabase(DEFAULT_DATABASE_NAME), tableName).toString(),
-            ft.getDataLocation().toString());
-        // now that URI is set correctly, set the original table's uri and then
-        // compare the two tables
-        tbl.setDataLocation(ft.getDataLocation());
-        assertTrue("Tables  doesn't match: " + tableName, ft.getTTable()
-            .equals(tbl.getTTable()));
-        assertEquals("SerializationLib is not set correctly", tbl
-            .getSerializationLib(), ThriftDeserializer.class.getName());
-        assertEquals("Serde is not set correctly", tbl.getDeserializer()
-            .getClass().getName(), ft.getDeserializer().getClass().getName());
-      } catch (HiveException e) {
-        System.err.println(StringUtils.stringifyException(e));
-        assertTrue("Unable to fetch table correctly: " + tableName, false);
-      }
+      validateTable(tbl, tableName);
       hm.dropTable(DEFAULT_DATABASE_NAME, tableName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));
@@ -254,6 +206,47 @@ public class TestHive extends TestCase {
     }
   }
 
+  /**
+   * Gets a table from the metastore and compares it to the original Table
+   *
+   * @param tbl
+   * @param tableName
+   * @throws MetaException
+   */
+  private void validateTable(Table tbl, String tableName) throws MetaException {
+    Warehouse wh = new Warehouse(hiveConf);
+    Table ft = null;
+    try {
+      ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+      assertNotNull("Unable to fetch table", ft);
+      ft.checkValidity();
+      assertEquals("Table names didn't match for table: " + tableName, tbl
+          .getTableName(), ft.getTableName());
+      assertEquals("Table owners didn't match for table: " + tableName, tbl
+          .getOwner(), ft.getOwner());
+      assertEquals("Table retention didn't match for table: " + tableName,
+          tbl.getRetention(), ft.getRetention());
+      assertEquals("Data location is not set correctly",
+          wh.getTablePath(hm.getDatabase(DEFAULT_DATABASE_NAME), tableName).toString(),
+          ft.getDataLocation().toString());
+      // now that URI and times are set correctly, set the original table's uri and times
+      // and then compare the two tables
+      tbl.setDataLocation(ft.getDataLocation());
+      tbl.setCreateTime(ft.getTTable().getCreateTime());
+      tbl.getParameters().put(hive_metastoreConstants.DDL_TIME,
+          ft.getParameters().get(hive_metastoreConstants.DDL_TIME));
+      assertTrue("Tables  doesn't match: " + tableName, ft.getTTable()
+          .equals(tbl.getTTable()));
+      assertEquals("SerializationLib is not set correctly", tbl
+          .getSerializationLib(), ft.getSerializationLib());
+      assertEquals("Serde is not set correctly", tbl.getDeserializer()
+          .getClass().getName(), ft.getDeserializer().getClass().getName());
+    } catch (HiveException e) {
+      System.err.println(StringUtils.stringifyException(e));
+      assertTrue("Unable to fetch table correctly: " + tableName, false);
+    }
+  }
+
   private static Table createTestTable(String dbName, String tableName) throws HiveException {
     Table tbl = new Table(dbName, tableName);
     tbl.setInputFormatClass(SequenceFileInputFormat.class.getName());
@@ -375,6 +368,125 @@ public class TestHive extends TestCase {
     }
   }
 
+  /**
+   * Tests creating a simple index on a simple table.
+   *
+   * @throws Throwable
+   */
+  public void testIndex() throws Throwable {
+    try{
+      // create a simple table
+      String tableName = "table_for_testindex";
+      try {
+        hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+      } catch (HiveException e) {
+        e.printStackTrace();
+        assertTrue("Unable to drop table", false);
+      }
+
+      Table tbl = new Table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+      List<FieldSchema> fields = tbl.getCols();
+
+      fields.add(new FieldSchema("col1", serdeConstants.INT_TYPE_NAME, "int -- first column"));
+      fields.add(new FieldSchema("col2", serdeConstants.STRING_TYPE_NAME,
+          "string -- second column"));
+      fields.add(new FieldSchema("col3", serdeConstants.DOUBLE_TYPE_NAME,
+          "double -- thrift column"));
+      tbl.setFields(fields);
+
+      tbl.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
+      tbl.setInputFormatClass(SequenceFileInputFormat.class);
+
+      // create table
+      try {
+        hm.createTable(tbl);
+      } catch (HiveException e) {
+        e.printStackTrace();
+        assertTrue("Unable to create table: " + tableName, false);
+      }
+
+      // Create a simple index
+      String indexName = "index_on_table_for_testindex";
+      String indexHandlerClass = HiveIndex.IndexType.COMPACT_SUMMARY_TABLE.getHandlerClsName();
+      List<String> indexedCols = new ArrayList<String>();
+      indexedCols.add("col1");
+      String indexTableName = "index_on_table_for_testindex_table";
+      boolean deferredRebuild = true;
+      String inputFormat = SequenceFileInputFormat.class.getName();
+      String outputFormat = SequenceFileOutputFormat.class.getName();
+      String serde = null;
+      String storageHandler = null;
+      String location = null;
+      String collItemDelim = null;
+      String fieldDelim = null;
+      String fieldEscape = null;
+      String lineDelim = null;
+      String mapKeyDelim = null;
+      String indexComment = null;
+      Map<String, String> indexProps = null;
+      Map<String, String> tableProps = null;
+      Map<String, String> serdeProps = new HashMap<String, String>();
+      hm.createIndex(tableName, indexName, indexHandlerClass, indexedCols, indexTableName,
+          deferredRebuild, inputFormat, outputFormat, serde, storageHandler, location,
+          indexProps, tableProps, serdeProps, collItemDelim, fieldDelim, fieldEscape, lineDelim,
+          mapKeyDelim, indexComment);
+
+      // Retrieve and validate the index
+      Index index = null;
+      try {
+        index = hm.getIndex(tableName, indexName);
+        assertNotNull("Unable to fetch index", index);
+        index.validate();
+        assertEquals("Index names don't match for index: " + indexName, indexName,
+            index.getIndexName());
+        assertEquals("Table names don't match for index: " + indexName, tableName,
+            index.getOrigTableName());
+        assertEquals("Index table names didn't match for index: " + indexName, indexTableName,
+            index.getIndexTableName());
+        assertEquals("Index handler classes didn't match for index: " + indexName,
+            indexHandlerClass, index.getIndexHandlerClass());
+        assertEquals("Deferred rebuild didn't match for index: " + indexName, deferredRebuild,
+            index.isDeferredRebuild());
+
+      } catch (HiveException e) {
+        System.err.println(StringUtils.stringifyException(e));
+        assertTrue("Unable to fetch index correctly: " + indexName, false);
+      }
+
+      // Drop index
+      try {
+        hm.dropIndex(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, indexName, true);
+      } catch (HiveException e) {
+        System.err.println(StringUtils.stringifyException(e));
+        assertTrue("Unable to drop index: " + indexName, false);
+      }
+
+      boolean dropIndexException = false;
+      try {
+        hm.getIndex(tableName, indexName);
+      } catch (HiveException e) {
+        // Expected since it was just dropped
+        dropIndexException = true;
+      }
+
+      assertTrue("Unable to drop index: " + indexName, dropIndexException);
+
+      // Drop table
+      try {
+        hm.dropTable(tableName);
+        Table droppedTable = hm.getTable(tableName, false);
+        assertNull("Unable to drop table " + tableName, droppedTable);
+      } catch (HiveException e) {
+        System.err.println(StringUtils.stringifyException(e));
+        assertTrue("Unable to drop table: " + tableName, false);
+      }
+    } catch (Throwable e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testIndex failed");
+      throw e;
+    }
+  }
+
   public void testHiveRefreshDatabase() throws Throwable{
     String testDatabaseName = "test_database";
     Database testDatabase = new Database();