You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kh...@apache.org on 2015/01/15 02:12:27 UTC

svn commit: r1651923 - in /hive/trunk: hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/ hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/ hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/...

Author: khorgath
Date: Thu Jan 15 01:12:26 2015
New Revision: 1651923

URL: http://svn.apache.org/r1651923
Log:
HIVE-9175 : Add alters to list of events handled by NotificationListener (Alan Gates via Sushanth Sowmyan)

Added:
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterPartitionMessage.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterTableMessage.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterPartitionMessage.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterTableMessage.java
Modified:
    hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.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/HCatEventMessage.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.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/JSONMessageDeserializer.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
    hive/trunk/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
    hive/trunk/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java Thu Jan 15 01:12:26 2015
@@ -132,8 +132,10 @@ public final class HCatConstants {
   public static final String HCAT_EVENT = "HCAT_EVENT";
   public static final String HCAT_ADD_PARTITION_EVENT = "ADD_PARTITION";
   public static final String HCAT_DROP_PARTITION_EVENT = "DROP_PARTITION";
+  public static final String HCAT_ALTER_PARTITION_EVENT = "ALTER_PARTITION";
   public static final String HCAT_PARTITION_DONE_EVENT = "PARTITION_DONE";
   public static final String HCAT_CREATE_TABLE_EVENT = "CREATE_TABLE";
+  public static final String HCAT_ALTER_TABLE_EVENT = "ALTER_TABLE";
   public static final String HCAT_DROP_TABLE_EVENT = "DROP_TABLE";
   public static final String HCAT_CREATE_DATABASE_EVENT = "CREATE_DATABASE";
   public static final String HCAT_DROP_DATABASE_EVENT = "DROP_DATABASE";

Modified: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java (original)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java Thu Jan 15 01:12:26 2015
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
@@ -140,7 +141,7 @@ public class DbNotificationListener exte
    * @throws MetaException
    */
   public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
-    /*Table before = tableEvent.getOldTable();
+    Table before = tableEvent.getOldTable();
     Table after = tableEvent.getNewTable();
     NotificationEvent event = new NotificationEvent(0, now(),
         HCatConstants.HCAT_ALTER_TABLE_EVENT,
@@ -149,8 +150,7 @@ public class DbNotificationListener exte
       event.setDbName(after.getDbName());
       event.setTableName(after.getTableName());
       enqueue(event);
-    }*/
-    // TODO - once HIVE-9175 is committed
+    }
   }
 
   /**
@@ -187,7 +187,16 @@ public class DbNotificationListener exte
    * @throws MetaException
    */
   public void onAlterPartition (AlterPartitionEvent partitionEvent)  throws MetaException {
-    // TODO, MessageFactory doesn't support Alter Partition yet.
+    Partition before = partitionEvent.getOldPartition();
+    Partition after = partitionEvent.getNewPartition();
+    NotificationEvent event = new NotificationEvent(0, now(),
+        HCatConstants.HCAT_ALTER_PARTITION_EVENT,
+        msgFactory.buildAlterPartitionMessage(before, after).toString());
+    if (event != null) {
+      event.setDbName(before.getDbName());
+      event.setTableName(before.getTableName());
+      enqueue(event);
+    }
   }
 
   /**

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=1651923&r1=1651922&r2=1651923&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 Thu Jan 15 01:12:26 2015
@@ -60,6 +60,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hive.hcatalog.common.HCatConstants;
+import org.apache.hive.hcatalog.messaging.AlterTableMessage;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage;
 import org.apache.hive.hcatalog.messaging.MessageFactory;
 import org.slf4j.Logger;
@@ -116,7 +117,7 @@ public class NotificationListener extend
     testAndCreateConnection();
   }
 
-  private static String getTopicName(Table table, ListenerEvent partitionEvent) {
+  private static String getTopicName(Table table) {
     return table.getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);
   }
 
@@ -129,7 +130,7 @@ public class NotificationListener extend
     if (partitionEvent.getStatus()) {
       Table table = partitionEvent.getTable();
       List<Partition> partitions = partitionEvent.getPartitions();
-      String topicName = getTopicName(table, partitionEvent);
+      String topicName = getTopicName(table);
       if (topicName != null && !topicName.equals("")) {
         send(messageFactory.buildAddPartitionMessage(table, partitions), topicName);
       } else {
@@ -144,6 +145,17 @@ public class NotificationListener extend
     }
   }
 
+  @Override
+  public void onAlterPartition(AlterPartitionEvent ape) throws MetaException {
+    if (ape.getStatus()) {
+      Partition before = ape.getOldPartition();
+      Partition after = ape.getNewPartition();
+
+      String topicName = getTopicName(ape.getTable());
+      send(messageFactory.buildAlterPartitionMessage(before, after), topicName);
+    }
+  }
+
   /**
    * Send dropped partition notifications. Subscribers can receive these notifications for a
    * particular table by listening on a topic named "dbName.tableName" with message selector
@@ -165,7 +177,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(partitionEvent.getTable(), partitionEvent);
+      String topicName = getTopicName(partitionEvent.getTable());
       if (topicName != null && !topicName.equals("")) {
         send(messageFactory.buildDropPartitionMessage(partitionEvent.getTable(), partition), topicName);
       } else {
@@ -241,6 +253,35 @@ public class NotificationListener extend
   }
 
   /**
+   * Send altered table notifications. Subscribers can receive these notifications for
+   * dropped tables by listening on topic "HCAT" with message selector string
+   * {@value org.apache.hive.hcatalog.common.HCatConstants#HCAT_EVENT} =
+   * {@value org.apache.hive.hcatalog.common.HCatConstants#HCAT_ALTER_TABLE_EVENT}
+   */
+  @Override
+  public void onAlterTable(AlterTableEvent tableEvent) throws MetaException {
+    if (tableEvent.getStatus()) {
+      Table before = tableEvent.getOldTable();
+      Table after = tableEvent.getNewTable();
+
+      // onCreateTable alters the table to add the topic name.  Since this class is generating
+      // that alter, we don't want to notify on that alter.  So take a quick look and see if
+      // that's what this this alter is, and if so swallow it.
+      if (after.getParameters() != null &&
+          after.getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME) != null &&
+          (before.getParameters() == null ||
+              before.getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME) == null)) {
+        return;
+      }
+      // I think this is wrong, the alter table statement should come on the table topic not the
+      // DB topic - Alan.
+      String topicName = getTopicPrefix(tableEvent.getHandler().getHiveConf()) + "." +
+          after.getDbName().toLowerCase();
+      send(messageFactory.buildAlterTableMessage(before, after), topicName);
+    }
+  }
+
+  /**
    * Send dropped table notifications. Subscribers can receive these notifications for
    * dropped tables by listening on topic "HCAT" with message selector string
    * {@value org.apache.hive.hcatalog.common.HCatConstants#HCAT_EVENT} =
@@ -262,6 +303,8 @@ public class NotificationListener extend
 
     if (tableEvent.getStatus()) {
       Table table = tableEvent.getTable();
+      // I think this is wrong, the drop table statement should come on the table topic not the
+      // DB topic - Alan.
       String topicName = getTopicPrefix(tableEvent.getHandler().getHiveConf()) + "." + table.getDbName().toLowerCase();
       send(messageFactory.buildDropTableMessage(table), topicName);
     }
@@ -435,14 +478,4 @@ public class NotificationListener extend
 //        if(lpde.getStatus())
 //            send(lpde.getPartitionName(),lpde.getTable().getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME),HCatConstants.HCAT_PARTITION_DONE_EVENT);
   }
-
-  @Override
-  public void onAlterPartition(AlterPartitionEvent ape) throws MetaException {
-    // no-op
-  }
-
-  @Override
-  public void onAlterTable(AlterTableEvent ate) throws MetaException {
-    // no-op
-  }
 }

Added: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterPartitionMessage.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterPartitionMessage.java?rev=1651923&view=auto
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterPartitionMessage.java (added)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterPartitionMessage.java Thu Jan 15 01:12:26 2015
@@ -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.hive.hcatalog.messaging;
+
+import java.util.List;
+
+/**
+ * HCat message sent when a table is Altered.
+ */
+public abstract class AlterPartitionMessage extends HCatEventMessage {
+
+  protected AlterPartitionMessage() {
+    super(EventType.ALTER_PARTITION);
+  }
+
+  public abstract String getTable();
+
+  public abstract List<String> getValues();
+
+  @Override
+  public HCatEventMessage checkValid() {
+    if (getTable() == null) throw new IllegalStateException("Table name unset.");
+    if (getValues() == null) throw new IllegalStateException("Partition values unset");
+    return super.checkValid();
+  }
+}
+

