You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2014/10/15 03:08:17 UTC

svn commit: r1631925 [1/2] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/ itests/util/src/main/java/org/apache/hadoop/hive/ql/ metastore/src/java/org/apache/hadoop/hive/me...

Author: navis
Date: Wed Oct 15 01:08:16 2014
New Revision: 1631925

URL: http://svn.apache.org/r1631925
Log:
HIVE-3781 : Index related events should be delivered to metastore event listener (Navis reviewed by Thejas M Nair)

Added:
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
    hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyPreListener.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropIndexDesc.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
    hive/trunk/ql/src/test/queries/clientpositive/drop_index.q
    hive/trunk/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/alter_concatenate_indexed_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auth.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_file_format.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables_compact.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_multiple.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_self_join.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_update.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_compression.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_serde.q.out
    hive/trunk/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx.q.out
    hive/trunk/ql/src/test/results/clientpositive/show_indexes_edge_cases.q.out
    hive/trunk/ql/src/test/results/clientpositive/show_indexes_syntax.q.out
    hive/trunk/ql/src/test/results/clientpositive/union_view.q.out

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Wed Oct 15 01:08:16 2014
@@ -312,7 +312,7 @@ public class HiveConf extends Configurat
         "When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode."),
 
     DROPIGNORESNONEXISTENT("hive.exec.drop.ignorenonexistent", true,
-        "Do not report an error if DROP TABLE/VIEW specifies a non-existent table/view"),
+        "Do not report an error if DROP TABLE/VIEW/Index specifies a non-existent table/view/index"),
 
     HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"),
 

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java Wed Oct 15 01:08:16 2014
@@ -29,26 +29,33 @@ import junit.framework.TestCase;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
@@ -183,12 +190,35 @@ public class TestMetaStoreEventListener 
     assertEquals(expectedDb, actualDb);
   }
 
+  private void validateIndex(Index expectedIndex, Index actualIndex) {
+    assertEquals(expectedIndex.getDbName(), actualIndex.getDbName());
+    assertEquals(expectedIndex.getIndexName(), actualIndex.getIndexName());
+    assertEquals(expectedIndex.getIndexHandlerClass(), actualIndex.getIndexHandlerClass());
+    assertEquals(expectedIndex.getOrigTableName(), actualIndex.getOrigTableName());
+    assertEquals(expectedIndex.getIndexTableName(), actualIndex.getIndexTableName());
+    assertEquals(expectedIndex.getSd().getLocation(), actualIndex.getSd().getLocation());
+  }
+
+  private void validateAddIndex(Index expectedIndex, Index actualIndex) {
+    validateIndex(expectedIndex, actualIndex);
+  }
+
+  private void validateAlterIndex(Index expectedOldIndex, Index actualOldIndex,
+      Index expectedNewIndex, Index actualNewIndex) {
+    validateIndex(expectedOldIndex, actualOldIndex);
+    validateIndex(expectedNewIndex, actualNewIndex);
+  }
+
+  private void validateDropIndex(Index expectedIndex, Index actualIndex) {
+    validateIndex(expectedIndex, actualIndex);
+  }
+
   public void testListener() throws Exception {
     int listSize = 0;
 
     List<ListenerEvent> notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
     List<PreEventContext> preNotifyList = DummyPreListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
     assertEquals(preNotifyList.size(), listSize);
 
     driver.run("create database " + dbName);
@@ -216,6 +246,48 @@ public class TestMetaStoreEventListener 
     assert tblEvent.getStatus();
     validateCreateTable(tbl, tblEvent.getTable());
 
+    driver.run("create index tmptbl_i on table tmptbl(a) as 'compact' " +
+        "WITH DEFERRED REBUILD IDXPROPERTIES ('prop1'='val1', 'prop2'='val2')");
+    listSize += 2;  // creates index table internally
+    assertEquals(notifyList.size(), listSize);
+
+    AddIndexEvent addIndexEvent = (AddIndexEvent)notifyList.get(listSize - 1);
+    assert addIndexEvent.getStatus();
+    PreAddIndexEvent preAddIndexEvent = (PreAddIndexEvent)(preNotifyList.get(preNotifyList.size() - 3));
+
+    Index oldIndex = msc.getIndex(dbName, "tmptbl", "tmptbl_i");
+
+    validateAddIndex(oldIndex, addIndexEvent.getIndex());
+
+    validateAddIndex(oldIndex, preAddIndexEvent.getIndex());
+
+    driver.run("alter index tmptbl_i on tmptbl set IDXPROPERTIES " +
+        "('prop1'='val1_new', 'prop3'='val3')");
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    Index newIndex = msc.getIndex(dbName, "tmptbl", "tmptbl_i");
+
+    AlterIndexEvent alterIndexEvent = (AlterIndexEvent) notifyList.get(listSize - 1);
+    assert alterIndexEvent.getStatus();
+    validateAlterIndex(oldIndex, alterIndexEvent.getOldIndex(),
+        newIndex, alterIndexEvent.getNewIndex());
+
+    PreAlterIndexEvent preAlterIndexEvent = (PreAlterIndexEvent) (preNotifyList.get(preNotifyList.size() - 1));
+    validateAlterIndex(oldIndex, preAlterIndexEvent.getOldIndex(),
+        newIndex, preAlterIndexEvent.getNewIndex());
+
+    driver.run("drop index tmptbl_i on tmptbl");
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    DropIndexEvent dropIndexEvent = (DropIndexEvent) notifyList.get(listSize - 1);
+    assert dropIndexEvent.getStatus();
+    validateDropIndex(newIndex, dropIndexEvent.getIndex());
+
+    PreDropIndexEvent preDropIndexEvent = (PreDropIndexEvent) (preNotifyList.get(preNotifyList.size() - 1));
+    validateDropIndex(newIndex, preDropIndexEvent.getIndex());
+
     driver.run("alter table tmptbl add partition (b='2011')");
     listSize++;
     assertEquals(notifyList.size(), listSize);

Modified: hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java Wed Oct 15 01:08:16 2014
@@ -559,7 +559,7 @@ public class QTestUtil {
          List<Index> indexes = db.getIndexes(dbName, tblName, (short)-1);
           if (indexes != null && indexes.size() > 0) {
             for (Index index : indexes) {
-              db.dropIndex(dbName, tblName, index.getIndexName(), true);
+              db.dropIndex(dbName, tblName, index.getIndexName(), true, true);
             }
           }
         }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Wed Oct 15 01:08:16 2014
@@ -148,6 +148,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
@@ -155,17 +157,21 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreAuthorizationCallEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
@@ -1429,17 +1435,15 @@ public class HiveMetaStore extends Thrif
       return (ms.getTable(dbname, name) != null);
     }
 
