Author: hashutosh
Date: Mon Mar  3 15:49:52 2014
New Revision: 1573590

URL: http://svn.apache.org/r1573590
Log:
HIVE-3938 : Hive MetaStore should send a single AddPartitionEvent for 
atomically added partition-set. (Mithun Radhakrishnan via Ashutosh Chauhan)

Modified:
    
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java
    
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java
    
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java
    
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
    
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
    
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
    
hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
    
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
    
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
    
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java

Modified: 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java
 (original)
+++ 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java
 Mon Mar  3 15:49:52 2014
@@ -21,6 +21,8 @@ package org.apache.hcatalog.listener;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import javax.jms.Connection;
 import javax.jms.ConnectionFactory;
@@ -91,15 +93,8 @@ public class NotificationListener extend
     createConnection();
   }
 
-  private static String getTopicName(Partition partition,
-                     ListenerEvent partitionEvent) throws MetaException {
-    try {
-      return partitionEvent.getHandler()
-        .get_table(partition.getDbName(), partition.getTableName())
-        .getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);
-    } catch (NoSuchObjectException e) {
-      throw new MetaException(e.toString());
-    }
+  private static String getTopicName(Table table, ListenerEvent 
partitionEvent) {
+    return table.getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);
   }
 
   @Override
@@ -109,22 +104,21 @@ public class NotificationListener extend
     // particular table by listening on a topic named "dbName.tableName"
     // and message selector string as "HCAT_EVENT = HCAT_ADD_PARTITION"
     if (partitionEvent.getStatus()) {
-
-      Partition partition = partitionEvent.getPartition();
-      String topicName = getTopicName(partition, partitionEvent);
-      if (topicName != null && !topicName.equals("")) {
-        
send(messageFactory.buildAddPartitionMessage(partitionEvent.getTable(), 
partition), topicName);
-      } else {
-        LOG.info("Topic name not found in metastore. Suppressing HCatalog 
notification for "
-          + partition.getDbName()
-          + "."
-          + partition.getTableName()
-          + " To enable notifications for this table, please do alter table 
set properties ("
-          + HCatConstants.HCAT_MSGBUS_TOPIC_NAME
-          + "=<dbname>.<tablename>) or whatever you want topic name to be.");
+        Table table = partitionEvent.getTable();
+        List<Partition> partitions = partitionEvent.getPartitions();
+        String topicName = getTopicName(table, partitionEvent);
+        if (topicName != null && !topicName.equals("")) {
+            send(messageFactory.buildAddPartitionMessage(table, partitions), 
topicName);
+        } else {
+            LOG.info("Topic name not found in metastore. Suppressing HCatalog 
notification for "
+                + partitions.get(0).getDbName()
+                + "."
+                + partitions.get(0).getTableName()
+                + " To enable notifications for this table, please do alter 
table set properties ("
+                + HCatConstants.HCAT_MSGBUS_TOPIC_NAME
+                + "=<dbname>.<tablename>) or whatever you want topic name to 
be.");
+        }
       }
-    }
-
   }
 
   /**
@@ -148,7 +142,7 @@ public class NotificationListener extend
       sd.setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSkewedInfo().setSkewedColNames(new ArrayList<String>());
-      String topicName = getTopicName(partition, partitionEvent);
+      String topicName = getTopicName(partitionEvent.getTable(), 
partitionEvent);
       if (topicName != null && !topicName.equals("")) {
         
send(messageFactory.buildDropPartitionMessage(partitionEvent.getTable(), 
partition), topicName);
       } else {

Modified: 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java
 (original)
+++ 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java
 Mon Mar  3 15:49:52 2014
@@ -26,6 +26,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hcatalog.messaging.json.JSONMessageFactory;
 
+import java.util.List;
+
 /**
  * Abstract Factory for the construction of HCatalog message instances.
  * @deprecated Use/modify {@link 
org.apache.hive.hcatalog.messaging.MessageFactory} instead
@@ -123,11 +125,11 @@ public abstract class MessageFactory {
 
   /**
    * Factory method for AddPartitionMessage.
-   * @param table The Table to which the partition is added.
-   * @param partition The Partition being added.
+   * @param table The Table to which the partitions are added.
+   * @param partitions The set of Partitions being added.
    * @return AddPartitionMessage instance.
    */
-  public abstract AddPartitionMessage buildAddPartitionMessage(Table table, 
Partition partition);
+  public abstract AddPartitionMessage buildAddPartitionMessage(Table table, 
List<Partition> partitions);
 
   /**
    * Factory method for DropPartitionMessage.

Modified: 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java
 (original)
+++ 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java
 Mon Mar  3 15:49:52 2014
@@ -31,8 +31,10 @@ import org.apache.hcatalog.messaging.Dro
 import org.apache.hcatalog.messaging.MessageDeserializer;
 import org.apache.hcatalog.messaging.MessageFactory;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -84,10 +86,9 @@ public class JSONMessageFactory extends 
   }
 
   @Override
-  public AddPartitionMessage buildAddPartitionMessage(Table table, Partition 
partition) {
-    return new JSONAddPartitionMessage(HCAT_SERVER_URL, 
HCAT_SERVICE_PRINCIPAL, partition.getDbName(),
-        partition.getTableName(), Arrays.asList(getPartitionKeyValues(table, 
partition)),
-        System.currentTimeMillis()/1000);
+  public AddPartitionMessage buildAddPartitionMessage(Table table, 
List<Partition> partitions) {
+    return new JSONAddPartitionMessage(HCAT_SERVER_URL, 
HCAT_SERVICE_PRINCIPAL, table.getDbName(),
+        table.getTableName(), getPartitionKeyValues(table, partitions), 
System.currentTimeMillis()/1000);
   }
 
   @Override
@@ -104,4 +105,11 @@ public class JSONMessageFactory extends 
           partition.getValues().get(i));
     return partitionKeys;
   }
-}
+
+  private static List<Map<String, String>> getPartitionKeyValues(Table table, 
List<Partition> partitions) {
+    List<Map<String, String>> partitionList = new ArrayList<Map<String, 
String>>(partitions.size());
+    for (Partition partition : partitions)
+      partitionList.add(getPartitionKeyValues(table, partition));
+    return partitionList;
+  }
+}
\ No newline at end of file

Modified: 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
 (original)
+++ 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
 Mon Mar  3 15:49:52 2014
@@ -21,6 +21,8 @@ package org.apache.hive.hcatalog.listene
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import javax.jms.Connection;
 import javax.jms.ConnectionFactory;
@@ -114,15 +116,8 @@ public class NotificationListener extend
     testAndCreateConnection();
   }
 
-  private static String getTopicName(Partition partition,
-                     ListenerEvent partitionEvent) throws MetaException {
-    try {
-      return partitionEvent.getHandler()
-        .get_table(partition.getDbName(), partition.getTableName())
-        .getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);
-    } catch (NoSuchObjectException e) {
-      throw new MetaException(e.toString());
-    }
+  private static String getTopicName(Table table, ListenerEvent 
partitionEvent) {
+    return table.getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);
   }
 
   @Override
@@ -132,22 +127,21 @@ public class NotificationListener extend
     // particular table by listening on a topic named "dbName.tableName"
     // and message selector string as "HCAT_EVENT = HCAT_ADD_PARTITION"
     if (partitionEvent.getStatus()) {
-
-      Partition partition = partitionEvent.getPartition();
-      String topicName = getTopicName(partition, partitionEvent);
+      Table table = partitionEvent.getTable();
+      List<Partition> partitions = partitionEvent.getPartitions();
+      String topicName = getTopicName(table, partitionEvent);
       if (topicName != null && !topicName.equals("")) {
-        
send(messageFactory.buildAddPartitionMessage(partitionEvent.getTable(), 
partition), topicName);
+        send(messageFactory.buildAddPartitionMessage(table, partitions), 
topicName);
       } else {
         LOG.info("Topic name not found in metastore. Suppressing HCatalog 
notification for "
-          + partition.getDbName()
-          + "."
-          + partition.getTableName()
-          + " To enable notifications for this table, please do alter table 
set properties ("
-          + HCatConstants.HCAT_MSGBUS_TOPIC_NAME
-          + "=<dbname>.<tablename>) or whatever you want topic name to be.");
+            + partitions.get(0).getDbName()
+            + "."
+            + partitions.get(0).getTableName()
+            + " To enable notifications for this table, please do alter table 
set properties ("
+            + HCatConstants.HCAT_MSGBUS_TOPIC_NAME
+            + "=<dbname>.<tablename>) or whatever you want topic name to be.");
       }
     }
-
   }
 
   /**
@@ -171,7 +165,7 @@ public class NotificationListener extend
       sd.setParameters(new HashMap<String, String>());
       sd.getSerdeInfo().setParameters(new HashMap<String, String>());
       sd.getSkewedInfo().setSkewedColNames(new ArrayList<String>());
-      String topicName = getTopicName(partition, partitionEvent);
+      String topicName = getTopicName(partitionEvent.getTable(), 
partitionEvent);
       if (topicName != null && !topicName.equals("")) {
         
send(messageFactory.buildDropPartitionMessage(partitionEvent.getTable(), 
partition), topicName);
       } else {

Modified: 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
 (original)
+++ 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
 Mon Mar  3 15:49:52 2014
@@ -26,6 +26,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hive.hcatalog.messaging.json.JSONMessageFactory;
 
+import java.util.List;
+
 /**
  * Abstract Factory for the construction of HCatalog message instances.
  */
@@ -120,13 +122,13 @@ public abstract class MessageFactory {
    */
   public abstract DropTableMessage buildDropTableMessage(Table table);
 
-  /**
-   * Factory method for AddPartitionMessage.
-   * @param table The Table to which the partition is added.
-   * @param partition The Partition being added.
-   * @return AddPartitionMessage instance.
-   */
-  public abstract AddPartitionMessage buildAddPartitionMessage(Table table, 
Partition partition);
+    /**
+     * Factory method for AddPartitionMessage.
+     * @param table The Table to which the partitions are added.
+     * @param partitions The set of Partitions being added.
+     * @return AddPartitionMessage instance.
+     */
+    public abstract AddPartitionMessage buildAddPartitionMessage(Table table, 
List<Partition> partitions);
 
   /**
    * Factory method for DropPartitionMessage.

Modified: 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
 (original)
+++ 
hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
 Mon Mar  3 15:49:52 2014
@@ -31,9 +31,7 @@ import org.apache.hive.hcatalog.messagin
 import org.apache.hive.hcatalog.messaging.MessageDeserializer;
 import org.apache.hive.hcatalog.messaging.MessageFactory;
 
-import java.util.Arrays;
-import java.util.LinkedHashMap;
-import java.util.Map;
+import java.util.*;
 
 /**
  * The JSON implementation of the MessageFactory. Constructs JSON 
implementations of
@@ -83,10 +81,9 @@ public class JSONMessageFactory extends 
   }
 
   @Override
-  public AddPartitionMessage buildAddPartitionMessage(Table table, Partition 
partition) {
-    return new JSONAddPartitionMessage(HCAT_SERVER_URL, 
HCAT_SERVICE_PRINCIPAL, partition.getDbName(),
-        partition.getTableName(), Arrays.asList(getPartitionKeyValues(table, 
partition)),
-        System.currentTimeMillis()/1000);
+  public AddPartitionMessage buildAddPartitionMessage(Table table, 
List<Partition> partitions) {
+    return new JSONAddPartitionMessage(HCAT_SERVER_URL, 
HCAT_SERVICE_PRINCIPAL, table.getDbName(),
+        table.getTableName(), getPartitionKeyValues(table, partitions), 
System.currentTimeMillis()/1000);
   }
 
   @Override
@@ -103,4 +100,11 @@ public class JSONMessageFactory extends 
           partition.getValues().get(i));
     return partitionKeys;
   }
+
+  private static List<Map<String, String>> getPartitionKeyValues(Table table, 
List<Partition> partitions) {
+    List<Map<String, String>> partitionList = new ArrayList<Map<String, 
String>>(partitions.size());
+    for (Partition partition : partitions)
+      partitionList.add(getPartitionKeyValues(table, partition));
+    return partitionList;
+  }
 }

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=1573590&r1=1573589&r2=1573590&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
 Mon Mar  3 15:49:52 2014
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.metastore;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -223,11 +224,30 @@ public class TestMetaStoreEventListener 
 
     AddPartitionEvent partEvent = 
(AddPartitionEvent)(notifyList.get(listSize-1));
     assert partEvent.getStatus();
-    validateAddPartition(part, partEvent.getPartition());
+    validateAddPartition(part, partEvent.getPartitions().get(0));
     validateTableInAddPartition(tbl, partEvent.getTable());
 
     PreAddPartitionEvent prePartEvent = 
(PreAddPartitionEvent)(preNotifyList.get(listSize-1));
-    validateAddPartition(part, prePartEvent.getPartition());
+    validateAddPartition(part, prePartEvent.getPartitions().get(0));
+
+    // Test adding multiple partitions in a single partition-set, atomically.
+    int currentTime = (int)System.currentTimeMillis();
+    HiveMetaStoreClient hmsClient = new HiveMetaStoreClient(hiveConf);
+    Table table = hmsClient.getTable(dbName, "tmptbl");
+    Partition partition1 = new Partition(Arrays.asList("20110101"), dbName, 
"tmptbl", currentTime,
+                                        currentTime, table.getSd(), 
table.getParameters());
+    Partition partition2 = new Partition(Arrays.asList("20110102"), dbName, 
"tmptbl", currentTime,
+                                        currentTime, table.getSd(), 
table.getParameters());
+    Partition partition3 = new Partition(Arrays.asList("20110103"), dbName, 
"tmptbl", currentTime,
+                                        currentTime, table.getSd(), 
table.getParameters());
+    hmsClient.add_partitions(Arrays.asList(partition1, partition2, 
partition3));
+    ++listSize;
+    AddPartitionEvent multiplePartitionEvent = 
(AddPartitionEvent)(notifyList.get(listSize-1));
+    assertEquals("Unexpected number of partitions in event!", 3, 
multiplePartitionEvent.getPartitions().size());
+    assertEquals("Unexpected table value.", table, 
multiplePartitionEvent.getTable());
+    assertEquals("Unexpected partition value.", partition1.getValues(), 
multiplePartitionEvent.getPartitions().get(0).getValues());
+    assertEquals("Unexpected partition value.", partition2.getValues(), 
multiplePartitionEvent.getPartitions().get(1).getValues());
+    assertEquals("Unexpected partition value.", partition3.getValues(), 
multiplePartitionEvent.getPartitions().get(2).getValues());
 
     driver.run(String.format("alter table %s touch partition (%s)", tblName, 
"b='2011'"));
     listSize++;
@@ -260,11 +280,11 @@ public class TestMetaStoreEventListener 
 
     AddPartitionEvent appendPartEvent =
         (AddPartitionEvent)(notifyList.get(listSize-1));
-    validateAddPartition(newPart, appendPartEvent.getPartition());
+    validateAddPartition(newPart, appendPartEvent.getPartitions().get(0));
 
     PreAddPartitionEvent preAppendPartEvent =
         (PreAddPartitionEvent)(preNotifyList.get(listSize-1));
-    validateAddPartition(newPart, preAppendPartEvent.getPartition());
+    validateAddPartition(newPart, preAppendPartEvent.getPartitions().get(0));
 
     driver.run(String.format("alter table %s rename to %s", tblName, renamed));
     listSize++;

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=1573590&r1=1573589&r2=1573590&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
 Mon Mar  3 15:49:52 2014
@@ -1634,9 +1634,6 @@ public class HiveMetaStore extends Thrif
         part.setTableName(tableName);
         part.setValues(part_vals);
 
-        PreAddPartitionEvent event = new PreAddPartitionEvent(part, this);
-        firePreEvent(event);
-
         MetaStoreUtils.validatePartitionNameCharacters(part_vals, 
partitionValidationPattern);
 
         tbl = ms.getTable(part.getDbName(), part.getTableName());
@@ -1649,6 +1646,8 @@ public class HiveMetaStore extends Thrif
               "Cannot append a partition to a view");
         }
 
+        firePreEvent(new PreAddPartitionEvent(tbl, part, this));
+
         part.setSd(tbl.getSd());
         partLocation = new Path(tbl.getSd().getLocation(), Warehouse
             .makePartName(tbl.getPartitionKeys(), part_vals));
@@ -1793,13 +1792,19 @@ public class HiveMetaStore extends Thrif
       Map<PartValEqWrapper, Boolean> addedPartitions = new 
HashMap<PartValEqWrapper, Boolean>();
       List<Partition> result = new ArrayList<Partition>();
       List<Partition> existingParts = null;
+      Table tbl = null;
       try {
         ms.openTransaction();
-        Table tbl = ms.getTable(dbName, tblName);
+        tbl = ms.getTable(dbName, tblName);
         if (tbl == null) {
           throw new InvalidObjectException("Unable to add partitions because "
               + "database or table " + dbName + "." + tblName + " does not 
exist");
         }
+
+        if (!parts.isEmpty()) {
+          firePreEvent(new PreAddPartitionEvent(tbl, parts, this));
+        }
+
         for (Partition part : parts) {
           if (!part.getTableName().equals(tblName) || 
!part.getDbName().equals(dbName)) {
             throw new MetaException("Partition does not belong to target table 
"
@@ -1839,18 +1844,12 @@ public class HiveMetaStore extends Thrif
               // we just created this directory - it's not a case of 
pre-creation, so we nuke
             }
           }
-          for (Partition part : parts) {
-            fireMetaStoreAddPartitionEvent(ms, part, null, success);
-          }
+          fireMetaStoreAddPartitionEvent(tbl, parts, null, false);
         } else {
-          for (Partition part : result) {
-            fireMetaStoreAddPartitionEvent(ms, part, null, success);
-          }
+          fireMetaStoreAddPartitionEvent(tbl, result, null, true);
           if (existingParts != null) {
             // The request has succeeded but we failed to add these partitions.
-            for (Partition part : existingParts) {
-              fireMetaStoreAddPartitionEvent(ms, part, null, false);
-            }
+            fireMetaStoreAddPartitionEvent(tbl, existingParts, null, false);
           }
         }
       }
@@ -1913,7 +1912,6 @@ public class HiveMetaStore extends Thrif
 
     private boolean startAddPartition(
         RawStore ms, Partition part, boolean ifNotExists) throws 
MetaException, TException {
-      firePreEvent(new PreAddPartitionEvent(part, this));
       MetaStoreUtils.validatePartitionNameCharacters(part.getValues(),
           partitionValidationPattern);
       boolean doesExist = ms.doesPartitionExist(
@@ -2007,16 +2005,19 @@ public class HiveMetaStore extends Thrif
         final Partition part, final EnvironmentContext envContext)
         throws InvalidObjectException, AlreadyExistsException, MetaException, 
TException {
       boolean success = false;
-      Partition retPtn = null;
+      Table tbl = null;
       try {
         ms.openTransaction();
-        Table tbl = ms.getTable(part.getDbName(), part.getTableName());
+        tbl = ms.getTable(part.getDbName(), part.getTableName());
         if (tbl == null) {
           throw new InvalidObjectException(
               "Unable to add partition because table or database do not 
exist");
         }
+
+        firePreEvent(new PreAddPartitionEvent(tbl, part, this));
+
         boolean shouldAdd = startAddPartition(ms, part, false);
-        assert shouldAdd; // start would thrrow if it already existed here
+        assert shouldAdd; // start would throw if it already existed here
         boolean madeDir = createLocationForAddedPartition(tbl, part);
         try {
           initializeAddedPartition(tbl, part, madeDir);
@@ -2033,20 +2034,22 @@ public class HiveMetaStore extends Thrif
         if (!success) {
           ms.rollbackTransaction();
         }
-        fireMetaStoreAddPartitionEvent(ms, part, envContext, success);
+        fireMetaStoreAddPartitionEvent(tbl, Arrays.asList(part), envContext, 
success);
       }
       return part;
     }
 
-    private void fireMetaStoreAddPartitionEvent(final RawStore ms,
-        final Partition part, final EnvironmentContext envContext, boolean 
success)
+    private void fireMetaStoreAddPartitionEvent(final Table tbl,
+        final List<Partition> parts, final EnvironmentContext envContext, 
boolean success)
           throws MetaException {
-      final Table tbl = ms.getTable(part.getDbName(), part.getTableName());
-      for (MetaStoreEventListener listener : listeners) {
+      if (tbl != null && parts != null && !parts.isEmpty()) {
         AddPartitionEvent addPartitionEvent =
-            new AddPartitionEvent(tbl, part, success, this);
+            new AddPartitionEvent(tbl, parts, success, this);
         addPartitionEvent.setEnvironmentContext(envContext);
-        listener.onAddPartition(addPartitionEvent);
+
+        for (MetaStoreEventListener listener : listeners) {
+          listener.onAddPartition(addPartitionEvent);
+        }
       }
     }
 

Modified: 
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
 (original)
+++ 
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
 Mon Mar  3 15:49:52 2014
@@ -22,28 +22,36 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 
+import java.util.Arrays;
+import java.util.List;
+
 public class AddPartitionEvent extends ListenerEvent {
 
   private final Table table;
-  private final Partition partition;
+  private final List<Partition> partitions;
 
-  public AddPartitionEvent (Table table, Partition partition, boolean status, 
HMSHandler handler) {
-    super (status, handler);
+  public AddPartitionEvent(Table table, List<Partition> partitions, boolean 
status, HMSHandler handler) {
+    super(status, handler);
     this.table = table;
-    this.partition = partition;
+    this.partitions = partitions;
   }
 
-  /**
-   * @return the partition
-   */
-  public Partition getPartition() {
-    return partition;
+  public AddPartitionEvent(Table table, Partition partition, boolean status, 
HMSHandler handler) {
+    this(table, Arrays.asList(partition), status, handler);
   }
 
   /**
-   * @return the table
+   * @return The table.
    */
   public Table getTable() {
     return table;
   }
+
+  /**
+   * @return List of partitions.
+   */
+  public List<Partition> getPartitions() {
+    return partitions;
+  }
+
 }

Modified: 
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
 (original)
+++ 
hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
 Mon Mar  3 15:49:52 2014
@@ -20,20 +20,37 @@ package org.apache.hadoop.hive.metastore
 
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.Arrays;
+import java.util.List;
 
 public class PreAddPartitionEvent extends PreEventContext {
 
-  private final Partition partition;
+  private final Table table;
+  private final List<Partition> partitions;
 
-  public PreAddPartitionEvent (Partition partition, HMSHandler handler) {
+  public PreAddPartitionEvent (Table table, List<Partition> partitions, 
HMSHandler handler) {
     super(PreEventType.ADD_PARTITION, handler);
-    this.partition = partition;
+    this.table = table;
+    this.partitions = partitions;
+  }
+
+  public PreAddPartitionEvent(Table table, Partition partition, HMSHandler 
handler) {
+    this(table, Arrays.asList(partition), handler);
+  }
+
+  /**
+   * @return the partitions
+   */
+  public List<Partition> getPartitions() {
+    return partitions;
   }
 
   /**
-   * @return the partition
+   * @return the table
    */
-  public Partition getPartition() {
-    return partition;
+  public Table getTable() {
+    return table ;
   }
 }

Modified: 
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
URL: 
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java?rev=1573590&r1=1573589&r2=1573590&view=diff
==============================================================================
--- 
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
 (original)
+++ 
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
 Mon Mar  3 15:49:52 2014
@@ -224,10 +224,11 @@ public class AuthorizationPreEventListen
   private void authorizeAddPartition(PreAddPartitionEvent context)
       throws InvalidOperationException, MetaException {
     try {
-      org.apache.hadoop.hive.metastore.api.Partition mapiPart = 
context.getPartition();
-      tAuthorizer.get().authorize(getPartitionFromApiPartition(mapiPart, 
context),
-          HiveOperation.ALTERTABLE_ADDPARTS.getInputRequiredPrivileges(),
-          HiveOperation.ALTERTABLE_ADDPARTS.getOutputRequiredPrivileges());
+      for (org.apache.hadoop.hive.metastore.api.Partition mapiPart : 
context.getPartitions()) {
+        tAuthorizer.get().authorize(getPartitionFromApiPartition(mapiPart, 
context),
+            HiveOperation.ALTERTABLE_ADDPARTS.getInputRequiredPrivileges(),
+            HiveOperation.ALTERTABLE_ADDPARTS.getOutputRequiredPrivileges());
+      }
     } catch (AuthorizationException e) {
       throw invalidOperationException(e);
     } catch (NoSuchObjectException e) {


Reply via email to