Added: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterTableMessage.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterTableMessage.java?rev=1651923&view=auto
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterTableMessage.java (added)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterTableMessage.java Thu Jan 15 01:12:26 2015
@@ -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.hive.hcatalog.messaging;
+
+/**
+ * HCat message sent when a table is Altered.
+ */
+public abstract class AlterTableMessage extends HCatEventMessage {
+
+  protected AlterTableMessage() {
+    super(EventType.ALTER_TABLE);
+  }
+
+  public abstract String getTable();
+
+  @Override
+  public HCatEventMessage checkValid() {
+    if (getTable() == null) throw new IllegalStateException("Table name unset.");
+    return super.checkValid();
+  }
+}

Modified: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java (original)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java Thu Jan 15 01:12:26 2015
@@ -37,7 +37,9 @@ public abstract class HCatEventMessage {
     CREATE_TABLE(HCatConstants.HCAT_CREATE_TABLE_EVENT),
     DROP_TABLE(HCatConstants.HCAT_DROP_TABLE_EVENT),
     ADD_PARTITION(HCatConstants.HCAT_ADD_PARTITION_EVENT),
-    DROP_PARTITION(HCatConstants.HCAT_DROP_PARTITION_EVENT);
+    DROP_PARTITION(HCatConstants.HCAT_DROP_PARTITION_EVENT),
+    ALTER_TABLE(HCatConstants.HCAT_ALTER_TABLE_EVENT),
+    ALTER_PARTITION(HCatConstants.HCAT_ALTER_PARTITION_EVENT);
 
     private String typeString;
 

Modified: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java (original)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java Thu Jan 15 01:12:26 2015
@@ -36,10 +36,14 @@ public abstract class MessageDeserialize
       return getDropDatabaseMessage(messageBody);
     case CREATE_TABLE:
       return getCreateTableMessage(messageBody);
+    case ALTER_TABLE:
+      return getAlterTableMessage(messageBody);
     case DROP_TABLE:
       return getDropTableMessage(messageBody);
     case ADD_PARTITION:
       return getAddPartitionMessage(messageBody);
+    case ALTER_PARTITION:
+      return getAlterPartitionMessage(messageBody);
     case DROP_PARTITION:
       return getDropPartitionMessage(messageBody);
 
@@ -64,6 +68,13 @@ public abstract class MessageDeserialize
   public abstract CreateTableMessage getCreateTableMessage(String messageBody);
 
   /**
+   * Method to de-serialize AlterTableMessge
+   * @param messageBody string message
+   * @return object message
+   */
+  public abstract AlterTableMessage getAlterTableMessage(String messageBody);
+
+  /**
    * Method to de-serialize DropTableMessage instance.
    */
   public abstract DropTableMessage getDropTableMessage(String messageBody);
@@ -74,6 +85,13 @@ public abstract class MessageDeserialize
   public abstract AddPartitionMessage getAddPartitionMessage(String messageBody);
 
   /**
+   * Method to deserialize AlterPartitionMessage
+   * @param messageBody the message in serialized form
+   * @return message in object form
+   */
+  public abstract AlterPartitionMessage getAlterPartitionMessage(String messageBody);
+
+  /**
    * Method to de-serialize DropPartitionMessage instance.
    */
   public abstract DropPartitionMessage getDropPartitionMessage(String messageBody);

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=1651923&r1=1651922&r2=1651923&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 Thu Jan 15 01:12:26 2015
@@ -119,6 +119,17 @@ public abstract class MessageFactory {
   public abstract CreateTableMessage buildCreateTableMessage(Table table);
 
   /**
+   * Factory method for AlterTableMessage.  Unlike most of these calls, this one can return null,
+   * which means no message should be sent.  This is because there are many flavors of alter
+   * table (add column, add partition, etc.).  Some are covered elsewhere (like add partition)
+   * and some are not yet supported.
+   * @param before The table before the alter
+   * @param after The table after the alter
+   * @return
+   */
+  public abstract AlterTableMessage buildAlterTableMessage(Table before, Table after);
+
+  /**
    * Factory method for DropTableMessage.
    * @param table The Table being dropped.
    * @return DropTableMessage instance.
@@ -144,6 +155,15 @@ public abstract class MessageFactory {
   public abstract AddPartitionMessage buildAddPartitionMessage(Table table, PartitionSpecProxy partitionSpec);
 
   /**
+   * Factory method for building AlterPartitionMessage
+   * @param before The partition before it was altered
+   * @param after The partition after it was altered
+   * @return a new AlterPartitionMessage
+   */
+  public abstract AlterPartitionMessage buildAlterPartitionMessage(Partition before,
+                                                                   Partition after);
+
+  /**
    * Factory method for DropPartitionMessage.
    * @param table The Table from which the partition is dropped.
    * @param partition The Partition being dropped.

Added: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterPartitionMessage.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterPartitionMessage.java?rev=1651923&view=auto
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterPartitionMessage.java (added)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterPartitionMessage.java Thu Jan 15 01:12:26 2015
@@ -0,0 +1,100 @@
+/**
+ * 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.hive.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hive.hcatalog.common.HCatConstants;
+import org.apache.hive.hcatalog.messaging.AlterPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterTableMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * JSON alter table message
+ */
+public class JSONAlterPartitionMessage extends AlterPartitionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<String> values;
+
+
+  public JSONAlterPartitionMessage(String server,
+                                   String servicePrincipal,
+                                   String db,
+                                   String table,
+                                   List<String> values,
+                                   Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    this.table = table;
+    this.timestamp = timestamp;
+    this.values = values;
+    checkValid();
+  }
+
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return db;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public List<String> getValues() {
+    return values;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+}

Added: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterTableMessage.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterTableMessage.java?rev=1651923&view=auto
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterTableMessage.java (added)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterTableMessage.java Thu Jan 15 01:12:26 2015
@@ -0,0 +1,87 @@
+/**
+ * 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.hive.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hive.hcatalog.common.HCatConstants;
+import org.apache.hive.hcatalog.messaging.AlterTableMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * JSON alter table message
+ */
+public class JSONAlterTableMessage extends AlterTableMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table;
+
+  @JsonProperty
+  Long timestamp;
+
+  public JSONAlterTableMessage(String server,
+                               String servicePrincipal,
+                               String db,
+                               String table,
+                               Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    this.table = table;
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return db;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+}

Modified: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java (original)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java Thu Jan 15 01:12:26 2015
@@ -19,7 +19,10 @@
 
 package org.apache.hive.hcatalog.messaging.json;
 
+import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.messaging.AddPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterTableMessage;
 import org.apache.hive.hcatalog.messaging.CreateDatabaseMessage;
 import org.apache.hive.hcatalog.messaging.CreateTableMessage;
 import org.apache.hive.hcatalog.messaging.DropDatabaseMessage;
@@ -71,6 +74,17 @@ public class JSONMessageDeserializer ext
   }
 
   @Override
+  public AlterTableMessage getAlterTableMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAlterTableMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct appropriate alter table type.",
+          exception);
+    }
+  }
+
+  @Override
   public DropTableMessage getDropTableMessage(String messageBody) {
     try {
       return mapper.readValue(messageBody, JSONDropTableMessage.class);
@@ -90,6 +104,15 @@ public class JSONMessageDeserializer ext
     }
   }
 