-    private void drop_table_core(final RawStore ms, final String dbname, final String name,
-        final boolean deleteData, final EnvironmentContext envContext)
-        throws NoSuchObjectException, MetaException, IOException,
-        InvalidObjectException, InvalidInputException {
+    private boolean drop_table_core(final RawStore ms, final String dbname, final String name,
+        final boolean deleteData, final EnvironmentContext envContext,
+        final String indexName) throws NoSuchObjectException,
+        MetaException, IOException, InvalidObjectException, InvalidInputException {
       boolean success = false;
       boolean isExternal = false;
       Path tblPath = null;
       List<Path> partPaths = null;
       Table tbl = null;
-      isExternal = false;
-      boolean isIndexTable = false;
       try {
         ms.openTransaction();
         // drop any partitions
@@ -1453,8 +1457,8 @@ public class HiveMetaStore extends Thrif
 
         firePreEvent(new PreDropTableEvent(tbl, deleteData, this));
 
-        isIndexTable = isIndexTable(tbl);
-        if (isIndexTable) {
+        boolean isIndexTable = isIndexTable(tbl);
+        if (indexName == null && isIndexTable) {
           throw new RuntimeException(
               "The table " + name + " is an index table. Please do drop index instead.");
         }
@@ -1476,7 +1480,8 @@ public class HiveMetaStore extends Thrif
         if (tbl.getSd().getLocation() != null) {
           tblPath = new Path(tbl.getSd().getLocation());
           if (!wh.isWritable(tblPath.getParent())) {
-            throw new MetaException("Table metadata not deleted since " +
+            String target = indexName == null ? "Table" : "Index table";
+            throw new MetaException(target + " metadata not deleted since " +
                 tblPath.getParent() + " is not writable by " +
                 hiveConf.getUser());
           }
@@ -1487,17 +1492,17 @@ public class HiveMetaStore extends Thrif
             tbl.getPartitionKeys(), deleteData && !isExternal);
 
         if (!ms.dropTable(dbname, name)) {
-          throw new MetaException("Unable to drop table");
+          String tableName = dbname + "." + name;
+          throw new MetaException(indexName == null ? "Unable to drop table " + tableName:
+              "Unable to drop index table " + tableName + " for index " + indexName);
         }
         success = ms.commitTransaction();
       } finally {
         if (!success) {
           ms.rollbackTransaction();
         } else if (deleteData && !isExternal) {
-          boolean ifPurge = false;
-          if (envContext != null){
-            ifPurge = Boolean.parseBoolean(envContext.getProperties().get("ifPurge"));
-          }
+          boolean ifPurge = envContext != null &&
+              Boolean.parseBoolean(envContext.getProperties().get("ifPurge"));
           // Delete the data in the partitions which have other locations
           deletePartitionData(partPaths, ifPurge);
           // Delete the data in the table
@@ -1510,6 +1515,7 @@ public class HiveMetaStore extends Thrif
           listener.onDropTable(dropTableEvent);
         }
       }
+      return success;
     }
 
     /**
@@ -1653,8 +1659,7 @@ public class HiveMetaStore extends Thrif
       boolean success = false;
       Exception ex = null;
       try {
-        drop_table_core(getMS(), dbname, name, deleteData, envContext);
-        success = true;
+        success = drop_table_core(getMS(), dbname, name, deleteData, envContext, null);
       } catch (IOException e) {
         ex = e;
         throw new MetaException(e.getMessage());
@@ -3209,7 +3214,12 @@ public class HiveMetaStore extends Thrif
 
       boolean success = false;
       Exception ex = null;
+      Index oldIndex = null;
       try {
+        oldIndex = get_index_by_name(dbname, base_table_name, index_name);
+
+        firePreEvent(new PreAlterIndexEvent(oldIndex, newIndex, this));
+
         getMS().alterIndex(dbname, base_table_name, index_name, newIndex);
         success = true;
       } catch (InvalidObjectException e) {
@@ -3226,6 +3236,10 @@ public class HiveMetaStore extends Thrif
         }
       } finally {
         endFunction("alter_index", success, ex, base_table_name);
+        for (MetaStoreEventListener listener : listeners) {
+          AlterIndexEvent alterIndexEvent = new AlterIndexEvent(oldIndex, newIndex, success, this);
+          listener.onAlterIndex(alterIndexEvent);
+        }
       }
       return;
     }
@@ -3771,6 +3785,8 @@ public class HiveMetaStore extends Thrif
 
       try {
         ms.openTransaction();
+        firePreEvent(new PreAddIndexEvent(index, this));
+
         Index old_index = null;
         try {
           old_index = get_index_by_name(index.getDbName(), index
@@ -3818,6 +3834,10 @@ public class HiveMetaStore extends Thrif
           }
           ms.rollbackTransaction();
         }
+        for (MetaStoreEventListener listener : listeners) {
+          AddIndexEvent addIndexEvent = new AddIndexEvent(index, success, this);
+          listener.onAddIndex(addIndexEvent);
+        }
       }
     }
 
@@ -3852,16 +3872,17 @@ public class HiveMetaStore extends Thrif
         MetaException, TException, IOException, InvalidObjectException, InvalidInputException {
 
       boolean success = false;
+      Index index = null;
       Path tblPath = null;
       List<Path> partPaths = null;
       try {
         ms.openTransaction();
 
         // drop the underlying index table
-        Index index = get_index_by_name(dbName, tblName, indexName);
-        if (index == null) {
-          throw new NoSuchObjectException(indexName + " doesn't exist");
-        }
+        index = get_index_by_name(dbName, tblName, indexName);  // throws exception if not exists
+
+        firePreEvent(new PreDropIndexEvent(index, this));
+
         ms.dropIndex(dbName, tblName, indexName);
 
         String idxTblName = index.getIndexTableName();
@@ -3882,26 +3903,29 @@ public class HiveMetaStore extends Thrif
           }
 
           // Drop the partitions and get a list of partition locations which need to be deleted
-          partPaths = dropPartitionsAndGetLocations(ms, dbName, idxTblName, tblPath,
+          partPaths = dropPartitionsAndGetLocations(ms, qualified[0], qualified[1], tblPath,
               tbl.getPartitionKeys(), deleteData);
 
-          if (!ms.dropTable(dbName, idxTblName)) {
+          if (!ms.dropTable(qualified[0], qualified[1])) {
             throw new MetaException("Unable to drop underlying data table "
-                + idxTblName + " for index " + idxTblName);
+                + qualified[0] + "." + qualified[1] + " for index " + indexName);
           }
         }
         success = ms.commitTransaction();
       } finally {
         if (!success) {
           ms.rollbackTransaction();
-          return false;
         } else if (deleteData && tblPath != null) {
           deletePartitionData(partPaths);
           deleteTableData(tblPath);
           // ok even if the data is not deleted
         }
+        for (MetaStoreEventListener listener : listeners) {
+          DropIndexEvent dropIndexEvent = new DropIndexEvent(index, success, this);
+          listener.onDropIndex(dropIndexEvent);
+        }
       }
-      return true;
+      return success;
     }
 
     @Override
@@ -3920,7 +3944,7 @@ public class HiveMetaStore extends Thrif
         ex = e;
         rethrowException(e);
       } finally {
-        endFunction("drop_index_by_name", ret != null, ex, tblName);
+        endFunction("get_index_by_name", ret != null, ex, tblName);
       }
       return ret;
     }
@@ -5746,7 +5770,7 @@ public class HiveMetaStore extends Thrif
     try {
       isMetaStoreRemote = true;
       // Server will create new threads up to max as necessary. After an idle
-      // period, it will destory threads to keep the number of threads in the
+      // period, it will destroy threads to keep the number of threads in the
       // pool to min.
       int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS);
       int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java Wed Oct 15 01:08:16 2014
@@ -21,6 +21,8 @@ package org.apache.hadoop.hive.metastore
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
@@ -28,6 +30,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
@@ -117,7 +120,27 @@ public abstract class MetaStoreEventList
    * @throws MetaException
    */
   public void onLoadPartitionDone(LoadPartitionDoneEvent partSetDoneEvent) throws MetaException {
+  }
+
+  /**
+   * @param indexEvent index event
+   * @throws MetaException
+   */
+  public void onAddIndex(AddIndexEvent indexEvent) throws MetaException {
+  }
+
+  /**
+   * @param indexEvent index event
+   * @throws MetaException
+   */
+  public void onDropIndex(DropIndexEvent indexEvent) throws MetaException {
+  }
 
+  /**
+   * @param indexEvent index event
+   * @throws MetaException
+   */
+  public void onAlterIndex(AlterIndexEvent indexEvent) throws MetaException {
   }
 
   @Override

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java Wed Oct 15 01:08:16 2014
@@ -64,7 +64,6 @@ import org.apache.hadoop.hive.common.cla
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
@@ -92,7 +91,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -197,7 +195,7 @@ public class ObjectStore implements RawS
   }
 
   /**
-   * Called whenever this object is instantiated using ReflectionUils, and also
+   * Called whenever this object is instantiated using ReflectionUtils, and also
    * on connection retries. In cases of connection retries, conf will usually
    * contain modified values.
    */
@@ -2665,7 +2663,7 @@ public class ObjectStore implements RawS
         throw new MetaException("table " + name + " doesn't exist");
       }
 
-      // For now only alter name, owner, paramters, cols, bucketcols are allowed
+      // For now only alter name, owner, parameters, cols, bucketcols are allowed
       oldt.setDatabase(newt.getDatabase());
       oldt.setTableName(newt.getTableName().toLowerCase());
       oldt.setParameters(newt.getParameters());
@@ -2708,7 +2706,7 @@ public class ObjectStore implements RawS
         throw new MetaException("index " + name + " doesn't exist");
       }
 