+  @Override
+  public AlterPartitionMessage getAlterPartitionMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAlterPartitionMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AlterPartitionMessage.", e);
+    }
+  }
+
   @Override
   public DropPartitionMessage getDropPartitionMessage(String messageBody) {
     try {

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=1651923&r1=1651922&r2=1651923&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 Thu Jan 15 01:12:26 2015
@@ -24,13 +24,12 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hive.hcatalog.messaging.AddPartitionMessage;
-// TODO, once HIVE-9175 is committed
-// import org.apache.hive.hcatalog.messaging.AlterTableMessage;
+import org.apache.hive.hcatalog.messaging.AlterPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterTableMessage;
 import org.apache.hive.hcatalog.messaging.CreateDatabaseMessage;
 import org.apache.hive.hcatalog.messaging.CreateTableMessage;
 import org.apache.hive.hcatalog.messaging.DropDatabaseMessage;
@@ -85,14 +84,11 @@ public class JSONMessageFactory extends
         table.getTableName(), now());
   }
 
-  // TODO Once HIVE-9175 is committed
-  /*
   @Override
   public AlterTableMessage buildAlterTableMessage(Table before, Table after) {
     return new JSONAlterTableMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, before.getDbName(),
         before.getTableName(), now());
   }
-  */
 
   @Override
   public DropTableMessage buildDropTableMessage(Table table) {
@@ -111,7 +107,13 @@ public class JSONMessageFactory extends
   @InterfaceStability.Evolving
   public AddPartitionMessage buildAddPartitionMessage(Table table, PartitionSpecProxy partitionSpec) {
     return new JSONAddPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, table.getDbName(),
-        table.getTableName(), getPartitionKeyValues(table, partitionSpec), System.currentTimeMillis()/1000);
+        table.getTableName(), getPartitionKeyValues(table, partitionSpec), now());
+  }
+
+  @Override
+  public AlterPartitionMessage buildAlterPartitionMessage(Partition before, Partition after) {
+    return new JSONAlterPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL,
+        before.getDbName(), before.getTableName(), before.getValues(), now());
   }
 
   @Override

Modified: hive/trunk/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java (original)
+++ hive/trunk/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java Thu Jan 15 01:12:26 2015
@@ -46,6 +46,8 @@ import org.apache.hive.hcatalog.common.H
 import org.apache.hive.hcatalog.mapreduce.HCatBaseTest;
 
 import org.apache.hive.hcatalog.messaging.AddPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterTableMessage;
 import org.apache.hive.hcatalog.messaging.CreateDatabaseMessage;
 import org.apache.hive.hcatalog.messaging.CreateTableMessage;
 import org.apache.hive.hcatalog.messaging.DropDatabaseMessage;
@@ -104,7 +106,9 @@ public class TestNotificationListener ex
         HCatConstants.HCAT_CREATE_DATABASE_EVENT,
         HCatConstants.HCAT_CREATE_TABLE_EVENT,
         HCatConstants.HCAT_ADD_PARTITION_EVENT,
+        HCatConstants.HCAT_ALTER_PARTITION_EVENT,
         HCatConstants.HCAT_DROP_PARTITION_EVENT,
+        HCatConstants.HCAT_ALTER_TABLE_EVENT,
         HCatConstants.HCAT_DROP_TABLE_EVENT,
         HCatConstants.HCAT_DROP_DATABASE_EVENT);
     Assert.assertEquals(expectedMessages, actualMessages);