-      // For now only alter paramters are allowed
+      // For now only alter parameters are allowed
       oldi.setParameters(newi.getParameters());
 
       // commit the changes
@@ -2878,7 +2876,7 @@ public class ObjectStore implements RawS
     MColumnDescriptor mcd = msd.getCD();
     // Because there is a 1-N relationship between CDs and SDs,
     // we must set the SD's CD to null first before dropping the storage descriptor
-    // to satisfy foriegn key constraints.
+    // to satisfy foreign key constraints.
     msd.setCD(null);
     removeUnusedColumnDescriptor(mcd);
   }
@@ -3019,19 +3017,26 @@ public class ObjectStore implements RawS
   }
 
   private Index convertToIndex(MIndex mIndex) throws MetaException {
-    if(mIndex == null) {
+    if (mIndex == null) {
       return null;
     }
 
+    MTable origTable = mIndex.getOrigTable();
+    MTable indexTable = mIndex.getIndexTable();
+
+    String[] qualified = MetaStoreUtils.getQualifiedName(
+        origTable.getDatabase().getName(), indexTable.getTableName());
+    String indexTableName = qualified[0] + "." + qualified[1];
+
     return new Index(
     mIndex.getIndexName(),
     mIndex.getIndexHandlerClass(),
-    mIndex.getOrigTable().getDatabase().getName(),
-    mIndex.getOrigTable().getTableName(),
+    origTable.getDatabase().getName(),
+    origTable.getTableName(),
     mIndex.getCreateTime(),
     mIndex.getLastAccessTime(),
-    mIndex.getIndexTable().getTableName(),
-    this.convertToStorageDescriptor(mIndex.getSd()),
+    indexTableName,
+    convertToStorageDescriptor(mIndex.getSd()),
     mIndex.getParameters(),
     mIndex.getDeferredRebuild());
 

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java?rev=1631925&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java Wed Oct 15 01:08:16 2014
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public class AddIndexEvent extends ListenerEvent {
+
+  private final Index index;
+
+  public AddIndexEvent(Index index, boolean status, HiveMetaStore.HMSHandler handler) {
+    super(status, handler);
+    this.index = index;
+  }
+
+  public Index getIndex() {
+    return index;
+  }
+}

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java?rev=1631925&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java Wed Oct 15 01:08:16 2014
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public class AlterIndexEvent extends ListenerEvent {
+
+  private final Index newIndex;
+  private final Index oldIndex;
+
+  public AlterIndexEvent(Index oldIndex, Index newIndex, boolean status,
+      HiveMetaStore.HMSHandler handler) {
+    super(status, handler);
+    this.oldIndex = oldIndex;
+    this.newIndex = newIndex;
+  }
+
+  public Index getOldIndex() {
+    return oldIndex;
+  }
+
+  public Index getNewIndex() {
+    return newIndex;
+  }
+}

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java?rev=1631925&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java Wed Oct 15 01:08:16 2014
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public class DropIndexEvent extends ListenerEvent {
+
+  private final Index index;
+
+  public DropIndexEvent(Index index, boolean status, HiveMetaStore.HMSHandler handler) {
+    super(status, handler);
+    this.index = index;
+  }
+
+  public Index getIndex() {
+    return index;
+  }
+}

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java?rev=1631925&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java Wed Oct 15 01:08:16 2014
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public class PreAddIndexEvent extends PreEventContext {
+
+private final Index table;
+
+  public PreAddIndexEvent(Index table, HiveMetaStore.HMSHandler handler) {
+    super(PreEventType.ADD_INDEX, handler);
+    this.table = table;
+  }
+
+  public Index getIndex() {
+    return table;
+  }
+}
+

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java?rev=1631925&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java Wed Oct 15 01:08:16 2014
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public class PreAlterIndexEvent extends PreEventContext {
+
+  private final Index newIndex;
+  private final Index oldIndex;
+
+  public PreAlterIndexEvent(Index oldIndex, Index newIndex, HiveMetaStore.HMSHandler handler) {
+    super(PreEventType.ALTER_INDEX, handler);
+    this.oldIndex = oldIndex;
+    this.newIndex = newIndex;
+  }
+
+  public Index getOldIndex() {
+    return oldIndex;
+  }
+
+  public Index getNewIndex() {
+    return newIndex;
+  }
+}
+

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java?rev=1631925&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java Wed Oct 15 01:08:16 2014
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public class PreDropIndexEvent extends PreEventContext {
+
+  private final Index index;
+
+  public PreDropIndexEvent(Index index, HiveMetaStore.HMSHandler handler) {
+    super(PreEventType.DROP_INDEX, handler);
+    this.index = index;
+  }
+
+  public Index getIndex() {
+    return index;
+  }
+}

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java Wed Oct 15 01:08:16 2014
@@ -39,7 +39,10 @@ public abstract class PreEventContext {
     LOAD_PARTITION_DONE,
     AUTHORIZATION_API_CALL,
     READ_TABLE,
-    READ_DATABASE
+    READ_DATABASE,
+    ADD_INDEX,
+    ALTER_INDEX,
+    DROP_INDEX
   }
 
   private final PreEventType eventType;

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java Wed Oct 15 01:08:16 2014
@@ -23,13 +23,16 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
@@ -60,52 +63,70 @@ public class DummyListener extends MetaS
 
   @Override
   public void onConfigChange(ConfigChangeEvent configChange) {
-    notifyList.add(configChange);
+    addEvent(configChange);
   }
 
   @Override
   public void onAddPartition(AddPartitionEvent partition) throws MetaException {
-    notifyList.add(partition);
+    addEvent(partition);
   }
 
   @Override
   public void onCreateDatabase(CreateDatabaseEvent db) throws MetaException {
-    notifyList.add(db);
+    addEvent(db);
   }
 
   @Override
   public void onCreateTable(CreateTableEvent table) throws MetaException {
-    notifyList.add(table);
+    addEvent(table);
   }
 
   @Override
   public void onDropDatabase(DropDatabaseEvent db) throws MetaException {
-    notifyList.add(db);
+    addEvent(db);
   }
 
   @Override
   public void onDropPartition(DropPartitionEvent partition) throws MetaException {
-    notifyList.add(partition);
+    addEvent(partition);
   }
 
   @Override
   public void onDropTable(DropTableEvent table) throws MetaException {
-    notifyList.add(table);
+    addEvent(table);
   }
 
   @Override
   public void onAlterTable(AlterTableEvent event) throws MetaException {
-    notifyList.add(event);
+    addEvent(event);
   }
 
   @Override
   public void onAlterPartition(AlterPartitionEvent event) throws MetaException {
-    notifyList.add(event);
+    addEvent(event);
   }
 
   @Override
   public void onLoadPartitionDone(LoadPartitionDoneEvent partEvent) throws MetaException {
-    notifyList.add(partEvent);
+    addEvent(partEvent);
   }
 
+  @Override
+  public void onAddIndex(AddIndexEvent indexEvent) throws MetaException {
+    addEvent(indexEvent);
+  }
+
+  @Override
+  public void onDropIndex(DropIndexEvent indexEvent) throws MetaException {
+    addEvent(indexEvent);
+  }
+
+  @Override
+  public void onAlterIndex(AlterIndexEvent indexEvent) throws MetaException {
+    addEvent(indexEvent);
+  }
+
+  private void addEvent(ListenerEvent event) {
+    notifyList.add(event);
+  }
 }

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyPreListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyPreListener.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyPreListener.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyPreListener.java Wed Oct 15 01:08:16 2014
@@ -30,7 +30,7 @@ import org.apache.hadoop.hive.metastore.
  *
  * DummyPreListener.
  *
- * An implemenation of MetaStorePreEventListener which stores the Events it's seen in a list.
+ * An implementation of MetaStorePreEventListener which stores the Events it's seen in a list.
  */
 public class DummyPreListener extends MetaStorePreEventListener {
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Wed Oct 15 01:08:16 2014
@@ -156,7 +156,6 @@ import org.apache.hadoop.hive.serde.serd
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
-import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
 import org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -781,7 +780,7 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int dropIndex(Hive db, DropIndexDesc dropIdx) throws HiveException {
-    db.dropIndex(dropIdx.getTableName(), dropIdx.getIndexName(), true);
+    db.dropIndex(dropIdx.getTableName(), dropIdx.getIndexName(), dropIdx.isThrowException(), true);
     return 0;
   }
 
@@ -2204,7 +2203,7 @@ public class DDLTask extends Task<DDLWor
    *           Throws this exception if an unexpected error occurs.
    */
   private int showTables(Hive db, ShowTablesDesc showTbls) throws HiveException {
-    // get the tables for the desired pattenn - populate the output stream
+    // get the tables for the desired pattern - populate the output stream
     List<String> tbls = null;
     String dbName = showTbls.getDbName();
 
@@ -3915,7 +3914,7 @@ public class DDLTask extends Task<DDLWor
     tbl.setInputFormatClass(crtTbl.getInputFormat());
     tbl.setOutputFormatClass(crtTbl.getOutputFormat());
 
-    // only persist input/ouput format to metadata when it is explicitly specified.
+    // only persist input/output format to metadata when it is explicitly specified.
     // Otherwise, load lazily via StorageHandler at query time.
     if (crtTbl.getInputFormat() != null && !crtTbl.getInputFormat().isEmpty()) {
       tbl.getTTable().getSd().setInputFormat(tbl.getInputFormatClass().getName());

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Wed Oct 15 01:08:16 2014
@@ -313,7 +313,7 @@ public class Hive {
    * @param name
    * @param deleteData
    * @param ignoreUnknownDb if true, will ignore NoSuchObjectException
-   * @param cascade           if true, delete all tables on the DB if exists. Othewise, the query
+   * @param cascade         if true, delete all tables on the DB if exists. Otherwise, the query
    *                        will fail if table still exists.
    * @throws HiveException
    * @throws NoSuchObjectException
@@ -331,7 +331,7 @@ public class Hive {
 
 
   /**
-   * Creates a table metdata and the directory for the table data
+   * Creates a table metadata and the directory for the table data
    *
    * @param tableName
    *          name of the table
@@ -355,7 +355,7 @@ public class Hive {
   }
 
   /**
-   * Creates a table metdata and the directory for the table data
+   * Creates a table metadata and the directory for the table data
    *
    * @param tableName
    *          name of the table
@@ -885,16 +885,21 @@ public class Hive {
     }
   }
 
-  public boolean dropIndex(String baseTableName, String index_name, boolean deleteData) throws HiveException {
+  public boolean dropIndex(String baseTableName, String index_name,
+      boolean throwException, boolean deleteData) throws HiveException {
     String[] names = Utilities.getDbTableName(baseTableName);
-    return dropIndex(names[0], names[1], index_name, deleteData);
+    return dropIndex(names[0], names[1], index_name, throwException, deleteData);
   }
 
-  public boolean dropIndex(String db_name, String tbl_name, String index_name, boolean deleteData) throws HiveException {
+  public boolean dropIndex(String db_name, String tbl_name, String index_name,
+      boolean throwException, boolean deleteData) throws HiveException {
     try {
       return getMSC().dropIndex(db_name, tbl_name, index_name, deleteData);
     } catch (NoSuchObjectException e) {
-      throw new HiveException("Partition or table doesn't exist. " + e.getMessage(), e);
+      if (throwException) {
+        throw new HiveException("Index " + index_name + " doesn't exist. ", e);
+      }
+      return false;
     } catch (Exception e) {
       throw new HiveException(e.getMessage(), e);
     }
@@ -2861,10 +2866,6 @@ private void constructOneLBLocationMap(F
     }
   }
 
-  public static String[] getQualifiedNames(String qualifiedName) {
-    return qualifiedName.split("\\.");
-  }
-
   public void createFunction(Function func) throws HiveException {
     try {
       getMSC().createFunction(func);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Wed Oct 15 01:08:16 2014
@@ -53,6 +53,7 @@ import org.apache.hadoop.hive.metastore.
 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.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.ql.Driver;
@@ -214,24 +215,6 @@ public class DDLSemanticAnalyzer extends
     return typeName;
   }
 
-  static class TablePartition {
-    String tableName;
-    HashMap<String, String> partSpec = null;
-
-    public TablePartition() {
-    }
-
-    public TablePartition(ASTNode tblPart) throws SemanticException {
-      tableName = getDotName((getQualifiedTableName((ASTNode) tblPart.getChild(0))));
-      if (tblPart.getChildCount() > 1) {
-        ASTNode part = (ASTNode) tblPart.getChild(1);
-        if (part.getToken().getType() == HiveParser.TOK_PARTSPEC) {
-          this.partSpec = DDLSemanticAnalyzer.getPartSpec(part);
-        }
-      }
-    }
-  }
-
   public DDLSemanticAnalyzer(HiveConf conf) throws SemanticException {
     this(conf, createHiveDB(conf));
   }
@@ -1139,20 +1122,25 @@ public class DDLSemanticAnalyzer extends
     // configured not to ignore this
     boolean throwException =
         !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
-    if (throwException) {
-      try {
-        Index idx = db.getIndex(tableName, indexName);
-      } catch (HiveException e) {
+    Table tbl = getTable(tableName, false);
+    if (throwException && tbl == null) {
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tableName));
+    }
+    try {
+      Index idx = db.getIndex(tableName, indexName);
+    } catch (HiveException e) {
+      if (!(e.getCause() instanceof NoSuchObjectException)) {
+        throw new SemanticException(ErrorMsg.GENERIC_ERROR.getMsg("dropping index"), e);
+      }
+      if (throwException) {
         throw new SemanticException(ErrorMsg.INVALID_INDEX.getMsg(indexName));
       }
     }
-
-    Table tbl = getTable(tableName, false);
     if (tbl != null) {
-      inputs.add(new ReadEntity(getTable(tableName)));
+      inputs.add(new ReadEntity(tbl));
     }
 
-    DropIndexDesc dropIdxDesc = new DropIndexDesc(indexName, tableName);
+    DropIndexDesc dropIdxDesc = new DropIndexDesc(indexName, tableName, throwException);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         dropIdxDesc), conf));
   }
@@ -2809,7 +2797,7 @@ public class DDLSemanticAnalyzer extends
    * @param ast
    *          The parsed command tree.
    * @throws SemanticException
-   *           Parsin failed
+   *           Parsing failed
    */
   private void analyzeAlterTableTouch(String[] qualified, CommonTree ast)
       throws SemanticException {
@@ -2933,8 +2921,8 @@ public class DDLSemanticAnalyzer extends
    *
    * @param ast Tree to extract partitions from.
    * @param tab Table.
-   * @param result Map of partitions by prefix length. Most of the time prefix length will
-   *               be the same for all partition specs, so we can just OR the expressions.
+   * @return    Map of partitions by prefix length. Most of the time prefix length will
+   *            be the same for all partition specs, so we can just OR the expressions.
    */
   private Map<Integer, List<ExprNodeGenericFuncDesc>> getFullPartitionSpecs(
       CommonTree ast, Table tab, boolean canGroupExprs) throws SemanticException {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropIndexDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropIndexDesc.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropIndexDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropIndexDesc.java Wed Oct 15 01:08:16 2014
@@ -24,15 +24,17 @@ public class DropIndexDesc {
   private String indexName;
   
   private String tableName;
+
+  private boolean throwException;
   
   /**
    * @param indexName
    * @param tableName
    */
-  public DropIndexDesc(String indexName, String tableName) {
-    super();
+  public DropIndexDesc(String indexName, String tableName, boolean throwException) {
     this.indexName = indexName;
     this.tableName = tableName;
+    this.throwException = throwException;
   }
 
   /**
@@ -63,4 +65,11 @@ public class DropIndexDesc {
     this.tableName = tableName;
   }
 
+  public boolean isThrowException() {
+    return throwException;
+  }
+
+  public void setThrowException(boolean throwException) {
+    this.throwException = throwException;
+  }
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Wed Oct 15 01:08:16 2014
@@ -550,7 +550,7 @@ public class TestHive extends TestCase {
             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,
+        assertEquals("Index table names didn't match for index: " + indexName, qIndexTableName,
             index.getIndexTableName());
         assertEquals("Index handler classes didn't match for index: " + indexName,
             indexHandlerClass, index.getIndexHandlerClass());
@@ -564,7 +564,7 @@ public class TestHive extends TestCase {
 
       // Drop index
       try {
-        hm.dropIndex(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, indexName, true);
+        hm.dropIndex(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, indexName, false, true);
       } catch (HiveException e) {
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to drop index: " + indexName, false);

Modified: hive/trunk/ql/src/test/queries/clientpositive/drop_index.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/drop_index.q?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/drop_index.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/drop_index.q Wed Oct 15 01:08:16 2014
@@ -1,3 +1,2 @@
-SET hive.exec.drop.ignorenonexistent=false;
 DROP INDEX IF EXISTS UnknownIndex ON src;
 DROP INDEX IF EXISTS UnknownIndex ON UnknownTable;

Modified: hive/trunk/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out Wed Oct 15 01:08:16 2014
@@ -76,5 +76,5 @@ PREHOOK: query: show indexes on src_rc_c
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: show indexes on src_rc_concatenate_test
 POSTHOOK: type: SHOWINDEXES
-src_rc_concatenate_test_index	src_rc_concatenate_test	key                 	default__src_rc_concatenate_test_src_rc_concatenate_test_index__	compact             	
+src_rc_concatenate_test_index	src_rc_concatenate_test	key                 	default.default__src_rc_concatenate_test_src_rc_concatenate_test_index__	compact             	
 FAILED: SemanticException org.apache.hadoop.hive.ql.parse.SemanticException: can not do merge because source table default.src_rc_concatenate_test is indexed.

Modified: hive/trunk/ql/src/test/results/clientpositive/alter_concatenate_indexed_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/alter_concatenate_indexed_table.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/alter_concatenate_indexed_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/alter_concatenate_indexed_table.q.out Wed Oct 15 01:08:16 2014
@@ -76,7 +76,7 @@ PREHOOK: query: show indexes on src_rc_c
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: show indexes on src_rc_concatenate_test
 POSTHOOK: type: SHOWINDEXES
-src_rc_concatenate_test_index	src_rc_concatenate_test	key                 	default__src_rc_concatenate_test_src_rc_concatenate_test_index__	compact             	
+src_rc_concatenate_test_index	src_rc_concatenate_test	key                 	default.default__src_rc_concatenate_test_src_rc_concatenate_test_index__	compact             	
 PREHOOK: query: alter table src_rc_concatenate_test concatenate
 PREHOOK: type: ALTER_TABLE_MERGE
 PREHOOK: Input: default@src_rc_concatenate_test
@@ -215,7 +215,7 @@ PREHOOK: query: show indexes on src_rc_c
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: show indexes on src_rc_concatenate_test_part
 POSTHOOK: type: SHOWINDEXES
-src_rc_concatenate_test_part_index	src_rc_concatenate_test_part	key                 	default__src_rc_concatenate_test_part_src_rc_concatenate_test_part_index__	compact             	
+src_rc_concatenate_test_part_index	src_rc_concatenate_test_part	key                 	default.default__src_rc_concatenate_test_part_src_rc_concatenate_test_part_index__	compact             	
 PREHOOK: query: alter table src_rc_concatenate_test_part partition (ds='2011') concatenate
 PREHOOK: type: ALTER_PARTITION_MERGE
 PREHOOK: Input: default@src_rc_concatenate_test_part

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auth.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auth.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auth.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auth.q.out Wed Oct 15 01:08:16 2014
@@ -24,7 +24,7 @@ PREHOOK: query: SHOW INDEXES ON foobar
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: SHOW INDEXES ON foobar
 POSTHOOK: type: SHOWINDEXES
-srcpart_auth_index  	foobar              	key                 	default__foobar_srcpart_auth_index__	bitmap              	
+srcpart_auth_index  	foobar              	key                 	default.default__foobar_srcpart_auth_index__	bitmap              	
 PREHOOK: query: grant select on table foobar to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@foobar

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto.q.out Wed Oct 15 01:08:16 2014
@@ -146,7 +146,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: ((key > 80) and (key < 100)) (type: boolean)
             Filter Operator
               predicate: ((key > 80) and (key < 100)) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_file_format.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_file_format.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_file_format.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_file_format.q.out Wed Oct 15 01:08:16 2014
@@ -40,7 +40,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: (key = 86) (type: boolean)
             Filter Operator
               predicate: (key = 86) (type: boolean)
@@ -156,7 +156,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: (key = 86) (type: boolean)
             Filter Operator
               predicate: (key = 86) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables.q.out Wed Oct 15 01:08:16 2014
@@ -203,7 +203,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__srcpart_srcpart_index__
+            alias: default.default__srcpart_srcpart_index__
             filterExpr: (((((key > 70) and (key < 90)) and (key > 80)) and (key < 100)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
             Filter Operator
               predicate: (((((key > 70) and (key < 90)) and (key > 80)) and (key < 100)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
@@ -295,7 +295,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: (((((key > 80) and (key < 100)) and (key > 70)) and (key < 90)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
             Filter Operator
               predicate: (((((key > 80) and (key < 100)) and (key > 70)) and (key < 90)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables_compact.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables_compact.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables_compact.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables_compact.q.out Wed Oct 15 01:08:16 2014
@@ -213,7 +213,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__srcpart_srcpart_index__
+            alias: default.default__srcpart_srcpart_index__
             filterExpr: ((((key > 70) and (key < 90)) and (key > 80)) and (key < 100)) (type: boolean)
             Filter Operator
               predicate: ((((key > 70) and (key < 90)) and (key > 80)) and (key < 100)) (type: boolean)
@@ -323,7 +323,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: ((((key > 80) and (key < 100)) and (key > 70)) and (key < 90)) (type: boolean)
             Filter Operator
               predicate: ((((key > 80) and (key < 100)) and (key > 70)) and (key < 90)) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_multiple.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_multiple.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_multiple.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_multiple.q.out Wed Oct 15 01:08:16 2014
@@ -60,7 +60,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_key_index__
+            alias: default.default__src_src_key_index__
             filterExpr: (key = 86) (type: boolean)
             Filter Operator
               predicate: (key = 86) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_partitioned.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_partitioned.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_partitioned.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_partitioned.q.out Wed Oct 15 01:08:16 2014
@@ -54,7 +54,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__srcpart_src_part_index__
+            alias: default.default__srcpart_src_part_index__
             filterExpr: ((key = 86) and (ds = '2008-04-09')) (type: boolean)
             Filter Operator
               predicate: (key = 86) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_self_join.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_self_join.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_self_join.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_self_join.q.out Wed Oct 15 01:08:16 2014
@@ -123,7 +123,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: (((key > 70) and (key < 90)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
             Filter Operator
               predicate: (((key > 70) and (key < 90)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
@@ -216,7 +216,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: (((key > 80) and (key < 100)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
             Filter Operator
               predicate: (((key > 80) and (key < 100)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_auto_update.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_auto_update.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_auto_update.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_auto_update.q.out Wed Oct 15 01:08:16 2014
@@ -222,7 +222,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__temp_temp_index__
+            alias: default.default__temp_temp_index__
             filterExpr: (key = 86) (type: boolean)
             Filter Operator
               predicate: (key = 86) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_bitmap_auto_partitioned.q.out Wed Oct 15 01:08:16 2014
@@ -52,7 +52,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__srcpart_src_part_index__
+            alias: default.default__srcpart_src_part_index__
             filterExpr: ((key = 86) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
             Filter Operator
               predicate: ((key = 86) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_bitmap_compression.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_bitmap_compression.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_bitmap_compression.q.out Wed Oct 15 01:08:16 2014
@@ -38,7 +38,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: (((key > 80) and (key < 100)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)
             Filter Operator
               predicate: (((key > 80) and (key < 100)) and (not EWAH_BITMAP_EMPTY(_bitmaps))) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_compression.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_compression.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_compression.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_compression.q.out Wed Oct 15 01:08:16 2014
@@ -42,7 +42,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__src_src_index__
+            alias: default.default__src_src_index__
             filterExpr: ((key > 80) and (key < 100)) (type: boolean)
             Filter Operator
               predicate: ((key > 80) and (key < 100)) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/index_serde.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/index_serde.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/index_serde.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/index_serde.q.out Wed Oct 15 01:08:16 2014
@@ -134,7 +134,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__doctors_doctors_index__
+            alias: default.default__doctors_doctors_index__
             filterExpr: (number > 6) (type: boolean)
             Filter Operator
               predicate: (number > 6) (type: boolean)

Modified: hive/trunk/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx.q.out Wed Oct 15 01:08:16 2014
@@ -257,7 +257,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__lineitem_lineitem_lshipdate_idx__
+            alias: default.default__lineitem_lineitem_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8675 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
@@ -584,7 +584,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__lineitem_lineitem_lshipdate_idx__
+            alias: default.default__lineitem_lineitem_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8675 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
@@ -762,10 +762,10 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: lastyear:default__lineitem_lineitem_lshipdate_idx__
+            alias: thisyear:default.default__lineitem_lineitem_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8675 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (year(l_shipdate) = 1997) (type: boolean)
+              predicate: (year(l_shipdate) = 1998) (type: boolean)
               Statistics: Num rows: 47 Data size: 4291 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
@@ -846,10 +846,10 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: thisyear:default__lineitem_lineitem_lshipdate_idx__
+            alias: lastyear:default.default__lineitem_lineitem_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8675 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (year(l_shipdate) = 1998) (type: boolean)
+              predicate: (year(l_shipdate) = 1997) (type: boolean)
               Statistics: Num rows: 47 Data size: 4291 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
@@ -916,7 +916,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: null-subquery1:dummy-subquery1:default__lineitem_lineitem_lshipdate_idx__
+            alias: null-subquery1:dummy-subquery1:default.default__lineitem_lineitem_lshipdate_idx__
             Statistics: Num rows: 95 Data size: 8675 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: l_shipdate (type: string), _count_of_l_shipdate (type: bigint)
@@ -1095,7 +1095,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__tbl_tbl_key_idx__
+            alias: default.default__tbl_tbl_key_idx__
             Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
             Select Operator
               expressions: key (type: int), _count_of_key (type: bigint)
@@ -2364,7 +2364,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__tblpart_tbl_part_index__
+            alias: default.default__tblpart_tbl_part_index__
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: (key < 10) (type: boolean)
@@ -2565,7 +2565,7 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: default__tbl_tbl_key_idx__
+            alias: default.default__tbl_tbl_key_idx__
             Statistics: Num rows: 6 Data size: 430 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: key (type: int), _count_of_key (type: bigint)

Modified: hive/trunk/ql/src/test/results/clientpositive/show_indexes_edge_cases.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/show_indexes_edge_cases.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/show_indexes_edge_cases.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/show_indexes_edge_cases.q.out Wed Oct 15 01:08:16 2014
@@ -118,10 +118,10 @@ PREHOOK: query: SHOW INDEXES ON show_idx
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: SHOW INDEXES ON show_idx_full
 POSTHOOK: type: SHOWINDEXES
-idx_1               	show_idx_full       	key                 	default__show_idx_full_idx_1__	compact             	
-idx_2               	show_idx_full       	value1              	default__show_idx_full_idx_2__	compact             	
-idx_comment         	show_idx_full       	value2              	default__show_idx_full_idx_comment__	compact             	index comment       
-idx_compound        	show_idx_full       	key, value1         	default__show_idx_full_idx_compound__	compact             	
+idx_1               	show_idx_full       	key                 	default.default__show_idx_full_idx_1__	compact             	
+idx_2               	show_idx_full       	value1              	default.default__show_idx_full_idx_2__	compact             	
+idx_comment         	show_idx_full       	value2              	default.default__show_idx_full_idx_comment__	compact             	index comment       
+idx_compound        	show_idx_full       	key, value1         	default.default__show_idx_full_idx_compound__	compact             	
 PREHOOK: query: EXPLAIN SHOW INDEXES ON show_idx_empty
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: EXPLAIN SHOW INDEXES ON show_idx_empty

Modified: hive/trunk/ql/src/test/results/clientpositive/show_indexes_syntax.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/show_indexes_syntax.q.out?rev=1631925&r1=1631924&r2=1631925&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/show_indexes_syntax.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/show_indexes_syntax.q.out Wed Oct 15 01:08:16 2014
@@ -53,7 +53,7 @@ PREHOOK: query: SHOW INDEX ON show_idx_t
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: SHOW INDEX ON show_idx_t1
 POSTHOOK: type: SHOWINDEXES
-idx_t1              	show_idx_t1         	key                 	default__show_idx_t1_idx_t1__	compact             	
+idx_t1              	show_idx_t1         	key                 	default.default__show_idx_t1_idx_t1__	compact             	
 PREHOOK: query: EXPLAIN
 SHOW INDEXES ON show_idx_t1
 PREHOOK: type: SHOWINDEXES
@@ -79,7 +79,7 @@ PREHOOK: query: SHOW INDEXES ON show_idx
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: SHOW INDEXES ON show_idx_t1
 POSTHOOK: type: SHOWINDEXES
-idx_t1              	show_idx_t1         	key                 	default__show_idx_t1_idx_t1__	compact             	
+idx_t1              	show_idx_t1         	key                 	default.default__show_idx_t1_idx_t1__	compact             	
 PREHOOK: query: EXPLAIN
 SHOW FORMATTED INDEXES ON show_idx_t1
 PREHOOK: type: SHOWINDEXES
@@ -108,7 +108,7 @@ POSTHOOK: type: SHOWINDEXES
 idx_name            	tab_name            	col_names           	idx_tab_name        	idx_type            	comment             
 	 	 	 	 	 
 	 	 	 	 	 
-idx_t1              	show_idx_t1         	key                 	default__show_idx_t1_idx_t1__	compact             	
+idx_t1              	show_idx_t1         	key                 	default.default__show_idx_t1_idx_t1__	compact             	
 PREHOOK: query: DROP TABLE show_idx_t1
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@show_idx_t1