@@ -120,7 +124,9 @@ public class TestNotificationListener ex
     kvs.put("b", "2011");
     client.markPartitionForEvent("mydb", "mytbl", kvs,
         PartitionEventType.LOAD_DONE);
+    driver.run("alter table mytbl partition (b='2011') set fileformat orc");
     driver.run("alter table mytbl drop partition(b='2011')");
+    driver.run("alter table mytbl add columns (c int comment 'this is an int', d decimal(3,2))");
     driver.run("drop table mytbl");
     driver.run("drop database mydb");
   }
@@ -170,6 +176,20 @@ public class TestNotificationListener ex
         Assert.assertEquals("mytbl", ((AddPartitionMessage) message2).getTable());
         Assert.assertEquals(1, ((AddPartitionMessage) message2).getPartitions().size());
         Assert.assertEquals("2011", ((AddPartitionMessage) message2).getPartitions().get(0).get("b"));
+      } else if (event.equals(HCatConstants.HCAT_ALTER_PARTITION_EVENT)) {
+        Assert.assertEquals("topic://hcat.mydb.mytbl", msg.getJMSDestination().toString());
+        // for alter partition events
+        AlterPartitionMessage message = deserializer.getAlterPartitionMessage(messageBody);
+        Assert.assertEquals("mytbl", message.getTable());
+        Assert.assertEquals("mydb", message.getDB());
+        Assert.assertEquals(1, message.getValues().size());
+        Assert.assertEquals("2011", message.getValues().get(0));
+        HCatEventMessage message2 = MessagingUtils.getMessage(msg);
+        Assert.assertTrue("Unexpected message-type.", message2 instanceof AlterPartitionMessage);
+        Assert.assertEquals("mydb", message2.getDB());
+        Assert.assertEquals("mytbl", ((AlterPartitionMessage) message2).getTable());
+        Assert.assertEquals(1, ((AlterPartitionMessage) message2).getValues().size());
+        Assert.assertEquals("2011", ((AlterPartitionMessage) message2).getValues().get(0));
       } else if (event.equals(HCatConstants.HCAT_DROP_PARTITION_EVENT)) {
 
         Assert.assertEquals("topic://hcat.mydb.mytbl", msg.getJMSDestination()
@@ -184,7 +204,8 @@ public class TestNotificationListener ex
         Assert.assertEquals("mydb", message2.getDB());
         Assert.assertEquals("mytbl", ((DropPartitionMessage) message2).getTable());
         Assert.assertEquals(1, ((DropPartitionMessage) message2).getPartitions().size());
-        Assert.assertEquals("2011", ((DropPartitionMessage) message2).getPartitions().get(0).get("b"));
+        Assert.assertEquals("2011", ((DropPartitionMessage) message2).getPartitions().get(0).get(
+            "b"));
       } else if (event.equals(HCatConstants.HCAT_DROP_TABLE_EVENT)) {
 
         Assert.assertEquals("topic://hcat.mydb", msg.getJMSDestination().toString());
@@ -199,11 +220,20 @@ public class TestNotificationListener ex
 
         Assert.assertEquals("topic://" + HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX, msg
             .getJMSDestination().toString());
-        DropDatabaseMessage message =  deserializer.getDropDatabaseMessage(messageBody);
+        DropDatabaseMessage message = deserializer.getDropDatabaseMessage(messageBody);
         Assert.assertEquals("mydb", message.getDB());
         HCatEventMessage message2 = MessagingUtils.getMessage(msg);
         Assert.assertTrue("Unexpected message-type.", message2 instanceof DropDatabaseMessage);
         Assert.assertEquals("mydb", message2.getDB());
+      } else if (event.equals(HCatConstants.HCAT_ALTER_TABLE_EVENT)) {
+        Assert.assertEquals("topic://hcat.mydb", msg.getJMSDestination().toString());
+        AlterTableMessage message = deserializer.getAlterTableMessage(messageBody);
+        Assert.assertEquals("mytbl", message.getTable());
+        Assert.assertEquals("mydb", message.getDB());
+        HCatEventMessage message2 = MessagingUtils.getMessage(msg);
+        Assert.assertTrue("Unexpected message-type.", message2 instanceof AlterTableMessage);
+        Assert.assertEquals("mydb", message2.getDB());
+        Assert.assertEquals("mytbl", ((AlterTableMessage) message2).getTable());
       } else if (event.equals(HCatConstants.HCAT_PARTITION_DONE_EVENT)) {
         // TODO: Fill in when PARTITION_DONE_EVENT is supported.
         Assert.assertTrue("Unexpected: HCAT_PARTITION_DONE_EVENT not supported (yet).", false);

Modified: hive/trunk/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java (original)
+++ hive/trunk/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java Thu Jan 15 01:12:26 2015
@@ -138,8 +138,6 @@ public class TestDbNotificationListener
         "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":\"mytable\",\"timestamp\":[0-9]+}"));
   }
 
-  // TODO After HIVE-9175 is committed
-  /*
   @Test
   public void alterTable() throws Exception {
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
@@ -147,12 +145,12 @@ public class TestDbNotificationListener
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
     StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
         serde, null, null, emptyParameters);
-    Table table = new Table("alttable", "default", "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    Table table = new Table("alttable", "default", "me", startTime, startTime, 0, sd,
+        new ArrayList<FieldSchema>(), emptyParameters, null, null, null);
     msClient.createTable(table);
 
-    table = new Table("alttable", "default", "me", startTime, startTime + 1, 0, sd, null,
-        emptyParameters, null, null, null);
+    table = new Table("alttable", "default", "me", startTime, startTime + 1, 0, sd,
+        new ArrayList<FieldSchema>(), emptyParameters, null, null, null);
     msClient.alter_table("default", "alttable", table);
 
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
@@ -168,7 +166,6 @@ public class TestDbNotificationListener
         "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":\"alttable\"," +
         "\"timestamp\":[0-9]+}"));
   }
-  */
 
   @Test
   public void dropTable() throws Exception {
@@ -228,6 +225,41 @@ public class TestDbNotificationListener
   }
 
   @Test
+  public void alterPartition() throws Exception {
+    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    cols.add(new FieldSchema("col1", "int", "nocomment"));
+    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+    partCols.add(new FieldSchema("ds", "string", ""));
+    SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
+        serde, null, null, emptyParameters);
+    Table table = new Table("alterparttable", "default", "me", startTime, startTime, 0, sd,
+        partCols, emptyParameters, null, null, null);
+    msClient.createTable(table);
+
+    Partition partition = new Partition(Arrays.asList("today"), "default", "alterparttable",
+        startTime, startTime, sd, emptyParameters);
+    msClient.add_partition(partition);
+
+    Partition newPart = new Partition(Arrays.asList("today"), "default", "alterparttable",
+        startTime, startTime + 1, sd, emptyParameters);
+    msClient.alter_partition("default", "alterparttable", newPart);
+
+    NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(3, rsp.getEventsSize());
+
+    NotificationEvent event = rsp.getEvents().get(2);
+    assertEquals(firstEventId + 3, event.getEventId());
+    assertTrue(event.getEventTime() >= startTime);
+    assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, event.getEventType());
+    assertEquals("default", event.getDbName());
+    assertEquals("alterparttable", event.getTableName());
+    assertTrue(event.getMessage().matches( "\\{\"eventType\":\"ALTER_PARTITION\",\"server\":\"\"," +
+        "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":\"alterparttable\"," +
+        "\"timestamp\":[0-9]+,\"values\":\\[\"today\"]}"));
+  }
+
+  @Test
   public void dropPartition() throws Exception {
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     cols.add(new FieldSchema("col1", "int", "nocomment"));

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=1651923&r1=1651922&r2=1651923&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 Thu Jan 15 01:12:26 2015
@@ -3125,9 +3125,14 @@ public class HiveMetaStore extends Thrif
 
         oldPart = alterHandler.alterPartition(getMS(), wh, db_name, tbl_name, part_vals, new_part);
 
+        // Only fetch the table if we actually have a listener
+        Table table = null;
         for (MetaStoreEventListener listener : listeners) {
+          if (table == null) {
+            table = getMS().getTable(db_name, tbl_name);
+          }
           AlterPartitionEvent alterPartitionEvent =
-              new AlterPartitionEvent(oldPart, new_part, true, this);
+              new AlterPartitionEvent(oldPart, new_part, table, true, this);
           alterPartitionEvent.setEnvironmentContext(envContext);
           listener.onAlterPartition(alterPartitionEvent);
         }
@@ -3178,6 +3183,8 @@ public class HiveMetaStore extends Thrif
         oldParts = alterHandler.alterPartitions(getMS(), wh, db_name, tbl_name, new_parts);
 
         Iterator<Partition> olditr = oldParts.iterator();
+        // Only fetch the table if we have a listener that needs it.
+        Table table = null;
         for (Partition tmpPart : new_parts) {
           Partition oldTmpPart = null;
           if (olditr.hasNext()) {
@@ -3187,8 +3194,11 @@ public class HiveMetaStore extends Thrif
             throw new InvalidOperationException("failed to alterpartitions");
           }
           for (MetaStoreEventListener listener : listeners) {
+            if (table == null) {
+              table = getMS().getTable(db_name, tbl_name);
+            }
             AlterPartitionEvent alterPartitionEvent =
-                new AlterPartitionEvent(oldTmpPart, tmpPart, true, this);
+                new AlterPartitionEvent(oldTmpPart, tmpPart, table, true, this);
             listener.onAlterPartition(alterPartitionEvent);
           }
         }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java?rev=1651923&r1=1651922&r2=1651923&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java Thu Jan 15 01:12:26 2015
@@ -20,17 +20,20 @@ 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;
 
 public class AlterPartitionEvent extends ListenerEvent {
 
   private final Partition oldPart;
   private final Partition newPart;
+  private final Table table;
 
-  public AlterPartitionEvent(Partition oldPart, Partition newPart,
+  public AlterPartitionEvent(Partition oldPart, Partition newPart, Table table,
       boolean status, HMSHandler handler) {
     super(status, handler);
     this.oldPart = oldPart;
     this.newPart = newPart;
+    this.table = table;
   }
 
   /**
@@ -47,4 +50,12 @@ public class AlterPartitionEvent extends
   public Partition getNewPartition() {
     return newPart;
   }
+
+  /**
+   * Get the table this partition is in
+   * @return
+   */
+  public Table getTable() {
+    return table;
+  }
 }