You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2017/11/02 16:23:03 UTC

[01/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Repository: hive
Updated Branches:
  refs/heads/master 10aa33072 -> c5a9673a0


http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
index 5cae281..50e4244 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
@@ -21,20 +21,32 @@ import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.collections.ListUtils;
 import org.apache.commons.lang.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.metastore.ColumnType;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Decimal;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregator;
 import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregatorFactory;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,9 +60,11 @@ import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
@@ -60,6 +74,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 public class MetaStoreUtils {
@@ -73,9 +88,20 @@ public class MetaStoreUtils {
       return val;
     }
   };
+  // Indicates a type was derived from the deserializer rather than Hive's metadata.
+  public static final String TYPE_FROM_DESERIALIZER = "<derived from deserializer>";
+
   private static final Charset ENCODING = StandardCharsets.UTF_8;
   private static final Logger LOG = LoggerFactory.getLogger(MetaStoreUtils.class);
 
+  // Right now we only support one special character '/'.
+  // More special characters can be added accordingly in the future.
+  // NOTE:
+  // If the following array is updated, please also be sure to update the
+  // configuration parameter documentation
+  // HIVE_SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES in HiveConf as well.
+  private static final char[] specialCharactersInTableNames = new char[] { '/' };
+
   /**
    * Catches exceptions that can't be handled and bundles them to MetaException
    *
@@ -139,8 +165,9 @@ public class MetaStoreUtils {
       return org.apache.commons.lang.StringUtils.defaultString(string);
     }
   };
+
   /**
-   * We have aneed to sanity-check the map before conversion from persisted objects to
+   * We have a need to sanity-check the map before conversion from persisted objects to
    * metadata thrift objects because null values in maps will cause a NPE if we send
    * across thrift. Pruning is appropriate for most cases except for databases such as
    * Oracle where Empty strings are stored as nulls, in which case we need to handle that.
@@ -359,4 +386,365 @@ public class MetaStoreUtils {
     }
     return colNames;
   }
+
+  /**
+   * validateName
+   *
+   * Checks the name conforms to our standars which are: "[a-zA-z_0-9]+". checks
+   * this is just characters and numbers and _
+   *
+   * @param name
+   *          the name to validate
+   * @param conf
+   *          hive configuration
+   * @return true or false depending on conformance
+   *              if it doesn't match the pattern.
+   */
+  public static boolean validateName(String name, Configuration conf) {
+    Pattern tpat = null;
+    String allowedCharacters = "\\w_";
+    if (conf != null
+        && MetastoreConf.getBoolVar(conf,
+        MetastoreConf.ConfVars.SUPPORT_SPECICAL_CHARACTERS_IN_TABLE_NAMES)) {
+      for (Character c : specialCharactersInTableNames) {
+        allowedCharacters += c;
+      }
+    }
+    tpat = Pattern.compile("[" + allowedCharacters + "]+");
+    Matcher m = tpat.matcher(name);
+    return m.matches();
+  }
+
+  /*
+   * At the Metadata level there are no restrictions on Column Names.
+   */
+  public static boolean validateColumnName(String name) {
+    return true;
+  }
+
+  static public String validateTblColumns(List<FieldSchema> cols) {
+    for (FieldSchema fieldSchema : cols) {
+      // skip this, as validateColumnName always returns true
+      /*
+      if (!validateColumnName(fieldSchema.getName())) {
+        return "name: " + fieldSchema.getName();
+      }
+      */
+      String typeError = validateColumnType(fieldSchema.getType());
+      if (typeError != null) {
+        return typeError;
+      }
+    }
+    return null;
+  }
+
+  private static String validateColumnType(String type) {
+    if (type.equals(TYPE_FROM_DESERIALIZER)) return null;
+    int last = 0;
+    boolean lastAlphaDigit = isValidTypeChar(type.charAt(last));
+    for (int i = 1; i <= type.length(); i++) {
+      if (i == type.length()
+          || isValidTypeChar(type.charAt(i)) != lastAlphaDigit) {
+        String token = type.substring(last, i);
+        last = i;
+        if (!ColumnType.AllTypes.contains(token)) {
+          return "type: " + type;
+        }
+        break;
+      }
+    }
+    return null;
+  }
+
+  private static boolean isValidTypeChar(char c) {
+    return Character.isLetterOrDigit(c) || c == '_';
+  }
+
+  /**
+   * Determines whether a table is an external table.
+   *
+   * @param table table of interest
+   *
+   * @return true if external
+   */
+  public static boolean isExternalTable(Table table) {
+    if (table == null) {
+      return false;
+    }
+    Map<String, String> params = table.getParameters();
+    if (params == null) {
+      return false;
+    }
+
+    return "TRUE".equalsIgnoreCase(params.get("EXTERNAL"));
+  }
+
+  // check if stats need to be (re)calculated
+  public static boolean requireCalStats(Configuration hiveConf, Partition oldPart,
+    Partition newPart, Table tbl, EnvironmentContext environmentContext) {
+
+    if (environmentContext != null
+        && environmentContext.isSetProperties()
+        && StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(
+            StatsSetupConst.DO_NOT_UPDATE_STATS))) {
+      return false;
+    }
+
+    if (MetaStoreUtils.isView(tbl)) {
+      return false;
+    }
+
+    if  (oldPart == null && newPart == null) {
+      return true;
+    }
+
+    // requires to calculate stats if new partition doesn't have it
+    if ((newPart == null) || (newPart.getParameters() == null)
+        || !containsAllFastStats(newPart.getParameters())) {
+      return true;
+    }
+
+    if (environmentContext != null && environmentContext.isSetProperties()) {
+      String statsType = environmentContext.getProperties().get(StatsSetupConst.STATS_GENERATED);
+      // no matter STATS_GENERATED is USER or TASK, all need to re-calculate the stats:
+      // USER: alter table .. update statistics
+      // TASK: from some sql operation which could collect and compute stats
+      if (StatsSetupConst.TASK.equals(statsType) || StatsSetupConst.USER.equals(statsType)) {
+        return true;
+      }
+    }
+
+    // requires to calculate stats if new and old have different fast stats
+    return !isFastStatsSame(oldPart, newPart);
+  }
+
+  public static boolean isView(Table table) {
+    if (table == null) {
+      return false;
+    }
+    return TableType.VIRTUAL_VIEW.toString().equals(table.getTableType());
+  }
+
+  /**
+   * @param partParams
+   * @return True if the passed Parameters Map contains values for all "Fast Stats".
+   */
+  private static boolean containsAllFastStats(Map<String, String> partParams) {
+    for (String stat : StatsSetupConst.fastStats) {
+      if (!partParams.containsKey(stat)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public static boolean isFastStatsSame(Partition oldPart, Partition newPart) {
+    // requires to calculate stats if new and old have different fast stats
+    if ((oldPart != null) && (oldPart.getParameters() != null)) {
+      for (String stat : StatsSetupConst.fastStats) {
+        if (oldPart.getParameters().containsKey(stat)) {
+          Long oldStat = Long.parseLong(oldPart.getParameters().get(stat));
+          Long newStat = Long.parseLong(newPart.getParameters().get(stat));
+          if (!oldStat.equals(newStat)) {
+            return false;
+          }
+        } else {
+          return false;
+        }
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public static boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh,
+                                             boolean madeDir, EnvironmentContext environmentContext) throws MetaException {
+    return updateTableStatsFast(db, tbl, wh, madeDir, false, environmentContext);
+  }
+
+  public static boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh,
+                                             boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
+    if (tbl.getPartitionKeysSize() == 0) {
+      // Update stats only when unpartitioned
+      FileStatus[] fileStatuses = wh.getFileStatusesForUnpartitionedTable(db, tbl);
+      return updateTableStatsFast(tbl, fileStatuses, madeDir, forceRecompute, environmentContext);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Updates the numFiles and totalSize parameters for the passed Table by querying
+   * the warehouse if the passed Table does not already have values for these parameters.
+   * @param tbl
+   * @param fileStatus
+   * @param newDir if true, the directory was just created and can be assumed to be empty
+   * @param forceRecompute Recompute stats even if the passed Table already has
+   * these parameters set
+   * @return true if the stats were updated, false otherwise
+   */
+  public static boolean updateTableStatsFast(Table tbl, FileStatus[] fileStatus, boolean newDir,
+                                             boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
+
+    Map<String,String> params = tbl.getParameters();
+
+    if ((params!=null) && params.containsKey(StatsSetupConst.DO_NOT_UPDATE_STATS)){
+      boolean doNotUpdateStats = Boolean.valueOf(params.get(StatsSetupConst.DO_NOT_UPDATE_STATS));
+      params.remove(StatsSetupConst.DO_NOT_UPDATE_STATS);
+      tbl.setParameters(params); // to make sure we remove this marker property
+      if (doNotUpdateStats){
+        return false;
+      }
+    }
+
+    boolean updated = false;
+    if (forceRecompute ||
+        params == null ||
+        !containsAllFastStats(params)) {
+      if (params == null) {
+        params = new HashMap<String,String>();
+      }
+      if (!newDir) {
+        // The table location already exists and may contain data.
+        // Let's try to populate those stats that don't require full scan.
+        LOG.info("Updating table stats fast for " + tbl.getTableName());
+        populateQuickStats(fileStatus, params);
+        LOG.info("Updated size of table " + tbl.getTableName() +" to "+ params.get(StatsSetupConst.TOTAL_SIZE));
+        if (environmentContext != null
+            && environmentContext.isSetProperties()
+            && StatsSetupConst.TASK.equals(environmentContext.getProperties().get(
+            StatsSetupConst.STATS_GENERATED))) {
+          StatsSetupConst.setBasicStatsState(params, StatsSetupConst.TRUE);
+        } else {
+          StatsSetupConst.setBasicStatsState(params, StatsSetupConst.FALSE);
+        }
+      }
+      tbl.setParameters(params);
+      updated = true;
+    }
+    return updated;
+  }
+
+  public static void populateQuickStats(FileStatus[] fileStatus, Map<String, String> params) {
+    int numFiles = 0;
+    long tableSize = 0L;
+    for (FileStatus status : fileStatus) {
+      // don't take directories into account for quick stats
+      if (!status.isDir()) {
+        tableSize += status.getLen();
+        numFiles += 1;
+      }
+    }
+    params.put(StatsSetupConst.NUM_FILES, Integer.toString(numFiles));
+    params.put(StatsSetupConst.TOTAL_SIZE, Long.toString(tableSize));
+  }
+
+  public static boolean areSameColumns(List<FieldSchema> oldCols, List<FieldSchema> newCols) {
+    return ListUtils.isEqualList(oldCols, newCols);
+  }
+
+  public static void updateBasicState(EnvironmentContext environmentContext, Map<String,String>
+      params) {
+    if (params == null) {
+      return;
+    }
+    if (environmentContext != null
+        && environmentContext.isSetProperties()
+        && StatsSetupConst.TASK.equals(environmentContext.getProperties().get(
+        StatsSetupConst.STATS_GENERATED))) {
+      StatsSetupConst.setBasicStatsState(params, StatsSetupConst.TRUE);
+    } else {
+      StatsSetupConst.setBasicStatsState(params, StatsSetupConst.FALSE);
+    }
+  }
+
+  public static boolean updatePartitionStatsFast(Partition part, Warehouse wh, EnvironmentContext environmentContext)
+      throws MetaException {
+    return updatePartitionStatsFast(part, wh, false, false, environmentContext);
+  }
+
+  public static boolean updatePartitionStatsFast(Partition part, Warehouse wh, boolean madeDir, EnvironmentContext environmentContext)
+      throws MetaException {
+    return updatePartitionStatsFast(part, wh, madeDir, false, environmentContext);
+  }
+
+  /**
+   * Updates the numFiles and totalSize parameters for the passed Partition by querying
+   *  the warehouse if the passed Partition does not already have values for these parameters.
+   * @param part
+   * @param wh
+   * @param madeDir if true, the directory was just created and can be assumed to be empty
+   * @param forceRecompute Recompute stats even if the passed Partition already has
+   * these parameters set
+   * @return true if the stats were updated, false otherwise
+   */
+  public static boolean updatePartitionStatsFast(Partition part, Warehouse wh,
+                                                 boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
+    return updatePartitionStatsFast(new PartitionSpecProxy.SimplePartitionWrapperIterator(part),
+        wh, madeDir, forceRecompute, environmentContext);
+  }
+  /**
+   * Updates the numFiles and totalSize parameters for the passed Partition by querying
+   *  the warehouse if the passed Partition does not already have values for these parameters.
+   * @param part
+   * @param wh
+   * @param madeDir if true, the directory was just created and can be assumed to be empty
+   * @param forceRecompute Recompute stats even if the passed Partition already has
+   * these parameters set
+   * @return true if the stats were updated, false otherwise
+   */
+  public static boolean updatePartitionStatsFast(PartitionSpecProxy.PartitionIterator part, Warehouse wh,
+                                                 boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
+    Map<String,String> params = part.getParameters();
+    boolean updated = false;
+    if (forceRecompute ||
+        params == null ||
+        !containsAllFastStats(params)) {
+      if (params == null) {
+        params = new HashMap<String,String>();
+      }
+      if (!madeDir) {
+        // The partition location already existed and may contain data. Lets try to
+        // populate those statistics that don't require a full scan of the data.
+        LOG.warn("Updating partition stats fast for: " + part.getTableName());
+        FileStatus[] fileStatus = wh.getFileStatusesForLocation(part.getLocation());
+        populateQuickStats(fileStatus, params);
+        LOG.warn("Updated size to " + params.get(StatsSetupConst.TOTAL_SIZE));
+        updateBasicState(environmentContext, params);
+      }
+      part.setParameters(params);
+      updated = true;
+    }
+    return updated;
+  }
+
+  /*
+     * This method is to check if the new column list includes all the old columns with same name and
+     * type. The column comment does not count.
+     */
+  public static boolean columnsIncludedByNameType(List<FieldSchema> oldCols,
+                                                  List<FieldSchema> newCols) {
+    if (oldCols.size() > newCols.size()) {
+      return false;
+    }
+
+    Map<String, String> columnNameTypePairMap = new HashMap<String, String>(newCols.size());
+    for (FieldSchema newCol : newCols) {
+      columnNameTypePairMap.put(newCol.getName().toLowerCase(), newCol.getType());
+    }
+    for (final FieldSchema oldCol : oldCols) {
+      if (!columnNameTypePairMap.containsKey(oldCol.getName())
+          || !columnNameTypePairMap.get(oldCol.getName()).equalsIgnoreCase(oldCol.getType())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /** Duplicates AcidUtils; used in a couple places in metastore. */
+  public static boolean isInsertOnlyTableParam(Map<String, String> params) {
+    String transactionalProp = params.get(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES);
+    return (transactionalProp != null && "insert_only".equalsIgnoreCase(transactionalProp));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
new file mode 100644
index 0000000..03ea7fc
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
@@ -0,0 +1,108 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.metastore.api.*;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+
+public class TestHiveAlterHandler {
+
+  @Test
+  public void testAlterTableAddColNotUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
+    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
+    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
+    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
+    FieldSchema col4 = new FieldSchema("col4", "string", "col4 comment");
+
+    StorageDescriptor oldSd = new StorageDescriptor();
+    oldSd.setCols(Arrays.asList(col1, col2, col3));
+    Table oldTable = new Table();
+    oldTable.setDbName("default");
+    oldTable.setTableName("test_table");
+    oldTable.setSd(oldSd);
+
+    StorageDescriptor newSd = new StorageDescriptor(oldSd);
+    newSd.setCols(Arrays.asList(col1, col2, col3, col4));
+    Table newTable = new Table(oldTable);
+    newTable.setSd(newSd);
+
+    RawStore msdb = Mockito.mock(RawStore.class);
+    Mockito.doThrow(new RuntimeException("shouldn't be called")).when(msdb).getTableColumnStatistics(
+        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3"));
+    HiveAlterHandler handler = new HiveAlterHandler();
+    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
+  }
+
+  @Test
+  public void testAlterTableDelColUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
+    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
+    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
+    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
+    FieldSchema col4 = new FieldSchema("col4", "string", "col4 comment");
+
+    StorageDescriptor oldSd = new StorageDescriptor();
+    oldSd.setCols(Arrays.asList(col1, col2, col3, col4));
+    Table oldTable = new Table();
+    oldTable.setDbName("default");
+    oldTable.setTableName("test_table");
+    oldTable.setSd(oldSd);
+
+    StorageDescriptor newSd = new StorageDescriptor(oldSd);
+    newSd.setCols(Arrays.asList(col1, col2, col3));
+    Table newTable = new Table(oldTable);
+    newTable.setSd(newSd);
+
+    RawStore msdb = Mockito.mock(RawStore.class);
+    HiveAlterHandler handler = new HiveAlterHandler();
+    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
+    Mockito.verify(msdb, Mockito.times(1)).getTableColumnStatistics(
+        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4")
+    );
+  }
+
+  @Test
+  public void testAlterTableChangePosNotUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
+    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
+    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
+    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
+    FieldSchema col4 = new FieldSchema("col4", "string", "col4 comment");
+
+    StorageDescriptor oldSd = new StorageDescriptor();
+    oldSd.setCols(Arrays.asList(col1, col2, col3, col4));
+    Table oldTable = new Table();
+    oldTable.setDbName("default");
+    oldTable.setTableName("test_table");
+    oldTable.setSd(oldSd);
+
+    StorageDescriptor newSd = new StorageDescriptor(oldSd);
+    newSd.setCols(Arrays.asList(col1, col4, col2, col3));
+    Table newTable = new Table(oldTable);
+    newTable.setSd(newSd);
+
+    RawStore msdb = Mockito.mock(RawStore.class);
+    Mockito.doThrow(new RuntimeException("shouldn't be called")).when(msdb).getTableColumnStatistics(
+        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4"));
+    HiveAlterHandler handler = new HiveAlterHandler();
+    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java
new file mode 100644
index 0000000..8d44bf8
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java
@@ -0,0 +1,108 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.util.concurrent.TimeUnit;
+
+import javax.jdo.JDOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestRetriesInRetryingHMSHandler {
+
+  private static Configuration conf;
+  private static final int RETRY_ATTEMPTS = 3;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.HMSHANDLERATTEMPTS, RETRY_ATTEMPTS);
+    MetastoreConf.setTimeVar(conf, ConfVars.HMSHANDLERINTERVAL, 10, TimeUnit.MILLISECONDS);
+    MetastoreConf.setBoolVar(conf, ConfVars.HMSHANDLERFORCERELOADCONF, false);
+  }
+
+  /*
+   * If the init method of HMSHandler throws exception for the first time
+   * while creating RetryingHMSHandler it should be retried
+   */
+  @Test
+  public void testRetryInit() throws MetaException {
+    IHMSHandler mockBaseHandler = Mockito.mock(IHMSHandler.class);
+    Mockito.when(mockBaseHandler.getConf()).thenReturn(conf);
+    Mockito
+    .doThrow(JDOException.class)
+    .doNothing()
+    .when(mockBaseHandler).init();
+    RetryingHMSHandler.getProxy(conf, mockBaseHandler, false);
+    Mockito.verify(mockBaseHandler, Mockito.times(2)).init();
+  }
+
+  /*
+   * init method in HMSHandler should not be retried if there are no exceptions
+   */
+  @Test
+  public void testNoRetryInit() throws MetaException {
+    IHMSHandler mockBaseHandler = Mockito.mock(IHMSHandler.class);
+    Mockito.when(mockBaseHandler.getConf()).thenReturn(conf);
+    Mockito.doNothing().when(mockBaseHandler).init();
+    RetryingHMSHandler.getProxy(conf, mockBaseHandler, false);
+    Mockito.verify(mockBaseHandler, Mockito.times(1)).init();
+  }
+
+  /*
+   * If the init method in HMSHandler throws exception all the times it should be retried until
+   * HiveConf.ConfVars.HMSHANDLERATTEMPTS is reached before giving up
+   */
+  @Test(expected = MetaException.class)
+  public void testRetriesLimit() throws MetaException {
+    IHMSHandler mockBaseHandler = Mockito.mock(IHMSHandler.class);
+    Mockito.when(mockBaseHandler.getConf()).thenReturn(conf);
+    Mockito.doThrow(JDOException.class).when(mockBaseHandler).init();
+    RetryingHMSHandler.getProxy(conf, mockBaseHandler, false);
+    Mockito.verify(mockBaseHandler, Mockito.times(RETRY_ATTEMPTS)).init();
+  }
+
+  /*
+   * Test retries when InvocationException wrapped in MetaException wrapped in JDOException
+   * is thrown
+   */
+  @Test
+  public void testWrappedMetaExceptionRetry() throws MetaException {
+    IHMSHandler mockBaseHandler = Mockito.mock(IHMSHandler.class);
+    Mockito.when(mockBaseHandler.getConf()).thenReturn(conf);
+    //JDOException wrapped in MetaException wrapped in InvocationException
+    MetaException me = new MetaException("Dummy exception");
+    me.initCause(new JDOException());
+    InvocationTargetException ex = new InvocationTargetException(me);
+    Mockito
+    .doThrow(me)
+    .doNothing()
+    .when(mockBaseHandler).init();
+    RetryingHMSHandler.getProxy(conf, mockBaseHandler, false);
+    Mockito.verify(mockBaseHandler, Mockito.times(2)).init();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java
new file mode 100644
index 0000000..2f98855
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java
@@ -0,0 +1,106 @@
+/*
+ * 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.messaging.json;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.json.JSONException;
+import org.junit.Test;
+import org.skyscreamer.jsonassert.JSONAssert;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+public class TestJSONMessageDeserializer {
+
+  public static class MyClass {
+    @JsonProperty
+    private int a;
+    @JsonProperty
+    private Map<String, String> map;
+    private long l;
+    private String shouldNotSerialize = "shouldNotSerialize";
+
+    //for jackson to instantiate
+    MyClass() {
+    }
+
+    MyClass(int a, Map<String, String> map, long l) {
+      this.a = a;
+      this.map = map;
+      this.l = l;
+    }
+
+    @JsonProperty
+    long getL() {
+      return l;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o)
+        return true;
+      if (o == null || getClass() != o.getClass())
+        return false;
+
+      MyClass myClass = (MyClass) o;
+
+      if (a != myClass.a)
+        return false;
+      if (l != myClass.l)
+        return false;
+      if (!map.equals(myClass.map))
+        return false;
+      return shouldNotSerialize.equals(myClass.shouldNotSerialize);
+    }
+
+    @Override
+    public int hashCode() {
+      int result = a;
+      result = 31 * result + map.hashCode();
+      result = 31 * result + (int) (l ^ (l >>> 32));
+      result = 31 * result + shouldNotSerialize.hashCode();
+      return result;
+    }
+  }
+
+  @Test
+  public void shouldNotSerializePropertiesNotAnnotated() throws IOException, JSONException {
+    MyClass obj = new MyClass(Integer.MAX_VALUE, new HashMap<String, String>() {{
+      put("a", "a");
+      put("b", "b");
+    }}, Long.MAX_VALUE);
+    String json = JSONMessageDeserializer.mapper.writeValueAsString(obj);
+    JSONAssert.assertEquals(
+        "{\"a\":2147483647,\"map\":{\"b\":\"b\",\"a\":\"a\"},\"l\":9223372036854775807}", json,
+        false);
+  }
+
+  @Test
+  public void shouldDeserializeJsonStringToObject() throws IOException {
+    String json = "{\"a\":47,\"map\":{\"a\":\"a\",\"b\":\"a value for b\"},\"l\":98}";
+    MyClass actual = JSONMessageDeserializer.mapper.readValue(json, MyClass.class);
+    MyClass expected = new MyClass(47, new HashMap<String, String>() {{
+      put("a", "a");
+      put("b", "a value for b");
+    }}, 98L);
+    assertEquals(expected, actual);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
new file mode 100644
index 0000000..32ad63a
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
@@ -0,0 +1,84 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestMetaStoreUtils {
+
+  @Test
+  public void testTrimMapNullsXform() throws Exception {
+    Map<String,String> m = new HashMap<>();
+    m.put("akey","aval");
+    m.put("blank","");
+    m.put("null",null);
+
+    Map<String,String> xformed = MetaStoreUtils.trimMapNulls(m,true);
+    assertEquals(3,xformed.size());
+    assert(xformed.containsKey("akey"));
+    assert(xformed.containsKey("blank"));
+    assert(xformed.containsKey("null"));
+    assertEquals("aval",xformed.get("akey"));
+    assertEquals("",xformed.get("blank"));
+    assertEquals("",xformed.get("null"));
+  }
+
+  @Test
+  public void testTrimMapNullsPrune() throws Exception {
+    Map<String,String> m = new HashMap<>();
+    m.put("akey","aval");
+    m.put("blank","");
+    m.put("null",null);
+
+    Map<String,String> pruned = MetaStoreUtils.trimMapNulls(m,false);
+    assertEquals(2,pruned.size());
+    assert(pruned.containsKey("akey"));
+    assert(pruned.containsKey("blank"));
+    assert(!pruned.containsKey("null"));
+    assertEquals("aval",pruned.get("akey"));
+    assertEquals("",pruned.get("blank"));
+    assert(!pruned.containsValue(null));
+  }
+
+  @Test
+  public void testcolumnsIncludedByNameType() {
+    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
+    FieldSchema col1a = new FieldSchema("col1", "string", "col1 but with a different comment");
+    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
+    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
+    Assert.assertTrue(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1), Arrays.asList(col1)));
+    Assert.assertTrue(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1), Arrays.asList(col1a)));
+    Assert.assertTrue(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col1, col2)));
+    Assert.assertTrue(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col2, col1)));
+    Assert.assertTrue(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col1, col2, col3)));
+    Assert.assertTrue(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col3, col2, col1)));
+    Assert.assertFalse(org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col1)));
+  }
+
+
+
+}


[12/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
HIVE-17812 Move remaining classes that HiveMetaStore depends on.  This closes #261.  (Alan Gates, reviewed by Vihang Karajgaonkar)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c5a9673a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c5a9673a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c5a9673a

Branch: refs/heads/master
Commit: c5a9673a014ad7362a3cbd8a53fe021ee7b7082f
Parents: 10aa330
Author: Alan Gates <ga...@hortonworks.com>
Authored: Thu Nov 2 09:21:25 2017 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Thu Nov 2 09:21:25 2017 -0700

----------------------------------------------------------------------
 .../hcatalog/listener/NotificationListener.java |  24 +-
 .../hive/hcatalog/streaming/TestStreaming.java  |  16 +-
 .../metastore/TestMetaStoreEventListener.java   |   4 +-
 .../hive/metastore/TestMetaStoreUtils.java      |  62 --
 ...DummyHiveMetastoreAuthorizationProvider.java |   4 +-
 .../security/InjectableDummyAuthenticator.java  |   4 +-
 .../hive/metastore/AcidEventListener.java       |  94 --
 .../hadoop/hive/metastore/AlterHandler.java     | 196 ----
 .../hive/metastore/FileMetadataManager.java     | 134 ---
 .../hive/metastore/HMSMetricsListener.java      |  90 --
 .../hadoop/hive/metastore/HiveAlterHandler.java | 900 -------------------
 .../hadoop/hive/metastore/HiveMetaStore.java    |  91 +-
 .../hive/metastore/HouseKeeperService.java      |  45 -
 .../hive/metastore/MetaStoreEventListener.java  | 227 -----
 .../MetaStoreEventListenerConstants.java        |  41 -
 .../hive/metastore/MetaStoreInitContext.java    |  27 -
 .../hive/metastore/MetaStoreInitListener.java   |  49 -
 .../metastore/MetaStoreListenerNotifier.java    | 266 ------
 .../metastore/MetaStorePreEventListener.java    |  57 --
 .../hadoop/hive/metastore/MetaStoreUtils.java   | 189 +---
 .../hive/metastore/RetryingHMSHandler.java      | 233 -----
 .../metastore/SessionPropertiesListener.java    |  45 -
 .../TransactionalMetaStoreEventListener.java    |  39 -
 .../TransactionalValidationListener.java        | 359 --------
 .../metastore/events/AddForeignKeyEvent.java    |  37 -
 .../hive/metastore/events/AddIndexEvent.java    |  40 -
 .../events/AddNotNullConstraintEvent.java       |  37 -
 .../metastore/events/AddPartitionEvent.java     |  82 --
 .../metastore/events/AddPrimaryKeyEvent.java    |  38 -
 .../events/AddUniqueConstraintEvent.java        |  37 -
 .../hive/metastore/events/AlterIndexEvent.java  |  47 -
 .../metastore/events/AlterPartitionEvent.java   |  75 --
 .../hive/metastore/events/AlterTableEvent.java  |  62 --
 .../metastore/events/ConfigChangeEvent.java     |  52 --
 .../metastore/events/CreateDatabaseEvent.java   |  43 -
 .../metastore/events/CreateFunctionEvent.java   |  43 -
 .../hive/metastore/events/CreateTableEvent.java |  43 -
 .../metastore/events/DropConstraintEvent.java   |  47 -
 .../metastore/events/DropDatabaseEvent.java     |  43 -
 .../metastore/events/DropFunctionEvent.java     |  43 -
 .../hive/metastore/events/DropIndexEvent.java   |  40 -
 .../metastore/events/DropPartitionEvent.java    |  70 --
 .../hive/metastore/events/DropTableEvent.java   |  54 --
 .../hive/metastore/events/EventCleanerTask.java |  52 --
 .../hive/metastore/events/InsertEvent.java      | 114 ---
 .../hive/metastore/events/ListenerEvent.java    | 178 ----
 .../events/LoadPartitionDoneEvent.java          |  57 --
 .../hive/metastore/events/PreAddIndexEvent.java |  41 -
 .../metastore/events/PreAddPartitionEvent.java  |  79 --
 .../metastore/events/PreAlterDatabaseEvent.java |  47 -
 .../metastore/events/PreAlterIndexEvent.java    |  47 -
 .../events/PreAlterPartitionEvent.java          |  65 --
 .../metastore/events/PreAlterTableEvent.java    |  53 --
 .../events/PreAuthorizationCallEvent.java       |  33 -
 .../events/PreCreateDatabaseEvent.java          |  43 -
 .../metastore/events/PreCreateTableEvent.java   |  43 -
 .../metastore/events/PreDropDatabaseEvent.java  |  43 -
 .../metastore/events/PreDropIndexEvent.java     |  40 -
 .../metastore/events/PreDropPartitionEvent.java |  67 --
 .../metastore/events/PreDropTableEvent.java     |  55 --
 .../hive/metastore/events/PreEventContext.java  |  74 --
 .../events/PreLoadPartitionDoneEvent.java       |  58 --
 .../metastore/events/PreReadDatabaseEvent.java  |  46 -
 .../metastore/events/PreReadTableEvent.java     |  47 -
 .../messaging/AddForeignKeyMessage.java         |  36 -
 .../messaging/AddNotNullConstraintMessage.java  |  36 -
 .../messaging/AddPartitionMessage.java          |  68 --
 .../messaging/AddPrimaryKeyMessage.java         |  35 -
 .../messaging/AddUniqueConstraintMessage.java   |  36 -
 .../metastore/messaging/AlterIndexMessage.java  |  33 -
 .../messaging/AlterPartitionMessage.java        |  69 --
 .../metastore/messaging/AlterTableMessage.java  |  58 --
 .../messaging/CreateDatabaseMessage.java        |  28 -
 .../messaging/CreateFunctionMessage.java        |  46 -
 .../metastore/messaging/CreateIndexMessage.java |  46 -
 .../metastore/messaging/CreateTableMessage.java |  53 --
 .../messaging/DropConstraintMessage.java        |  29 -
 .../messaging/DropDatabaseMessage.java          |  27 -
 .../messaging/DropFunctionMessage.java          |  38 -
 .../metastore/messaging/DropIndexMessage.java   |  46 -
 .../messaging/DropPartitionMessage.java         |  49 -
 .../metastore/messaging/DropTableMessage.java   |  46 -
 .../hive/metastore/messaging/EventMessage.java  | 111 ---
 .../hive/metastore/messaging/InsertMessage.java |  75 --
 .../messaging/MessageDeserializer.java          | 179 ----
 .../metastore/messaging/MessageFactory.java     | 291 ------
 .../metastore/messaging/PartitionFiles.java     |  53 --
 .../json/JSONAddForeignKeyMessage.java          | 101 ---
 .../json/JSONAddNotNullConstraintMessage.java   |  96 --
 .../messaging/json/JSONAddPartitionMessage.java | 171 ----
 .../json/JSONAddPrimaryKeyMessage.java          | 101 ---
 .../json/JSONAddUniqueConstraintMessage.java    |  98 --
 .../messaging/json/JSONAlterIndexMessage.java   |  98 --
 .../json/JSONAlterPartitionMessage.java         | 150 ----
 .../messaging/json/JSONAlterTableMessage.java   | 123 ---
 .../json/JSONCreateDatabaseMessage.java         |  71 --
 .../json/JSONCreateFunctionMessage.java         |  86 --
 .../messaging/json/JSONCreateIndexMessage.java  |  87 --
 .../messaging/json/JSONCreateTableMessage.java  | 130 ---
 .../json/JSONDropConstraintMessage.java         |  90 --
 .../messaging/json/JSONDropDatabaseMessage.java |  71 --
 .../messaging/json/JSONDropFunctionMessage.java |  79 --
 .../messaging/json/JSONDropIndexMessage.java    |  91 --
 .../json/JSONDropPartitionMessage.java          | 130 ---
 .../messaging/json/JSONDropTableMessage.java    | 116 ---
 .../messaging/json/JSONInsertMessage.java       | 144 ---
 .../messaging/json/JSONMessageDeserializer.java | 243 -----
 .../messaging/json/JSONMessageFactory.java      | 354 --------
 .../hive/metastore/TestHiveAlterHandler.java    | 108 ---
 .../hive/metastore/TestMetaStoreUtils.java      |  43 -
 .../TestRetriesInRetryingHMSHandler.java        | 109 ---
 .../json/TestJSONMessageDeserializer.java       | 106 ---
 .../hive/ql/parse/MetaDataExportListener.java   |  16 +-
 .../HadoopDefaultMetastoreAuthenticator.java    |   4 +-
 .../HiveMetastoreAuthenticationProvider.java    |   4 +-
 ...faultHiveMetastoreAuthorizationProvider.java |   4 +-
 .../HiveAuthorizationProviderBase.java          |   6 +-
 .../HiveMetastoreAuthorizationProvider.java     |   4 +-
 .../MetaStoreAuthzAPIAuthorizerEmbedOnly.java   |   4 +-
 .../StorageBasedAuthorizationProvider.java      |   4 +-
 .../ql/txn/AcidCompactionHistoryService.java    |  82 --
 .../hive/ql/txn/AcidHouseKeeperService.java     |  82 --
 .../hadoop/hive/ql/txn/AcidWriteSetService.java |  78 --
 .../txn/compactor/HouseKeeperServiceBase.java   |  96 --
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |  16 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |  30 +-
 .../hive/ql/lockmgr/TestDbTxnManager.java       |  25 +-
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |  12 +-
 .../org/apache/hadoop/hive/io/HdfsUtils.java    |  11 -
 standalone-metastore/pom.xml                    |   6 +
 .../hive/metastore/api/ClientCapability.java    |   8 +-
 .../hive/metastore/AcidEventListener.java       |  95 ++
 .../hadoop/hive/metastore/AlterHandler.java     | 195 ++++
 .../hive/metastore/FileMetadataManager.java     | 124 +++
 .../hive/metastore/HMSMetricsListener.java      |  90 ++
 .../hadoop/hive/metastore/HiveAlterHandler.java | 897 ++++++++++++++++++
 .../hadoop/hive/metastore/IHMSHandler.java      |  59 ++
 .../hive/metastore/MetaStoreEventListener.java  | 227 +++++
 .../MetaStoreEventListenerConstants.java        |  41 +
 .../hive/metastore/MetaStoreInitContext.java    |  27 +
 .../hive/metastore/MetaStoreInitListener.java   |  49 +
 .../metastore/MetaStoreListenerNotifier.java    | 266 ++++++
 .../metastore/MetaStorePreEventListener.java    |  57 ++
 .../hive/metastore/RetryingHMSHandler.java      | 232 +++++
 .../metastore/SessionPropertiesListener.java    |  46 +
 .../TransactionalMetaStoreEventListener.java    |  39 +
 .../TransactionalValidationListener.java        | 360 ++++++++
 .../hive/metastore/conf/MetastoreConf.java      |  16 +
 .../metastore/events/AddForeignKeyEvent.java    |  41 +
 .../hive/metastore/events/AddIndexEvent.java    |  40 +
 .../events/AddNotNullConstraintEvent.java       |  42 +
 .../metastore/events/AddPartitionEvent.java     |  84 ++
 .../metastore/events/AddPrimaryKeyEvent.java    |  42 +
 .../events/AddUniqueConstraintEvent.java        |  42 +
 .../hive/metastore/events/AlterIndexEvent.java  |  46 +
 .../metastore/events/AlterPartitionEvent.java   |  75 ++
 .../hive/metastore/events/AlterTableEvent.java  |  63 ++
 .../metastore/events/ConfigChangeEvent.java     |  52 ++
 .../metastore/events/CreateDatabaseEvent.java   |  43 +
 .../metastore/events/CreateFunctionEvent.java   |  43 +
 .../hive/metastore/events/CreateTableEvent.java |  43 +
 .../metastore/events/DropConstraintEvent.java   |  51 ++
 .../metastore/events/DropDatabaseEvent.java     |  43 +
 .../metastore/events/DropFunctionEvent.java     |  43 +
 .../hive/metastore/events/DropIndexEvent.java   |  40 +
 .../metastore/events/DropPartitionEvent.java    |  70 ++
 .../hive/metastore/events/DropTableEvent.java   |  54 ++
 .../hive/metastore/events/EventCleanerTask.java |  52 ++
 .../hive/metastore/events/InsertEvent.java      | 129 +++
 .../hive/metastore/events/ListenerEvent.java    | 178 ++++
 .../events/LoadPartitionDoneEvent.java          |  57 ++
 .../hive/metastore/events/PreAddIndexEvent.java |  41 +
 .../metastore/events/PreAddPartitionEvent.java  |  79 ++
 .../metastore/events/PreAlterDatabaseEvent.java |  47 +
 .../metastore/events/PreAlterIndexEvent.java    |  47 +
 .../events/PreAlterPartitionEvent.java          |  65 ++
 .../metastore/events/PreAlterTableEvent.java    |  53 ++
 .../events/PreAuthorizationCallEvent.java       |  33 +
 .../events/PreCreateDatabaseEvent.java          |  43 +
 .../metastore/events/PreCreateTableEvent.java   |  43 +
 .../metastore/events/PreDropDatabaseEvent.java  |  43 +
 .../metastore/events/PreDropIndexEvent.java     |  40 +
 .../metastore/events/PreDropPartitionEvent.java |  67 ++
 .../metastore/events/PreDropTableEvent.java     |  55 ++
 .../hive/metastore/events/PreEventContext.java  |  73 ++
 .../events/PreLoadPartitionDoneEvent.java       |  58 ++
 .../metastore/events/PreReadDatabaseEvent.java  |  46 +
 .../metastore/events/PreReadTableEvent.java     |  47 +
 .../messaging/AddForeignKeyMessage.java         |  36 +
 .../messaging/AddNotNullConstraintMessage.java  |  36 +
 .../messaging/AddPartitionMessage.java          |  68 ++
 .../messaging/AddPrimaryKeyMessage.java         |  35 +
 .../messaging/AddUniqueConstraintMessage.java   |  36 +
 .../metastore/messaging/AlterIndexMessage.java  |  33 +
 .../messaging/AlterPartitionMessage.java        |  69 ++
 .../metastore/messaging/AlterTableMessage.java  |  58 ++
 .../messaging/CreateDatabaseMessage.java        |  28 +
 .../messaging/CreateFunctionMessage.java        |  46 +
 .../metastore/messaging/CreateIndexMessage.java |  46 +
 .../metastore/messaging/CreateTableMessage.java |  53 ++
 .../messaging/DropConstraintMessage.java        |  29 +
 .../messaging/DropDatabaseMessage.java          |  27 +
 .../messaging/DropFunctionMessage.java          |  38 +
 .../metastore/messaging/DropIndexMessage.java   |  46 +
 .../messaging/DropPartitionMessage.java         |  49 +
 .../metastore/messaging/DropTableMessage.java   |  46 +
 .../hive/metastore/messaging/EventMessage.java  | 111 +++
 .../hive/metastore/messaging/InsertMessage.java |  75 ++
 .../messaging/MessageDeserializer.java          | 179 ++++
 .../metastore/messaging/MessageFactory.java     | 289 ++++++
 .../metastore/messaging/PartitionFiles.java     |  53 ++
 .../json/JSONAddForeignKeyMessage.java          | 101 +++
 .../json/JSONAddNotNullConstraintMessage.java   |  96 ++
 .../messaging/json/JSONAddPartitionMessage.java | 171 ++++
 .../json/JSONAddPrimaryKeyMessage.java          | 101 +++
 .../json/JSONAddUniqueConstraintMessage.java    |  98 ++
 .../messaging/json/JSONAlterIndexMessage.java   |  98 ++
 .../json/JSONAlterPartitionMessage.java         | 148 +++
 .../messaging/json/JSONAlterTableMessage.java   | 123 +++
 .../json/JSONCreateDatabaseMessage.java         |  71 ++
 .../json/JSONCreateFunctionMessage.java         |  86 ++
 .../messaging/json/JSONCreateIndexMessage.java  |  87 ++
 .../messaging/json/JSONCreateTableMessage.java  | 130 +++
 .../json/JSONDropConstraintMessage.java         |  90 ++
 .../messaging/json/JSONDropDatabaseMessage.java |  71 ++
 .../messaging/json/JSONDropFunctionMessage.java |  78 ++
 .../messaging/json/JSONDropIndexMessage.java    |  90 ++
 .../json/JSONDropPartitionMessage.java          | 130 +++
 .../messaging/json/JSONDropTableMessage.java    | 116 +++
 .../messaging/json/JSONInsertMessage.java       | 144 +++
 .../messaging/json/JSONMessageDeserializer.java | 243 +++++
 .../messaging/json/JSONMessageFactory.java      | 354 ++++++++
 .../txn/AcidCompactionHistoryService.java       |  62 ++
 .../metastore/txn/AcidHouseKeeperService.java   |  63 ++
 .../hive/metastore/txn/AcidWriteSetService.java |  61 ++
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   6 +
 .../hadoop/hive/metastore/utils/FileUtils.java  |  16 +
 .../hadoop/hive/metastore/utils/HdfsUtils.java  |  21 +
 .../hadoop/hive/metastore/utils/JavaUtils.java  |  14 +
 .../hive/metastore/utils/MetaStoreUtils.java    | 390 +++++++-
 .../hive/metastore/TestHiveAlterHandler.java    | 108 +++
 .../TestRetriesInRetryingHMSHandler.java        | 108 +++
 .../json/TestJSONMessageDeserializer.java       | 106 +++
 .../metastore/utils/TestMetaStoreUtils.java     |  84 ++
 244 files changed, 10351 insertions(+), 10260 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
index f7c2f44..66349c5 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
@@ -34,12 +34,9 @@ import javax.naming.InitialContext;
 import javax.naming.NamingException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
@@ -54,6 +51,7 @@ import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage;
 import org.apache.hive.hcatalog.messaging.MessageFactory;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -182,7 +180,7 @@ public class NotificationListener extends MetaStoreEventListener {
     // by listening on a topic named "HCAT" and message selector string
     // as "HCAT_EVENT = HCAT_ADD_DATABASE"
     if (dbEvent.getStatus()) {
-      String topicName = getTopicPrefix(dbEvent.getHandler().getHiveConf());
+      String topicName = getTopicPrefix(dbEvent.getIHMSHandler().getConf());
       send(messageFactory.buildCreateDatabaseMessage(dbEvent.getDatabase()), topicName);
     }
   }
@@ -193,7 +191,7 @@ public class NotificationListener extends MetaStoreEventListener {
     // by listening on a topic named "HCAT" and message selector string
     // as "HCAT_EVENT = HCAT_DROP_DATABASE"
     if (dbEvent.getStatus()) {
-      String topicName = getTopicPrefix(dbEvent.getHandler().getHiveConf());
+      String topicName = getTopicPrefix(dbEvent.getIHMSHandler().getConf());
       send(messageFactory.buildDropDatabaseMessage(dbEvent.getDatabase()), topicName);
     }
   }
@@ -205,8 +203,8 @@ public class NotificationListener extends MetaStoreEventListener {
     // as "HCAT_EVENT = HCAT_ADD_TABLE"
     if (tableEvent.getStatus()) {
       Table tbl = tableEvent.getTable();
-      HMSHandler handler = tableEvent.getHandler();
-      HiveConf conf = handler.getHiveConf();
+      IHMSHandler handler = tableEvent.getIHMSHandler();
+      Configuration conf = handler.getConf();
       Table newTbl;
       try {
         newTbl = handler.get_table_core(tbl.getDbName(), tbl.getTableName())
@@ -216,11 +214,7 @@ public class NotificationListener extends MetaStoreEventListener {
           getTopicPrefix(conf) + "." + newTbl.getDbName().toLowerCase() + "."
             + newTbl.getTableName().toLowerCase());
         handler.alter_table(newTbl.getDbName(), newTbl.getTableName(), newTbl);
-      } catch (InvalidOperationException e) {
-        MetaException me = new MetaException(e.toString());
-        me.initCause(e);
-        throw me;
-      } catch (NoSuchObjectException e) {
+      } catch (TException e) {
         MetaException me = new MetaException(e.toString());
         me.initCause(e);
         throw me;
@@ -258,7 +252,7 @@ public class NotificationListener extends MetaStoreEventListener {
       }
       // 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()) + "." +
+      String topicName = getTopicPrefix(tableEvent.getIHMSHandler().getConf()) + "." +
           after.getDbName().toLowerCase();
       send(messageFactory.buildAlterTableMessage(before, after), topicName);
     }
@@ -288,7 +282,7 @@ public class NotificationListener extends MetaStoreEventListener {
       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();
+      String topicName = getTopicPrefix(tableEvent.getIHMSHandler().getConf()) + "." + table.getDbName().toLowerCase();
       send(messageFactory.buildDropTableMessage(table), topicName);
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
index 3b5066f..49aad39 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnInfo;
 import org.apache.hadoop.hive.metastore.api.TxnState;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
@@ -77,7 +78,6 @@ import org.apache.hadoop.hive.ql.io.orc.Reader;
 import org.apache.hadoop.hive.ql.io.orc.RecordReader;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.ql.txn.compactor.Worker;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
@@ -740,11 +740,8 @@ public class TestStreaming {
     //ensure txn timesout
     conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1, TimeUnit.MILLISECONDS);
     AcidHouseKeeperService houseKeeperService = new AcidHouseKeeperService();
-    houseKeeperService.start(conf);
-    while(houseKeeperService.getIsAliveCounter() <= Integer.MIN_VALUE) {
-      Thread.sleep(100);//make sure it has run at least once
-    }
-    houseKeeperService.stop();
+    houseKeeperService.setConf(conf);
+    houseKeeperService.run();
     try {
       //should fail because the TransactionBatch timed out
       txnBatch.commit();
@@ -757,12 +754,7 @@ public class TestStreaming {
     txnBatch.beginNextTransaction();
     txnBatch.commit();
     txnBatch.beginNextTransaction();
-    int lastCount = houseKeeperService.getIsAliveCounter();
-    houseKeeperService.start(conf);
-    while(houseKeeperService.getIsAliveCounter() <= lastCount) {
-      Thread.sleep(100);//make sure it has run at least once
-    }
-    houseKeeperService.stop();
+    houseKeeperService.run();
     try {
       //should fail because the TransactionBatch timed out
       txnBatch.commit();

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
index 42e476a..4df2f86 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
@@ -509,13 +509,13 @@ public class TestMetaStoreEventListener extends TestCase {
     closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
     ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
     int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
-    HiveMetaStore.HMSHandler beforeHandler = event.getHandler();
+    IHMSHandler beforeHandler = event.getIHMSHandler();
     closingClient.close();
 
     Thread.sleep(5 * 1000);
     event = (ConfigChangeEvent) DummyListener.getLastEvent();
     int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
-    HiveMetaStore.HMSHandler afterHandler = event.getHandler();
+    IHMSHandler afterHandler = event.getIHMSHandler();
     // Meta-conf cleanup should trigger an event to listener
     assertNotSame(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
     // Both the handlers should be same

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java
deleted file mode 100644
index c4dd97e..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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;
-
-import junit.framework.TestCase;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class TestMetaStoreUtils extends TestCase {
-
-  public void testTrimMapNullsXform() throws Exception {
-    Map<String,String> m = new HashMap<String,String>();
-    m.put("akey","aval");
-    m.put("blank","");
-    m.put("null",null);
-
-    Map<String,String> xformed = MetaStoreUtils.trimMapNulls(m,true);
-    assertEquals(3,xformed.size());
-    assert(xformed.containsKey("akey"));
-    assert(xformed.containsKey("blank"));
-    assert(xformed.containsKey("null"));
-    assertEquals("aval",xformed.get("akey"));
-    assertEquals("",xformed.get("blank"));
-    assertEquals("",xformed.get("null"));
-  }
-
-  public void testTrimMapNullsPrune() throws Exception {
-    Map<String,String> m = new HashMap<String,String>();
-    m.put("akey","aval");
-    m.put("blank","");
-    m.put("null",null);
-
-    Map<String,String> pruned = MetaStoreUtils.trimMapNulls(m,false);
-    assertEquals(2,pruned.size());
-    assert(pruned.containsKey("akey"));
-    assert(pruned.containsKey("blank"));
-    assert(!pruned.containsKey("null"));
-    assertEquals("aval",pruned.get("akey"));
-    assertEquals("",pruned.get("blank"));
-    assert(!pruned.containsValue(null));
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java
index 93637d4..6662fcd 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -201,7 +201,7 @@ public class DummyHiveMetastoreAuthorizationProvider implements HiveMetastoreAut
   }
 
   @Override
-  public void setMetaStoreHandler(HMSHandler handler) {
+  public void setMetaStoreHandler(IHMSHandler handler) {
     debugLog("DHMAP.setMetaStoreHandler");
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
index 322834e..d4cf026 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.security;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -91,7 +91,7 @@ public class InjectableDummyAuthenticator implements HiveMetastoreAuthentication
   }
 
   @Override
-  public void setMetaStoreHandler(HMSHandler handler) {
+  public void setMetaStoreHandler(IHMSHandler handler) {
     hmap.setMetaStoreHandler(handler);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/AcidEventListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
deleted file mode 100644
index 71ad916..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.HiveObjectType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.DropTableEvent;
-import org.apache.hadoop.hive.metastore.txn.TxnStore;
-import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-
-
-/**
- * It handles cleanup of dropped partition/table/database in ACID related metastore tables
- */
-public class AcidEventListener extends MetaStoreEventListener {
-
-  private TxnStore txnHandler;
-  private HiveConf hiveConf;
-
-  public AcidEventListener(Configuration configuration) {
-    super(configuration);
-    hiveConf = (HiveConf) configuration;
-  }
-
-  @Override
-  public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
-    // We can loop thru all the tables to check if they are ACID first and then perform cleanup,
-    // but it's more efficient to unconditionally perform cleanup for the database, especially
-    // when there are a lot of tables
-    txnHandler = getTxnHandler();
-    txnHandler.cleanupRecords(HiveObjectType.DATABASE, dbEvent.getDatabase(), null, null);
-  }
-
-  @Override
-  public void onDropTable(DropTableEvent tableEvent)  throws MetaException {
-    if (TxnUtils.isAcidTable(tableEvent.getTable())) {
-      txnHandler = getTxnHandler();
-      txnHandler.cleanupRecords(HiveObjectType.TABLE, null, tableEvent.getTable(), null);
-    }
-  }
-
-  @Override
-  public void onDropPartition(DropPartitionEvent partitionEvent)  throws MetaException {
-    if (TxnUtils.isAcidTable(partitionEvent.getTable())) {
-      txnHandler = getTxnHandler();
-      txnHandler.cleanupRecords(HiveObjectType.PARTITION, null, partitionEvent.getTable(),
-          partitionEvent.getPartitionIterator());
-    }
-  }
-
-  private TxnStore getTxnHandler() {
-    boolean hackOn = HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_IN_TEST) ||
-        HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_IN_TEZ_TEST);
-    String origTxnMgr = null;
-    boolean origConcurrency = false;
-
-    // Since TxnUtils.getTxnStore calls TxnHandler.setConf -> checkQFileTestHack -> TxnDbUtil.setConfValues,
-    // which may change the values of below two entries, we need to avoid pulluting the original values
-    if (hackOn) {
-      origTxnMgr = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER);
-      origConcurrency = hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
-    }
-
-    txnHandler = TxnUtils.getTxnStore(hiveConf);
-
-    // Set them back
-    if (hackOn) {
-      hiveConf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, origTxnMgr);
-      hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, origConcurrency);
-    }
-
-    return txnHandler;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/AlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/AlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/AlterHandler.java
deleted file mode 100644
index a3d322f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/AlterHandler.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/**
- * 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;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-
-/**
- * Interface for Alter Table and Alter Partition code
- */
-public interface AlterHandler extends Configurable {
-
-  /**
-   * @deprecated As of release 2.2.0. Replaced by {@link #alterTable(RawStore, Warehouse, String,
-   * String, Table, EnvironmentContext, HMSHandler)}
-   *
-   * handles alter table, the changes could be cascaded to partitions if applicable
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   *          Hive Warehouse where table data is stored
-   * @param dbname
-   *          database of the table being altered
-   * @param name
-   *          original name of the table being altered. same as
-   *          <i>newTable.tableName</i> if alter op is not a rename.
-   * @param newTable
-   *          new table object
-   * @throws InvalidOperationException
-   *           thrown if the newTable object is invalid
-   * @throws MetaException
-   *           thrown if there is any other error
-   */
-  @Deprecated
-  void alterTable(RawStore msdb, Warehouse wh, String dbname,
-    String name, Table newTable, EnvironmentContext envContext)
-      throws InvalidOperationException, MetaException;
-
-  /**
-   * handles alter table, the changes could be cascaded to partitions if applicable
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   *          Hive Warehouse where table data is stored
-   * @param dbname
-   *          database of the table being altered
-   * @param name
-   *          original name of the table being altered. same as
-   *          <i>newTable.tableName</i> if alter op is not a rename.
-   * @param newTable
-   *          new table object
-   * @param handler
-   *          HMSHandle object (required to log event notification)
-   * @throws InvalidOperationException
-   *           thrown if the newTable object is invalid
-   * @throws MetaException
-   *           thrown if there is any other error
-   */
-  void alterTable(RawStore msdb, Warehouse wh, String dbname,
-      String name, Table newTable, EnvironmentContext envContext,
-      HMSHandler handler) throws InvalidOperationException, MetaException;
-
-  /**
-   * @deprecated As of release 2.2.0.  Replaced by {@link #alterPartition(RawStore, Warehouse, String,
-   * String, List, Partition, EnvironmentContext, HMSHandler)}
-   *
-   * handles alter partition
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param part_vals
-   *          original values of the partition being altered
-   * @param new_part
-   *          new partition object
-   * @return the altered partition
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  @Deprecated
-  Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<String> part_vals, final Partition new_part,
-    EnvironmentContext environmentContext)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
-
-  /**
-   * handles alter partition
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param part_vals
-   *          original values of the partition being altered
-   * @param new_part
-   *          new partition object
-   * @param handler
-   *          HMSHandle object (required to log event notification)
-   * @return the altered partition
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<String> part_vals, final Partition new_part, EnvironmentContext environmentContext,
-    HMSHandler handler)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
-
-  /**
-   * @deprecated As of release 2.2.0. Replaced by {@link #alterPartitions(RawStore, Warehouse, String,
-   * String, List, EnvironmentContext, HMSHandler)}
-   *
-   * handles alter partitions
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param new_parts
-   *          new partition list
-   * @return the altered partition list
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  @Deprecated
-  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh,
-    final String dbname, final String name, final List<Partition> new_parts,
-    EnvironmentContext environmentContext)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
-
-  /**
-   * handles alter partitions
-   *
-   * @param msdb
-   *          object to get metadata
-   * @param wh
-   * @param dbname
-   *          database of the partition being altered
-   * @param name
-   *          table of the partition being altered
-   * @param new_parts
-   *          new partition list
-   * @param handler
-   *          HMSHandle object (required to log event notification)
-   * @return the altered partition list
-   * @throws InvalidOperationException
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   */
-  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh,
-    final String dbname, final String name, final List<Partition> new_parts,
-    EnvironmentContext environmentContext,HMSHandler handler)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java b/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
deleted file mode 100644
index 67aa144..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-
-import org.apache.hadoop.fs.LocatedFileStatus;
-
-import org.apache.hadoop.fs.RemoteIterator;
-
-import java.util.LinkedList;
-
-import java.util.Queue;
-
-import java.util.ArrayList;
-
-import java.util.List;
-
-import com.google.common.collect.Lists;
-
-import java.io.IOException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.io.HdfsUtils;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.ThreadLocalRawStore;
-import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.shims.HadoopShims;
-import org.apache.hadoop.hive.shims.ShimLoader;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class FileMetadataManager {
-  private static final Log LOG = LogFactory.getLog(FileMetadataManager.class);
-  private static final HadoopShims SHIMS = ShimLoader.getHadoopShims();
-
-  private final ThreadLocalRawStore tlms;
-  private final ExecutorService threadPool;
-  private final HiveConf conf;
-
-  private final class CacheUpdateRequest implements Callable<Void> {
-    FileMetadataExprType type;
-    String location;
-
-    public CacheUpdateRequest(FileMetadataExprType type, String location) {
-      this.type = type;
-      this.location = location;
-    }
-
-    @Override
-    public Void call() throws Exception {
-      try {
-        cacheMetadata(type, location);
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-      } catch (Exception ex) {
-        // Nobody can see this exception on the threadpool; just log it.
-        LOG.error("Failed to cache file metadata in background for " + type + ", " + location, ex);
-      }
-      return null;
-    }
-  }
-
-  public FileMetadataManager(ThreadLocalRawStore tlms, HiveConf conf) {
-    this.tlms = tlms;
-    this.conf = conf;
-    int numThreads = HiveConf.getIntVar(conf, ConfVars.METASTORE_HBASE_FILE_METADATA_THREADS);
-    this.threadPool = Executors.newFixedThreadPool(numThreads,
-        new ThreadFactoryBuilder().setNameFormat("File-Metadata-%d").setDaemon(true).build());
-  }
-
-  public void queueCacheMetadata(String location, FileMetadataExprType type) {
-    threadPool.submit(new CacheUpdateRequest(type, location));
-  }
-
-  private void cacheMetadata(FileMetadataExprType type, String location)
-      throws MetaException, IOException, InterruptedException {
-    Path path = new Path(location);
-    FileSystem fs = path.getFileSystem(conf);
-    List<Path> files;
-    if (!fs.isDirectory(path)) {
-      files = Lists.newArrayList(path);
-    } else {
-      files = new ArrayList<>();
-      RemoteIterator<LocatedFileStatus> iter = fs.listFiles(path, true);
-      while (iter.hasNext()) {
-        // TODO: use fileId right from the list after HDFS-7878; or get dfs client and do it
-        LocatedFileStatus lfs = iter.next();
-        if (lfs.isDirectory()) continue;
-        files.add(lfs.getPath());
-      }
-    }
-    for (Path file : files) {
-      long fileId;
-      // TODO: use the other HdfsUtils here
-      if (!(fs instanceof DistributedFileSystem)) return;
-      try {
-        fileId = SHIMS.getFileId((DistributedFileSystem)fs,
-            Path.getPathWithoutSchemeAndAuthority(file).toString());
-      } catch (UnsupportedOperationException ex) {
-        LOG.error("Cannot cache file metadata for " + location + "; "
-            + fs.getClass().getCanonicalName() + " does not support fileId");
-        return;
-      }
-      LOG.info("Caching file metadata for " + file + " (file ID " + fileId + ")");
-      file = HdfsUtils.getFileIdPath(fs, file, fileId);
-      tlms.getMS().getFileMetadataHandler(type).cacheFileMetadata(fileId, fs, file);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
deleted file mode 100644
index a2ad06f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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;
-
-import com.codahale.metrics.Counter;
-import com.codahale.metrics.MetricRegistry;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
-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.DropPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.DropTableEvent;
-import org.apache.hadoop.hive.metastore.metrics.Metrics;
-import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Report metrics of metadata added, deleted by this Hive Metastore.
- */
-public class HMSMetricsListener extends MetaStoreEventListener {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(HMSMetricsListener.class);
-
-  private Counter createdDatabases, deletedDatabases, createdTables, deletedTables, createdParts,
-      deletedParts;
-
-  public HMSMetricsListener(Configuration config) {
-    super(config);
-    createdDatabases = Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_DATABASES);
-    deletedDatabases = Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_DATABASES);
-    createdTables = Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_TABLES);
-    deletedTables = Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_TABLES);
-    createdParts = Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_PARTITIONS);
-    deletedParts = Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_PARTITIONS);
-  }
-
-  @Override
-  public void onCreateDatabase(CreateDatabaseEvent dbEvent) throws MetaException {
-    HiveMetaStore.HMSHandler.databaseCount.incrementAndGet();
-    createdDatabases.inc();
-  }
-
-  @Override
-  public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
-    HiveMetaStore.HMSHandler.databaseCount.decrementAndGet();
-    deletedDatabases.inc();
-  }
-
-  @Override
-  public void onCreateTable(CreateTableEvent tableEvent) throws MetaException {
-    HiveMetaStore.HMSHandler.tableCount.incrementAndGet();
-    createdTables.inc();
-  }
-
-  @Override
-  public void onDropTable(DropTableEvent tableEvent) throws MetaException {
-    HiveMetaStore.HMSHandler.tableCount.decrementAndGet();
-    deletedTables.inc();
-  }
-
-  @Override
-  public void onDropPartition(DropPartitionEvent partitionEvent) throws MetaException {
-    HiveMetaStore.HMSHandler.partCount.decrementAndGet();
-    deletedParts.inc();
-  }
-
-  @Override
-  public void onAddPartition(AddPartitionEvent partitionEvent) throws MetaException {
-    HiveMetaStore.HMSHandler.partCount.incrementAndGet();
-    createdParts.inc();
-  }
-}


[07/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java
deleted file mode 100644
index e99a31f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.messaging.DropIndexMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON Implementation of DropIndexMessage.
- */
-public class JSONDropIndexMessage extends DropIndexMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, indexName, origTableName, indexTableName;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONDropIndexMessage() {}
-
-  public JSONDropIndexMessage(String server, String servicePrincipal, Index index, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = index.getDbName();
-    this.indexName = index.getIndexName();
-    this.origTableName = index.getOrigTableName();
-    this.indexTableName = index.getIndexTableName();
-
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  @Override
-  public String getDB() { return db; }
-
-  @Override
-  public String getServer() { return server; }
-
-  @Override
-  public String getServicePrincipal() { return servicePrincipal; }
-
-  @Override
-  public Long getTimestamp() { return timestamp; }
-
-  @Override
-  public String getIndexName() {
-    return indexName;
-  }
-
-  @Override
-  public String getOrigTableName() {
-    return origTableName;
-  }
-
-  @Override
-  public String getIndexTableName() {
-    return indexTableName;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java
deleted file mode 100644
index 576806c..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import java.util.List;
-import java.util.Map;
-
-/**
- * JSON implementation of DropPartitionMessage.
- */
-public class JSONDropPartitionMessage extends DropPartitionMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, table, tableType, tableObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  List<Map<String, String>> partitions;
-
-  /**
-   * Default Constructor. Required for Jackson.
-   */
-  public JSONDropPartitionMessage() {
-  }
-
-  public JSONDropPartitionMessage(String server, String servicePrincipal, String db, String table,
-      List<Map<String, String>> partitions, Long timestamp) {
-    this(server, servicePrincipal, db, table,  null, partitions, timestamp);
-  }
-
-  public JSONDropPartitionMessage(String server, String servicePrincipal, String db, String table,
-      String tableType, List<Map<String, String>> partitions, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = db;
-    this.table = table;
-    this.tableType = tableType;
-    this.partitions = partitions;
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  public JSONDropPartitionMessage(String server, String servicePrincipal, Table tableObj,
-      List<Map<String, String>> partitionKeyValues, long timestamp) {
-    this(server, servicePrincipal, tableObj.getDbName(), tableObj.getTableName(),
-        tableObj.getTableType(), partitionKeyValues, timestamp);
-    try {
-      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return db;
-  }
-
-  @Override
-  public String getTable() {
-    return table;
-  }
-
-  @Override
-  public String getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public List<Map<String, String>> getPartitions() {
-    return partitions;
-  }
-
-  @Override
-  public Table getTableObj() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjJson, Table.class);
-  }
-
-  public String getTableObjJson() {
-    return tableObjJson;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java
deleted file mode 100644
index 4334f73..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.DropTableMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON implementation of DropTableMessage.
- */
-public class JSONDropTableMessage extends DropTableMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, table, tableType, tableObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONDropTableMessage() {
-  }
-
-  public JSONDropTableMessage(String server, String servicePrincipal, String db, String table,
-      Long timestamp) {
-    this(server, servicePrincipal, db, table, null, timestamp);
-  }
-
-  public JSONDropTableMessage(String server, String servicePrincipal, String db, String table,
-      String tableType, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = db;
-    this.table = table;
-    this.tableType = tableType;
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  public JSONDropTableMessage(String server, String servicePrincipal, Table tableObj,
-      Long timestamp) {
-    this(server, servicePrincipal, tableObj.getDbName(), tableObj.getTableName(),
-        tableObj.getTableType(), timestamp);
-    try {
-      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-    checkValid();
-  }
-
-  @Override
-  public String getTable() {
-    return table;
-  }
-
-  @Override
-  public String getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public Table getTableObj() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
-  }
-
-  @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 toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
deleted file mode 100644
index 1369fd2..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import com.google.common.collect.Lists;
-
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * JSON implementation of InsertMessage
- */
-public class JSONInsertMessage extends InsertMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, table, tableType, tableObjJson, ptnObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  String replace;
-
-  @JsonProperty
-  List<String> files;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONInsertMessage() {
-  }
-
-  public JSONInsertMessage(String server, String servicePrincipal, Table tableObj, Partition ptnObj,
-                           boolean replace, Iterator<String> fileIter, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-
-    if (null == tableObj) {
-      throw new IllegalArgumentException("Table not valid.");
-    }
-
-    this.db = tableObj.getDbName();
-    this.table = tableObj.getTableName();
-    this.tableType = tableObj.getTableType();
-
-    try {
-      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
-      if (null != ptnObj) {
-        this.ptnObjJson = JSONMessageFactory.createPartitionObjJson(ptnObj);
-      } else {
-        this.ptnObjJson = null;
-      }
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-
-    this.timestamp = timestamp;
-    this.replace = Boolean.toString(replace);
-    this.files = Lists.newArrayList(fileIter);
-
-    checkValid();
-  }
-
-  @Override
-  public String getTable() {
-    return table;
-  }
-
-  @Override
-  public String getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public Iterable<String> getFiles() {
-    return files;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return db;
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public boolean isReplace() { return Boolean.parseBoolean(replace); }
-
-  @Override
-  public Table getTableObj() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
-  }
-
-  @Override
-  public Partition getPtnObj() throws Exception {
-    return ((null == ptnObjJson) ? null : (Partition) JSONMessageFactory.getTObj(ptnObjJson, Partition.class));
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
deleted file mode 100644
index 7f588a0..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
+++ /dev/null
@@ -1,243 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddPrimaryKeyMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
-import org.apache.hadoop.hive.metastore.messaging.AlterIndexMessage;
-import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateIndexMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropIndexMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropTableMessage;
-import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
-import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
-import org.codehaus.jackson.map.DeserializationConfig;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
-
-/**
- * MessageDeserializer implementation, for deserializing from JSON strings.
- */
-public class JSONMessageDeserializer extends MessageDeserializer {
-
-  static ObjectMapper mapper = new ObjectMapper(); // Thread-safe.
-
-  static {
-    mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
-    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, false);
-    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, false);
-    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, false);
-  }
-
-  @Override
-  public CreateDatabaseMessage getCreateDatabaseMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONCreateDatabaseMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONCreateDatabaseMessage.", exception);
-    }
-  }
-
-  @Override
-  public DropDatabaseMessage getDropDatabaseMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONDropDatabaseMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONDropDatabaseMessage.", exception);
-    }
-  }
-
-  @Override
-  public CreateTableMessage getCreateTableMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONCreateTableMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONCreateTableMessage.", exception);
-    }
-  }
-
-  @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);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONDropTableMessage.", exception);
-    }
-  }
-
-  @Override
-  public AddPartitionMessage getAddPartitionMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONAddPartitionMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct AddPartitionMessage.", exception);
-    }
-  }
-
-  @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 {
-      return mapper.readValue(messageBody, JSONDropPartitionMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct DropPartitionMessage.", exception);
-    }
-  }
-
-  @Override
-  public CreateFunctionMessage getCreateFunctionMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONCreateFunctionMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONCreateFunctionMessage.", exception);
-    }
-  }
-
-  @Override
-  public DropFunctionMessage getDropFunctionMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONDropFunctionMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONDropDatabaseMessage.", exception);
-    }
-  }
-
-  @Override
-  public CreateIndexMessage getCreateIndexMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONCreateIndexMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONCreateIndexMessage.", exception);
-    }
-  }
-
-  @Override
-  public DropIndexMessage getDropIndexMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONDropIndexMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONDropIndexMessage.", exception);
-    }
-  }
-
-  @Override
-  public AlterIndexMessage getAlterIndexMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONAlterIndexMessage.class);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not construct JSONAlterIndexMessage.", exception);
-    }
-  }
-
-  @Override
-  public InsertMessage getInsertMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONInsertMessage.class);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not construct InsertMessage", e);
-    }
-  }
-
-  @Override
-  public AddPrimaryKeyMessage getAddPrimaryKeyMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONAddPrimaryKeyMessage.class);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not construct AddPrimaryKeyMessage", e);
-    }
-  }
-
-  @Override
-  public AddForeignKeyMessage getAddForeignKeyMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONAddForeignKeyMessage.class);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not construct AddForeignKeyMessage", e);
-    }
-  }
-
-  @Override
-  public AddUniqueConstraintMessage getAddUniqueConstraintMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONAddUniqueConstraintMessage.class);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not construct AddUniqueConstraintMessage", e);
-    }
-  }
-
-  @Override
-  public AddNotNullConstraintMessage getAddNotNullConstraintMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONAddNotNullConstraintMessage.class);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not construct AddNotNullConstraintMessage", e);
-    }
-  }
-
-  @Override
-  public DropConstraintMessage getDropConstraintMessage(String messageBody) {
-    try {
-      return mapper.readValue(messageBody, JSONDropConstraintMessage.class);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not construct DropConstraintMessage", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
deleted file mode 100644
index c7877ee..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/**
- * 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.messaging.json;
-
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-import com.google.common.collect.Iterables;
-
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.NotificationEvent;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddPrimaryKeyMessage;
-import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
-import org.apache.hadoop.hive.metastore.messaging.AlterIndexMessage;
-import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateIndexMessage;
-import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropIndexMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.DropTableMessage;
-import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
-import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
-import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
-import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
-import org.apache.thrift.TBase;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.node.ObjectNode;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-
-/**
- * The JSON implementation of the MessageFactory. Constructs JSON implementations of each
- * message-type.
- */
-public class JSONMessageFactory extends MessageFactory {
-
-  private static final Logger LOG = LoggerFactory.getLogger(JSONMessageFactory.class.getName());
-
-  private static JSONMessageDeserializer deserializer = new JSONMessageDeserializer();
-
-  @Override
-  public MessageDeserializer getDeserializer() {
-    return deserializer;
-  }
-
-  @Override
-  public String getMessageFormat() {
-    return "json-0.2";
-  }
-
-  @Override
-  public CreateDatabaseMessage buildCreateDatabaseMessage(Database db) {
-    return new JSONCreateDatabaseMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, db.getName(), now());
-  }
-
-  @Override
-  public DropDatabaseMessage buildDropDatabaseMessage(Database db) {
-    return new JSONDropDatabaseMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, db.getName(), now());
-  }
-
-  @Override
-  public CreateTableMessage buildCreateTableMessage(Table table, Iterator<String> fileIter) {
-    return new JSONCreateTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table, fileIter, now());
-  }
-
-  @Override
-  public AlterTableMessage buildAlterTableMessage(Table before, Table after, boolean isTruncateOp) {
-    return new JSONAlterTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, before, after, isTruncateOp, now());
-  }
-
-  @Override
-  public DropTableMessage buildDropTableMessage(Table table) {
-    return new JSONDropTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table, now());
-  }
-
-  @Override
-  public AddPartitionMessage buildAddPartitionMessage(Table table,
-      Iterator<Partition> partitionsIterator, Iterator<PartitionFiles> partitionFileIter) {
-    return new JSONAddPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table,
-        partitionsIterator, partitionFileIter, now());
-  }
-
-  @Override
-  public AlterPartitionMessage buildAlterPartitionMessage(Table table, Partition before,
-      Partition after, boolean isTruncateOp) {
-    return new JSONAlterPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table, before, after, isTruncateOp,
-        now());
-  }
-
-  @Override
-  public DropPartitionMessage buildDropPartitionMessage(Table table,
-      Iterator<Partition> partitionsIterator) {
-    return new JSONDropPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table,
-        getPartitionKeyValues(table, partitionsIterator), now());
-  }
-
-  @Override
-  public CreateFunctionMessage buildCreateFunctionMessage(Function fn) {
-    return new JSONCreateFunctionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, fn, now());
-  }
-
-  @Override
-  public DropFunctionMessage buildDropFunctionMessage(Function fn) {
-    return new JSONDropFunctionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, fn, now());
-  }
-
-  @Override
-  public CreateIndexMessage buildCreateIndexMessage(Index idx) {
-    return new JSONCreateIndexMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, idx, now());
-  }
-
-  @Override
-  public DropIndexMessage buildDropIndexMessage(Index idx) {
-    return new JSONDropIndexMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, idx, now());
-  }
-
-  @Override
-  public AlterIndexMessage buildAlterIndexMessage(Index before, Index after) {
-    return new JSONAlterIndexMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, before, after, now());
-  }
-
-  @Override
-  public InsertMessage buildInsertMessage(Table tableObj, Partition partObj,
-                                          boolean replace, Iterator<String> fileIter) {
-    return new JSONInsertMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, tableObj, partObj, replace, fileIter, now());
-  }
-
-  @Override
-  public AddPrimaryKeyMessage buildAddPrimaryKeyMessage(List<SQLPrimaryKey> pks) {
-    return new JSONAddPrimaryKeyMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, pks, now());
-  }
-
-  @Override
-  public AddForeignKeyMessage buildAddForeignKeyMessage(List<SQLForeignKey> fks) {
-    return new JSONAddForeignKeyMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, fks, now());
-  }
-
-  @Override
-  public AddUniqueConstraintMessage buildAddUniqueConstraintMessage(List<SQLUniqueConstraint> uks) {
-    return new JSONAddUniqueConstraintMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, uks, now());
-  }
-
-  @Override
-  public AddNotNullConstraintMessage buildAddNotNullConstraintMessage(List<SQLNotNullConstraint> nns) {
-    return new JSONAddNotNullConstraintMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, nns, now());
-  }
-
-  @Override
-  public DropConstraintMessage buildDropConstraintMessage(String dbName, String tableName,
-      String constraintName) {
-    return new JSONDropConstraintMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, dbName, tableName,
-        constraintName, now());
-  }
-
-  private long now() {
-    return System.currentTimeMillis() / 1000;
-  }
-
-  static Map<String, String> getPartitionKeyValues(Table table, Partition partition) {
-    Map<String, String> partitionKeys = new LinkedHashMap<String, String>();
-    for (int i = 0; i < table.getPartitionKeysSize(); ++i)
-      partitionKeys.put(table.getPartitionKeys().get(i).getName(), partition.getValues().get(i));
-    return partitionKeys;
-  }
-
-  static List<Map<String, String>> getPartitionKeyValues(final Table table,
-      Iterator<Partition> iterator) {
-    return Lists.newArrayList(Iterators.transform(iterator,
-        new com.google.common.base.Function<Partition, Map<String, String>>() {
-          @Override
-          public Map<String, String> apply(@Nullable Partition partition) {
-            return getPartitionKeyValues(table, partition);
-          }
-        }));
-  }
-
-  static String createPrimaryKeyObjJson(SQLPrimaryKey primaryKeyObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(primaryKeyObj, "UTF-8");
-  }
-
-  static String createForeignKeyObjJson(SQLForeignKey foreignKeyObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(foreignKeyObj, "UTF-8");
-  }
-
-  static String createUniqueConstraintObjJson(SQLUniqueConstraint uniqueConstraintObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(uniqueConstraintObj, "UTF-8");
-  }
-
-  static String createNotNullConstraintObjJson(SQLNotNullConstraint notNullConstaintObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(notNullConstaintObj, "UTF-8");
-  }
-
-  static String createTableObjJson(Table tableObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(tableObj, "UTF-8");
-  }
-
-  static String createPartitionObjJson(Partition partitionObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(partitionObj, "UTF-8");
-  }
-
-  static String createFunctionObjJson(Function functionObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(functionObj, "UTF-8");
-  }
-
-  static String createIndexObjJson(Index indexObj) throws TException {
-    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-    return serializer.toString(indexObj, "UTF-8");
-  }
-
-  public static ObjectNode getJsonTree(NotificationEvent event) throws Exception {
-    return getJsonTree(event.getMessage());
-  }
-
-  public static ObjectNode getJsonTree(String eventMessage) throws Exception {
-    JsonParser jsonParser = (new JsonFactory()).createJsonParser(eventMessage);
-    ObjectMapper mapper = new ObjectMapper();
-    return mapper.readValue(jsonParser, ObjectNode.class);
-  }
-
-  public static Table getTableObj(ObjectNode jsonTree) throws Exception {
-    TDeserializer deSerializer = new TDeserializer(new TJSONProtocol.Factory());
-    Table tableObj = new Table();
-    String tableJson = jsonTree.get("tableObjJson").asText();
-    deSerializer.deserialize(tableObj, tableJson, "UTF-8");
-    return tableObj;
-  }
-
-  /*
-   * TODO: Some thoughts here : We have a current todo to move some of these methods over to
-   * MessageFactory instead of being here, so we can override them, but before we move them over,
-   * we should keep the following in mind:
-   *
-   * a) We should return Iterables, not Lists. That makes sure that we can be memory-safe when
-   * implementing it rather than forcing ourselves down a path wherein returning List is part of
-   * our interface, and then people use .size() or somesuch which makes us need to materialize
-   * the entire list and not change. Also, returning Iterables allows us to do things like
-   * Iterables.transform for some of these.
-   * b) We should not have "magic" names like "tableObjJson", because that breaks expectation of a
-   * couple of things - firstly, that of serialization format, although that is fine for this
-   * JSONMessageFactory, and secondly, that makes us just have a number of mappings, one for each
-   * obj type, and sometimes, as the case is with alter, have multiples. Also, any event-specific
-   * item belongs in that event message / event itself, as opposed to in the factory. It's okay to
-   * have utility accessor methods here that are used by each of the messages to provide accessors.
-   * I'm adding a couple of those here.
-   *
-   */
-
-  public static TBase getTObj(String tSerialized, Class<? extends TBase> objClass) throws Exception{
-    TDeserializer thriftDeSerializer = new TDeserializer(new TJSONProtocol.Factory());
-    TBase obj = objClass.newInstance();
-    thriftDeSerializer.deserialize(obj, tSerialized, "UTF-8");
-    return obj;
-  }
-
-  public static Iterable<? extends TBase> getTObjs(
-      Iterable<String> objRefStrs, final Class<? extends TBase> objClass) throws Exception {
-
-    try {
-      return Iterables.transform(objRefStrs, new com.google.common.base.Function<String,TBase>(){
-        @Override
-        public TBase apply(@Nullable String objStr){
-          try {
-            return getTObj(objStr, objClass);
-          } catch (Exception e) {
-            throw new RuntimeException(e);
-          }
-        }
-      });
-    } catch (RuntimeException re){
-      // We have to add this bit of exception handling here, because Function.apply does not allow us to throw
-      // the actual exception that might be a checked exception, so we wind up needing to throw a RuntimeException
-      // with the previously thrown exception as its cause. However, since RuntimeException.getCause() returns
-      // a throwable instead of an Exception, we have to account for the possibility that the underlying code
-      // might have thrown a Throwable that we wrapped instead, in which case, continuing to throw the
-      // RuntimeException is the best thing we can do.
-      Throwable t = re.getCause();
-      if (t instanceof Exception){
-        throw (Exception) t;
-      } else {
-        throw re;
-      }
-    }
-  }
-
-  // If we do not need this format of accessor using ObjectNode, this is a candidate for removal as well
-  public static Iterable<? extends TBase> getTObjs(
-      ObjectNode jsonTree, String objRefListName, final Class<? extends TBase> objClass) throws Exception {
-    Iterable<JsonNode> jsonArrayIterator = jsonTree.get(objRefListName);
-    com.google.common.base.Function<JsonNode,String> textExtractor =
-        new com.google.common.base.Function<JsonNode, String>() {
-      @Nullable
-      @Override
-      public String apply(@Nullable JsonNode input) {
-        return input.asText();
-      }
-    };
-    return getTObjs(Iterables.transform(jsonArrayIterator, textExtractor), objClass);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
deleted file mode 100644
index 03ea7fc..0000000
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.hive.metastore.api.*;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.util.Arrays;
-
-public class TestHiveAlterHandler {
-
-  @Test
-  public void testAlterTableAddColNotUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
-    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
-    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
-    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
-    FieldSchema col4 = new FieldSchema("col4", "string", "col4 comment");
-
-    StorageDescriptor oldSd = new StorageDescriptor();
-    oldSd.setCols(Arrays.asList(col1, col2, col3));
-    Table oldTable = new Table();
-    oldTable.setDbName("default");
-    oldTable.setTableName("test_table");
-    oldTable.setSd(oldSd);
-
-    StorageDescriptor newSd = new StorageDescriptor(oldSd);
-    newSd.setCols(Arrays.asList(col1, col2, col3, col4));
-    Table newTable = new Table(oldTable);
-    newTable.setSd(newSd);
-
-    RawStore msdb = Mockito.mock(RawStore.class);
-    Mockito.doThrow(new RuntimeException("shouldn't be called")).when(msdb).getTableColumnStatistics(
-        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3"));
-    HiveAlterHandler handler = new HiveAlterHandler();
-    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
-  }
-
-  @Test
-  public void testAlterTableDelColUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
-    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
-    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
-    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
-    FieldSchema col4 = new FieldSchema("col4", "string", "col4 comment");
-
-    StorageDescriptor oldSd = new StorageDescriptor();
-    oldSd.setCols(Arrays.asList(col1, col2, col3, col4));
-    Table oldTable = new Table();
-    oldTable.setDbName("default");
-    oldTable.setTableName("test_table");
-    oldTable.setSd(oldSd);
-
-    StorageDescriptor newSd = new StorageDescriptor(oldSd);
-    newSd.setCols(Arrays.asList(col1, col2, col3));
-    Table newTable = new Table(oldTable);
-    newTable.setSd(newSd);
-
-    RawStore msdb = Mockito.mock(RawStore.class);
-    HiveAlterHandler handler = new HiveAlterHandler();
-    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
-    Mockito.verify(msdb, Mockito.times(1)).getTableColumnStatistics(
-        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4")
-    );
-  }
-
-  @Test
-  public void testAlterTableChangePosNotUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
-    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
-    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
-    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
-    FieldSchema col4 = new FieldSchema("col4", "string", "col4 comment");
-
-    StorageDescriptor oldSd = new StorageDescriptor();
-    oldSd.setCols(Arrays.asList(col1, col2, col3, col4));
-    Table oldTable = new Table();
-    oldTable.setDbName("default");
-    oldTable.setTableName("test_table");
-    oldTable.setSd(oldSd);
-
-    StorageDescriptor newSd = new StorageDescriptor(oldSd);
-    newSd.setCols(Arrays.asList(col1, col4, col2, col3));
-    Table newTable = new Table(oldTable);
-    newTable.setSd(newSd);
-
-    RawStore msdb = Mockito.mock(RawStore.class);
-    Mockito.doThrow(new RuntimeException("shouldn't be called")).when(msdb).getTableColumnStatistics(
-        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4"));
-    HiveAlterHandler handler = new HiveAlterHandler();
-    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java
deleted file mode 100644
index e5c8a40..0000000
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreUtils.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Arrays;
-
-public class TestMetaStoreUtils {
-
-  @Test
-  public void testcolumnsIncludedByNameType() {
-    FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
-    FieldSchema col1a = new FieldSchema("col1", "string", "col1 but with a different comment");
-    FieldSchema col2 = new FieldSchema("col2", "string", "col2 comment");
-    FieldSchema col3 = new FieldSchema("col3", "string", "col3 comment");
-    Assert.assertTrue(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1), Arrays.asList(col1)));
-    Assert.assertTrue(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1), Arrays.asList(col1a)));
-    Assert.assertTrue(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col1, col2)));
-    Assert.assertTrue(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col2, col1)));
-    Assert.assertTrue(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col1, col2, col3)));
-    Assert.assertTrue(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col3, col2, col1)));
-    Assert.assertFalse(MetaStoreUtils.columnsIncludedByNameType(Arrays.asList(col1, col2), Arrays.asList(col1)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/test/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java
deleted file mode 100644
index 0d2a9cb..0000000
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * 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;
-
-import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.util.concurrent.TimeUnit;
-
-import javax.jdo.JDOException;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-public class TestRetriesInRetryingHMSHandler {
-
-  private static HiveConf hiveConf;
-  private static final int RETRY_ATTEMPTS = 3;
-
-  @BeforeClass
-  public static void setup() throws IOException {
-    hiveConf = new HiveConf();
-    int port = MetaStoreTestUtils.findFreePort();
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    hiveConf.setIntVar(HiveConf.ConfVars.HMSHANDLERATTEMPTS, RETRY_ATTEMPTS);
-    hiveConf.setTimeVar(HiveConf.ConfVars.HMSHANDLERINTERVAL, 10, TimeUnit.MILLISECONDS);
-    hiveConf.setBoolVar(HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF, false);
-  }
-
-  /*
-   * If the init method of HMSHandler throws exception for the first time
-   * while creating RetryingHMSHandler it should be retried
-   */
-  @Test
-  public void testRetryInit() throws MetaException {
-    IHMSHandler mockBaseHandler = Mockito.mock(HiveMetaStore.HMSHandler.class);
-    Mockito.when(mockBaseHandler.getConf()).thenReturn(hiveConf);
-    Mockito
-    .doThrow(JDOException.class)
-    .doNothing()
-    .when(mockBaseHandler).init();
-    RetryingHMSHandler.getProxy(hiveConf, mockBaseHandler, false);
-    Mockito.verify(mockBaseHandler, Mockito.times(2)).init();
-  }
-
-  /*
-   * init method in HMSHandler should not be retried if there are no exceptions
-   */
-  @Test
-  public void testNoRetryInit() throws MetaException {
-    IHMSHandler mockBaseHandler = Mockito.mock(HiveMetaStore.HMSHandler.class);
-    Mockito.when(mockBaseHandler.getConf()).thenReturn(hiveConf);
-    Mockito.doNothing().when(mockBaseHandler).init();
-    RetryingHMSHandler.getProxy(hiveConf, mockBaseHandler, false);
-    Mockito.verify(mockBaseHandler, Mockito.times(1)).init();
-  }
-
-  /*
-   * If the init method in HMSHandler throws exception all the times it should be retried until
-   * HiveConf.ConfVars.HMSHANDLERATTEMPTS is reached before giving up
-   */
-  @Test(expected = MetaException.class)
-  public void testRetriesLimit() throws MetaException {
-    IHMSHandler mockBaseHandler = Mockito.mock(HiveMetaStore.HMSHandler.class);
-    Mockito.when(mockBaseHandler.getConf()).thenReturn(hiveConf);
-    Mockito.doThrow(JDOException.class).when(mockBaseHandler).init();
-    RetryingHMSHandler.getProxy(hiveConf, mockBaseHandler, false);
-    Mockito.verify(mockBaseHandler, Mockito.times(RETRY_ATTEMPTS)).init();
-  }
-
-  /*
-   * Test retries when InvocationException wrapped in MetaException wrapped in JDOException
-   * is thrown
-   */
-  @Test
-  public void testWrappedMetaExceptionRetry() throws MetaException {
-    IHMSHandler mockBaseHandler = Mockito.mock(HiveMetaStore.HMSHandler.class);
-    Mockito.when(mockBaseHandler.getConf()).thenReturn(hiveConf);
-    //JDOException wrapped in MetaException wrapped in InvocationException
-    MetaException me = new MetaException("Dummy exception");
-    me.initCause(new JDOException());
-    InvocationTargetException ex = new InvocationTargetException(me);
-    Mockito
-    .doThrow(me)
-    .doNothing()
-    .when(mockBaseHandler).init();
-    RetryingHMSHandler.getProxy(hiveConf, mockBaseHandler, false);
-    Mockito.verify(mockBaseHandler, Mockito.times(2)).init();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/test/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java b/metastore/src/test/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java
deleted file mode 100644
index 9e22d8f..0000000
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/messaging/json/TestJSONMessageDeserializer.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.json.JSONException;
-import org.junit.Test;
-import org.skyscreamer.jsonassert.JSONAssert;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestJSONMessageDeserializer {
-
-  public static class MyClass {
-    @JsonProperty
-    private int a;
-    @JsonProperty
-    private Map<String, String> map;
-    private long l;
-    private String shouldNotSerialize = "shouldNotSerialize";
-
-    //for jackson to instantiate
-    MyClass() {
-    }
-
-    MyClass(int a, Map<String, String> map, long l) {
-      this.a = a;
-      this.map = map;
-      this.l = l;
-    }
-
-    @JsonProperty
-    long getL() {
-      return l;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o)
-        return true;
-      if (o == null || getClass() != o.getClass())
-        return false;
-
-      MyClass myClass = (MyClass) o;
-
-      if (a != myClass.a)
-        return false;
-      if (l != myClass.l)
-        return false;
-      if (!map.equals(myClass.map))
-        return false;
-      return shouldNotSerialize.equals(myClass.shouldNotSerialize);
-    }
-
-    @Override
-    public int hashCode() {
-      int result = a;
-      result = 31 * result + map.hashCode();
-      result = 31 * result + (int) (l ^ (l >>> 32));
-      result = 31 * result + shouldNotSerialize.hashCode();
-      return result;
-    }
-  }
-
-  @Test
-  public void shouldNotSerializePropertiesNotAnnotated() throws IOException, JSONException {
-    MyClass obj = new MyClass(Integer.MAX_VALUE, new HashMap<String, String>() {{
-      put("a", "a");
-      put("b", "b");
-    }}, Long.MAX_VALUE);
-    String json = JSONMessageDeserializer.mapper.writeValueAsString(obj);
-    JSONAssert.assertEquals(
-        "{\"a\":2147483647,\"map\":{\"b\":\"b\",\"a\":\"a\"},\"l\":9223372036854775807}", json,
-        false);
-  }
-
-  @Test
-  public void shouldDeserializeJsonStringToObject() throws IOException {
-    String json = "{\"a\":47,\"map\":{\"a\":\"a\",\"b\":\"a value for b\"},\"l\":98}";
-    MyClass actual = JSONMessageDeserializer.mapper.readValue(json, MyClass.class);
-    MyClass expected = new MyClass(47, new HashMap<String, String>() {{
-      put("a", "a");
-      put("b", "a value for b");
-    }}, 98L);
-    assertEquals(expected, actual);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
index db624c0..7a28b43 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
@@ -21,15 +21,15 @@ import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
@@ -58,17 +58,17 @@ public class MetaDataExportListener extends MetaStorePreEventListener {
     String name = tbl.getTableName();
     org.apache.hadoop.hive.ql.metadata.Table mTbl = new org.apache.hadoop.hive.ql.metadata.Table(
         tbl);
-    HMSHandler handler = tableEvent.getHandler();
-    HiveConf hiveconf = handler.getHiveConf();
-    Warehouse wh = new Warehouse(hiveconf);
+    IHMSHandler handler = tableEvent.getHandler();
+    Configuration conf = handler.getConf();
+    Warehouse wh = new Warehouse(conf);
     Path tblPath = new Path(tbl.getSd().getLocation());
     fs = wh.getFs(tblPath);
     Date now = new Date();
     SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
     String dateString = sdf.format(now);
-    String exportPathString = hiveconf.getVar(HiveConf.ConfVars.METADATA_EXPORT_LOCATION);
-    boolean moveMetadataToTrash = hiveconf
-        .getBoolVar(HiveConf.ConfVars.MOVE_EXPORTED_METADATA_TO_TRASH);
+    String exportPathString = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.METADATA_EXPORT_LOCATION);
+    boolean moveMetadataToTrash = MetastoreConf
+        .getBoolVar(conf, MetastoreConf.ConfVars.MOVE_EXPORTED_METADATA_TO_TRASH);
     Path exportPath = null;
     if (exportPathString != null && exportPathString.length() == 0) {
       exportPath = fs.getHomeDirectory();

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java b/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java
index c372027..07b506e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java
@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.hive.ql.security;
 
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 
 public class HadoopDefaultMetastoreAuthenticator extends HadoopDefaultAuthenticator
   implements HiveMetastoreAuthenticationProvider {
 
   @Override
-  public void setMetaStoreHandler(HMSHandler handler) {
+  public void setMetaStoreHandler(IHMSHandler handler) {
     setConf(handler.getConf());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java
index 73e9bfb..9b83c23 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hive.ql.security;
 
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 
 
 /**
@@ -36,6 +36,6 @@ public interface HiveMetastoreAuthenticationProvider extends HiveAuthenticationP
    * authentication that needs to be done.
    * @param handler
    */
-  void setMetaStoreHandler(HMSHandler handler);
+  void setMetaStoreHandler(IHMSHandler handler);
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java
index 4120298..d3242e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.ql.security.authorization;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
@@ -32,7 +32,7 @@ public class DefaultHiveMetastoreAuthorizationProvider extends BitSetCheckedAuth
   }
 
   @Override
-  public void setMetaStoreHandler(HMSHandler handler) {
+  public void setMetaStoreHandler(IHMSHandler handler) {
     hive_db.setHandler(handler);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
index 191426c..fa66a21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hive.ql.security.authorization;
 
 import java.util.List;
 
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
@@ -43,7 +43,7 @@ public abstract class HiveAuthorizationProviderBase implements
 
     private final boolean hasHiveClient;
     private final HiveConf conf;
-    private HMSHandler handler;
+    private IHMSHandler handler;
 
     public HiveProxy(Hive hive) {
       this.hasHiveClient = hive != null;
@@ -57,7 +57,7 @@ public abstract class HiveAuthorizationProviderBase implements
       this.handler = null;
     }
 
-    public void setHandler(HMSHandler handler){
+    public void setHandler(IHMSHandler handler){
       this.handler = handler;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java
index 23161e3..bf21de7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hive.ql.security.authorization;
 
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
@@ -37,7 +37,7 @@ public interface HiveMetastoreAuthorizationProvider extends HiveAuthorizationPro
    * before any of the authorize methods are called.
    * @param handler
    */
-  void setMetaStoreHandler(HMSHandler handler);
+  void setMetaStoreHandler(IHMSHandler handler);
 
   /**
    * Authorize metastore authorization api call.

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/MetaStoreAuthzAPIAuthorizerEmbedOnly.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/MetaStoreAuthzAPIAuthorizerEmbedOnly.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/MetaStoreAuthzAPIAuthorizerEmbedOnly.java
index 5f1725c..00d95ef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/MetaStoreAuthzAPIAuthorizerEmbedOnly.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/MetaStoreAuthzAPIAuthorizerEmbedOnly.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -76,7 +76,7 @@ public class MetaStoreAuthzAPIAuthorizerEmbedOnly extends HiveAuthorizationProvi
   }
 
   @Override
-  public void setMetaStoreHandler(HMSHandler handler) {
+  public void setMetaStoreHandler(IHMSHandler handler) {
     // no-op - HMSHander not needed by this impl
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
index 32d8b3e..56fc9e4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
@@ -27,6 +27,7 @@ import java.util.List;
 
 import javax.security.auth.login.LoginException;
 
+import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,7 +37,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
@@ -271,7 +271,7 @@ public class StorageBasedAuthorizationProvider extends HiveAuthorizationProvider
   }
 
   @Override
-  public void setMetaStoreHandler(HMSHandler handler) {
+  public void setMetaStoreHandler(IHMSHandler handler) {
     hive_db.setHandler(handler);
     this.wh = handler.getWh();
     this.isRunFromMetaStore = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidCompactionHistoryService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidCompactionHistoryService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidCompactionHistoryService.java
deleted file mode 100644
index 5d9e7be..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidCompactionHistoryService.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * 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.ql.txn;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.txn.TxnStore;
-import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.ql.txn.compactor.HouseKeeperServiceBase;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Purges obsolete items from compaction history data
- */
-public class AcidCompactionHistoryService extends HouseKeeperServiceBase {
-  private static final Logger LOG = LoggerFactory.getLogger(AcidCompactionHistoryService.class);
-
-  @Override
-  protected long getStartDelayMs() {
-    return 0;
-  }
-  @Override
-  protected long getIntervalMs() {
-    return hiveConf.getTimeVar(HiveConf.ConfVars.COMPACTOR_HISTORY_REAPER_INTERVAL, TimeUnit.MILLISECONDS);
-  }
-  @Override
-  protected Runnable getScheduedAction(HiveConf hiveConf, AtomicInteger isAliveCounter) {
-    return new ObsoleteEntryReaper(hiveConf, isAliveCounter);
-  }
-
-  @Override
-  public String getServiceDescription() {
-    return "Removes obsolete entries from Compaction History";
-  }
-  
-  private static final class ObsoleteEntryReaper implements Runnable {
-    private final TxnStore txnHandler;
-    private final AtomicInteger isAliveCounter;
-    private ObsoleteEntryReaper(HiveConf hiveConf, AtomicInteger isAliveCounter) {
-      txnHandler = TxnUtils.getTxnStore(hiveConf);
-      this.isAliveCounter = isAliveCounter;
-    }
-    
-    @Override
-    public void run() {
-      TxnStore.MutexAPI.LockHandle handle = null;
-      try {
-        handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.CompactionHistory.name());
-        long startTime = System.currentTimeMillis();
-        txnHandler.purgeCompactionHistory();
-        int count = isAliveCounter.incrementAndGet(); 
-        LOG.info("History reaper reaper ran for " + (System.currentTimeMillis() - startTime)/1000 + "seconds.  isAliveCounter=" + count);
-      }
-      catch(Throwable t) {
-        LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
-      }
-      finally {
-        if(handle != null) {
-          handle.releaseLocks();
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
deleted file mode 100644
index 13b10de..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidHouseKeeperService.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * 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.ql.txn;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.txn.TxnStore;
-import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.ql.txn.compactor.HouseKeeperServiceBase;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Performs background tasks for Transaction management in Hive.
- * Runs inside Hive Metastore Service.
- */
-public class AcidHouseKeeperService extends HouseKeeperServiceBase {
-  private static final Logger LOG = LoggerFactory.getLogger(AcidHouseKeeperService.class);
-
-  @Override
-  protected long getStartDelayMs() {
-    return hiveConf.getTimeVar(HiveConf.ConfVars.HIVE_TIMEDOUT_TXN_REAPER_START, TimeUnit.MILLISECONDS);
-  }
-  @Override
-  protected long getIntervalMs() {
-    return hiveConf.getTimeVar(HiveConf.ConfVars.HIVE_TIMEDOUT_TXN_REAPER_INTERVAL, TimeUnit.MILLISECONDS);
-  }
-  @Override
-  protected Runnable getScheduedAction(HiveConf hiveConf, AtomicInteger isAliveCounter) {
-    return new TimedoutTxnReaper(hiveConf, isAliveCounter);
-  }
-
-  @Override
-  public String getServiceDescription() {
-    return "Abort expired transactions";
-  }
-
-  private static final class TimedoutTxnReaper implements Runnable {
-    private final TxnStore txnHandler;
-    private final AtomicInteger isAliveCounter;
-    private TimedoutTxnReaper(HiveConf hiveConf, AtomicInteger isAliveCounter) {
-      txnHandler = TxnUtils.getTxnStore(hiveConf);
-      this.isAliveCounter = isAliveCounter;
-    }
-    @Override
-    public void run() {
-      TxnStore.MutexAPI.LockHandle handle = null;
-      try {
-        handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.HouseKeeper.name());
-        long startTime = System.currentTimeMillis();
-        txnHandler.performTimeOuts();
-        int count = isAliveCounter.incrementAndGet();
-        LOG.info("timeout reaper ran for " + (System.currentTimeMillis() - startTime)/1000 + "seconds.  isAliveCounter=" + count);
-      }
-      catch(Throwable t) {
-        LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
-      }
-      finally {
-        if(handle != null) {
-          handle.releaseLocks();
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java
deleted file mode 100644
index 2b8f815..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/AcidWriteSetService.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * 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.ql.txn;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.txn.TxnStore;
-import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.ql.txn.compactor.HouseKeeperServiceBase;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Periodically cleans WriteSet tracking information used in Transaction management
- */
-public class AcidWriteSetService extends HouseKeeperServiceBase {
-  private static final Logger LOG = LoggerFactory.getLogger(AcidWriteSetService.class);
-  @Override
-  protected long getStartDelayMs() {
-    return 0;
-  }
-  @Override
-  protected long getIntervalMs() {
-    return hiveConf.getTimeVar(HiveConf.ConfVars.WRITE_SET_REAPER_INTERVAL, TimeUnit.MILLISECONDS);
-  }
-  @Override
-  protected Runnable getScheduedAction(HiveConf hiveConf, AtomicInteger isAliveCounter) {
-    return new WriteSetReaper(hiveConf, isAliveCounter);
-  }
-  @Override
-  public String getServiceDescription() {
-    return "Periodically cleans obsolete WriteSet tracking information used in Transaction management";
-  }
-  private static final class WriteSetReaper implements Runnable {
-    private final TxnStore txnHandler;
-    private final AtomicInteger isAliveCounter;
-    private WriteSetReaper(HiveConf hiveConf, AtomicInteger isAliveCounter) {
-      txnHandler = TxnUtils.getTxnStore(hiveConf);
-      this.isAliveCounter = isAliveCounter;
-    }
-    @Override
-    public void run() {
-      TxnStore.MutexAPI.LockHandle handle = null;
-      try {
-        handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.WriteSetCleaner.name());
-        long startTime = System.currentTimeMillis();
-        txnHandler.performWriteSetGC();
-        int count = isAliveCounter.incrementAndGet();
-        LOG.info("cleaner ran for " + (System.currentTimeMillis() - startTime)/1000 + "seconds.  isAliveCounter=" + count);
-      }
-      catch(Throwable t) {
-        LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
-      }
-      finally {
-        if(handle != null) {
-          handle.releaseLocks();
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
deleted file mode 100644
index 0aa160c..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/HouseKeeperServiceBase.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * 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.ql.txn.compactor;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HouseKeeperService;
-import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
-import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public abstract class HouseKeeperServiceBase implements HouseKeeperService {
-  private static final Logger LOG = LoggerFactory.getLogger(HouseKeeperServiceBase.class);
-  private ScheduledExecutorService pool = null;
-  protected final AtomicInteger isAliveCounter = new AtomicInteger(Integer.MIN_VALUE);
-  protected HiveConf hiveConf;
-
-  @Override
-  public void start(HiveConf hiveConf) throws Exception {
-    this.hiveConf = hiveConf;
-    HiveTxnManager mgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(hiveConf);
-    if(!mgr.supportsAcid()) {
-      LOG.info(this.getClass().getName() + " not started since " +
-        mgr.getClass().getName()  + " does not support Acid.");
-      return;//there are no transactions in this case
-    }
-    pool = Executors.newScheduledThreadPool(1, new ThreadFactory() {
-      private final AtomicInteger threadCounter = new AtomicInteger();
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t =
-            new Thread(r, HouseKeeperServiceBase.this.getClass().getName() + "-"
-                + threadCounter.getAndIncrement());
-        t.setDaemon(true);
-        return t;
-      }
-    });
-
-    TimeUnit tu = TimeUnit.MILLISECONDS;
-    pool.scheduleAtFixedRate(getScheduedAction(hiveConf, isAliveCounter), getStartDelayMs(),
-      getIntervalMs(), tu);
-    LOG.info("Started " + this.getClass().getName() + " with delay/interval = " + getStartDelayMs() + "/" +
-      getIntervalMs() + " " + tu);
-  }
-
-  @Override
-  public void stop() {
-    if(pool != null && !pool.isShutdown()) {
-      pool.shutdown();
-    }
-    pool = null;
-  }
-
-  /**
-   * This is used for testing only.  Each time the housekeeper runs, counter is incremented by 1.
-   * Starts with {@link java.lang.Integer#MIN_VALUE}
-   */
-  @Override
-  public int getIsAliveCounter() {
-    return isAliveCounter.get();
-  }
-
-  /**
-   * Delay in millis before first run of the task of this service.
-   */
-  protected abstract long getStartDelayMs();
-  /**
-   * Determines how frequently the service is running its task.
-   */
-  protected abstract long getIntervalMs();
-
-  /**
-   * The actual task implementation.  Must increment the counter on each iteration.
-   */
-  protected abstract Runnable getScheduedAction(HiveConf hiveConf, AtomicInteger isAliveCounter);
-}


[06/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 39d6b2b..662462c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hive.ql;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.RunnableConfigurable;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.LockState;
 import org.apache.hadoop.hive.metastore.api.LockType;
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.TxnInfo;
 import org.apache.hadoop.hive.metastore.api.TxnState;
+import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
@@ -36,7 +37,6 @@ import org.apache.hadoop.hive.ql.io.BucketCodec;
 import org.apache.hadoop.hive.ql.lockmgr.TestDbTxnManager2;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Ignore;
@@ -337,11 +337,11 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     runStatementOnDriver("start transaction");
     runStatementOnDriver("delete from " + Table.ACIDTBL + " where a = 5");
     //make sure currently running txn is considered aborted by housekeeper
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_TIMEDOUT_TXN_REAPER_START, 0, TimeUnit.SECONDS);
     hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 2, TimeUnit.MILLISECONDS);
-    AcidHouseKeeperService houseKeeperService = new AcidHouseKeeperService();
+    RunnableConfigurable houseKeeperService = new AcidHouseKeeperService();
+    houseKeeperService.setConf(hiveConf);
     //this will abort the txn
-    TestTxnCommands2.runHouseKeeperService(houseKeeperService, hiveConf);
+    houseKeeperService.run();
     //this should fail because txn aborted due to timeout
     CommandProcessorResponse cpr = runStatementOnDriverNegative("delete from " + Table.ACIDTBL + " where a = 5");
     Assert.assertTrue("Actual: " + cpr.getErrorMessage(), cpr.getErrorMessage().contains("Transaction manager has aborted the transaction txnid:1"));
@@ -349,6 +349,8 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     //now test that we don't timeout locks we should not
     //heartbeater should be running in the background every 1/2 second
     hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1, TimeUnit.SECONDS);
+    // Have to reset the conf when we change it so that the change takes affect
+    houseKeeperService.setConf(hiveConf);
     //hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER, true);
     runStatementOnDriver("start transaction");
     runStatementOnDriver("select count(*) from " + Table.ACIDTBL + " where a = 17");
@@ -380,14 +382,14 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     ShowLocksResponse slr = txnHandler.showLocks(new ShowLocksRequest());
     TestDbTxnManager2.checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", Table.ACIDTBL.name, null, slr.getLocks());
     pause(750);
-    TestTxnCommands2.runHouseKeeperService(houseKeeperService, hiveConf);
+    houseKeeperService.run();
     pause(750);
     slr = txnHandler.showLocks(new ShowLocksRequest());
     Assert.assertEquals("Unexpected lock count: " + slr, 1, slr.getLocks().size());
     TestDbTxnManager2.checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", Table.ACIDTBL.name, null, slr.getLocks());
 
     pause(750);
-    TestTxnCommands2.runHouseKeeperService(houseKeeperService, hiveConf);
+    houseKeeperService.run();
     slr = txnHandler.showLocks(new ShowLocksRequest());
     Assert.assertEquals("Unexpected lock count: " + slr, 1, slr.getLocks().size());
     TestDbTxnManager2.checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", Table.ACIDTBL.name, null, slr.getLocks());

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index 3737b6a..2faf098 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HouseKeeperService;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
@@ -46,6 +45,7 @@ import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
@@ -54,7 +54,6 @@ import org.apache.hadoop.hive.ql.io.BucketCodec;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.txn.AcidCompactionHistoryService;
 import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService;
 import org.apache.hadoop.hive.ql.txn.compactor.Cleaner;
 import org.apache.hadoop.hive.ql.txn.compactor.Initiator;
@@ -1041,7 +1040,8 @@ public class TestTxnCommands2 {
 
     hiveConf.setTimeVar(HiveConf.ConfVars.COMPACTOR_HISTORY_REAPER_INTERVAL, 10, TimeUnit.MILLISECONDS);
     AcidCompactionHistoryService compactionHistoryService = new AcidCompactionHistoryService();
-    runHouseKeeperService(compactionHistoryService, hiveConf);//should not remove anything from history
+    compactionHistoryService.setConf(hiveConf);
+    compactionHistoryService.run();
     checkCompactionState(new CompactionsByState(numAttemptedCompactions,numFailedCompactions,0,0,0,0,numFailedCompactions + numAttemptedCompactions), countCompacts(txnHandler));
 
     txnHandler.compact(new CompactionRequest("default", tblName, CompactionType.MAJOR));
@@ -1054,7 +1054,7 @@ public class TestTxnCommands2 {
     numAttemptedCompactions++;
     checkCompactionState(new CompactionsByState(numAttemptedCompactions,numFailedCompactions + 2,0,0,0,0,numFailedCompactions + 2 + numAttemptedCompactions), countCompacts(txnHandler));
 
-    runHouseKeeperService(compactionHistoryService, hiveConf);//should remove history so that we have
+    compactionHistoryService.run();
     //COMPACTOR_HISTORY_RETENTION_FAILED failed compacts left (and no other since we only have failed ones here)
     checkCompactionState(new CompactionsByState(
       hiveConf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_ATTEMPTED),
@@ -1078,7 +1078,7 @@ public class TestTxnCommands2 {
         hiveConf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_ATTEMPTED)+ 1), countCompacts(txnHandler));
 
     runCleaner(hiveConf); // transition to Success state
-    runHouseKeeperService(compactionHistoryService, hiveConf);//should not purge anything as all items within retention sizes
+    compactionHistoryService.run();
     checkCompactionState(new CompactionsByState(
       hiveConf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_ATTEMPTED),
       hiveConf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED),0,0,1,0,
@@ -1198,26 +1198,6 @@ public class TestTxnCommands2 {
     t.run();
   }
 
-  public static void runHouseKeeperService(HouseKeeperService houseKeeperService, HiveConf conf) throws Exception {
-    int lastCount = houseKeeperService.getIsAliveCounter();
-    houseKeeperService.start(conf);
-    int maxIter = 10;
-    int iterCount = 0;
-    while(houseKeeperService.getIsAliveCounter() <= lastCount) {
-      if(iterCount++ >= maxIter) {
-        //prevent test hangs
-        throw new IllegalStateException("HouseKeeper didn't run after " + (iterCount - 1) + " waits");
-      }
-      try {
-        Thread.sleep(100);//make sure it has run at least once
-      }
-      catch(InterruptedException ex) {
-        //...
-      }
-    }
-    houseKeeperService.stop();
-  }
-
   /**
    * HIVE-12352 has details
    * @throws Exception

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
index e46e65b..406bdea 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java
@@ -19,12 +19,14 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.ThreadPool;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
 import org.apache.hadoop.hive.metastore.api.TxnState;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
@@ -38,7 +40,6 @@ import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -202,24 +203,9 @@ public class TestDbTxnManager {
    * aborts timed out transactions
    */
   private void runReaper() throws Exception {
-    int lastCount = houseKeeperService.getIsAliveCounter();
-    houseKeeperService.start(conf);
-    int maxIter = 10;
-    int iterCount = 0;
-    while(houseKeeperService.getIsAliveCounter() <= lastCount) {
-      if(iterCount++ >= maxIter) {
-        //prevent test hangs
-        throw new IllegalStateException("Reaper didn't run after " + iterCount + " waits");
-      }
-      try {
-        Thread.sleep(100);//make sure it has run at least once
-      }
-      catch(InterruptedException ex) {
-        //...
-      }
-    }
-    houseKeeperService.stop();
+    houseKeeperService.run();
   }
+
   @Test
   public void testExceptions() throws Exception {
     addPartitionOutput(newTable(true), WriteEntity.WriteType.INSERT);
@@ -491,13 +477,14 @@ public class TestDbTxnManager {
     conf.setTimeVar(HiveConf.ConfVars.HIVE_TIMEDOUT_TXN_REAPER_START, 0, TimeUnit.SECONDS);
     conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 10, TimeUnit.SECONDS);
     houseKeeperService = new AcidHouseKeeperService();
+    houseKeeperService.setConf(conf);
   }
 
   @After
   public void tearDown() throws Exception {
-    if(houseKeeperService != null) houseKeeperService.stop();
     if (txnMgr != null) txnMgr.closeTxnManager();
     TxnDbUtil.cleanDb(conf);
+    ThreadPool.shutdown();
   }
 
   private static class MockQueryPlan extends QueryPlan {

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 15045d6..3c172b9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -19,12 +19,13 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.metastore.RunnableConfigurable;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.txn.AcidWriteSetService;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.TestTxnCommands2;
-import org.apache.hadoop.hive.ql.txn.AcidWriteSetService;
 import org.junit.After;
 import org.junit.Assert;
 import org.apache.hadoop.hive.common.FileUtils;
@@ -1036,7 +1037,8 @@ public class TestDbTxnManager2 {
       1, TxnDbUtil.countQueryAgent(conf, "select count(*) from WRITE_SET"));
 
     AcidWriteSetService houseKeeper = new AcidWriteSetService();
-    TestTxnCommands2.runHouseKeeperService(houseKeeper, conf);
+    houseKeeper.setConf(conf);
+    houseKeeper.run();
     //since T3 overlaps with Long Running (still open) GC does nothing
     Assert.assertEquals(1, TxnDbUtil.countQueryAgent(conf, "select count(*) from WRITE_SET"));
     checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 17 where a = 1"));//no rows match
@@ -1050,7 +1052,7 @@ public class TestDbTxnManager2 {
 
     locks = getLocks(txnMgr);
     Assert.assertEquals("Unexpected lock count", 0, locks.size());
-    TestTxnCommands2.runHouseKeeperService(houseKeeper, conf);
+    houseKeeper.run();
     Assert.assertEquals(0, TxnDbUtil.countQueryAgent(conf, "select count(*) from WRITE_SET"));
   }
   /**
@@ -1120,7 +1122,9 @@ public class TestDbTxnManager2 {
     Assert.assertEquals("Unexpected lock count", 1, locks.size());
     checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB2", null, locks);
     txnMgr.commitTxn();
-    TestTxnCommands2.runHouseKeeperService(new AcidWriteSetService(), conf);
+    RunnableConfigurable writeSetService = new AcidWriteSetService();
+    writeSetService.setConf(conf);
+    writeSetService.run();
     Assert.assertEquals(0, TxnDbUtil.countQueryAgent(conf, "select count(*) from WRITE_SET"));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
index 277738f..85ee8c7 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
@@ -51,17 +51,6 @@ import com.google.common.collect.Iterables;
 public class HdfsUtils {
   private static final Logger LOG = LoggerFactory.getLogger("shims.HdfsUtils");
 
-  // TODO: this relies on HDFS not changing the format; we assume if we could get inode ID, this
-  //       is still going to work. Otherwise, file IDs can be turned off. Later, we should use
-  //       as public utility method in HDFS to obtain the inode-based path.
-  private static final String HDFS_ID_PATH_PREFIX = "/.reserved/.inodes/";
-
-  public static Path getFileIdPath(
-      FileSystem fileSystem, Path path, long fileId) {
-    return (fileSystem instanceof DistributedFileSystem)
-        ? new Path(HDFS_ID_PATH_PREFIX + fileId) : path;
-  }
-
   /**
    * Copy the permissions, group, and ACLs from a source {@link HadoopFileStatus} to a target {@link Path}. This method
    * will only log a warning if permissions cannot be set, no exception will be thrown.

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/pom.xml
----------------------------------------------------------------------
diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml
index 8df622f..a2a34a5 100644
--- a/standalone-metastore/pom.xml
+++ b/standalone-metastore/pom.xml
@@ -236,6 +236,12 @@
       <artifactId>javax.jdo</artifactId>
       <version>${datanucleus-jdo.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.skyscreamer</groupId>
+      <artifactId>jsonassert</artifactId>
+      <version>1.4.0</version>
+      <scope>test</scope>
+    </dependency>
 
     <!-- test scope dependencies -->
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapability.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapability.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapability.java
index 8fc8311..8b0d71c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapability.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapability.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
@@ -7,17 +7,13 @@
 package org.apache.hadoop.hive.metastore.api;
 
 
-import java.util.Map;
-import java.util.HashMap;
-import org.apache.thrift.TEnum;
-
 public enum ClientCapability implements org.apache.thrift.TEnum {
   TEST_CAPABILITY(1),
   INSERT_ONLY_TABLES(2);
 
   private final int value;
 
-  private ClientCapability(int value) {
+  ClientCapability(int value) {
     this.value = value;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AcidEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
new file mode 100644
index 0000000..1512ffb
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
@@ -0,0 +1,95 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+
+
+/**
+ * It handles cleanup of dropped partition/table/database in ACID related metastore tables
+ */
+public class AcidEventListener extends MetaStoreEventListener {
+
+  private TxnStore txnHandler;
+  private Configuration conf;
+
+  public AcidEventListener(Configuration configuration) {
+    super(configuration);
+    conf = configuration;
+  }
+
+  @Override
+  public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
+    // We can loop thru all the tables to check if they are ACID first and then perform cleanup,
+    // but it's more efficient to unconditionally perform cleanup for the database, especially
+    // when there are a lot of tables
+    txnHandler = getTxnHandler();
+    txnHandler.cleanupRecords(HiveObjectType.DATABASE, dbEvent.getDatabase(), null, null);
+  }
+
+  @Override
+  public void onDropTable(DropTableEvent tableEvent)  throws MetaException {
+    if (TxnUtils.isAcidTable(tableEvent.getTable())) {
+      txnHandler = getTxnHandler();
+      txnHandler.cleanupRecords(HiveObjectType.TABLE, null, tableEvent.getTable(), null);
+    }
+  }
+
+  @Override
+  public void onDropPartition(DropPartitionEvent partitionEvent)  throws MetaException {
+    if (TxnUtils.isAcidTable(partitionEvent.getTable())) {
+      txnHandler = getTxnHandler();
+      txnHandler.cleanupRecords(HiveObjectType.PARTITION, null, partitionEvent.getTable(),
+          partitionEvent.getPartitionIterator());
+    }
+  }
+
+  private TxnStore getTxnHandler() {
+    boolean hackOn = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) ||
+        MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEZ_TEST);
+    String origTxnMgr = null;
+    boolean origConcurrency = false;
+
+    // Since TxnUtils.getTxnStore calls TxnHandler.setConf -> checkQFileTestHack -> TxnDbUtil.setConfValues,
+    // which may change the values of below two entries, we need to avoid pulluting the original values
+    if (hackOn) {
+      origTxnMgr = MetastoreConf.getVar(conf, ConfVars.HIVE_TXN_MANAGER);
+      origConcurrency = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY);
+    }
+
+    txnHandler = TxnUtils.getTxnStore(conf);
+
+    // Set them back
+    if (hackOn) {
+      MetastoreConf.setVar(conf, ConfVars.HIVE_TXN_MANAGER, origTxnMgr);
+      MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, origConcurrency);
+    }
+
+    return txnHandler;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
new file mode 100644
index 0000000..fc0b4d7
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
@@ -0,0 +1,195 @@
+/*
+ * 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;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+/**
+ * Interface for Alter Table and Alter Partition code
+ */
+public interface AlterHandler extends Configurable {
+
+  /**
+   * @deprecated As of release 2.2.0. Replaced by {@link #alterTable(RawStore, Warehouse, String,
+   * String, Table, EnvironmentContext, IHMSHandler)}
+   *
+   * handles alter table, the changes could be cascaded to partitions if applicable
+   *
+   * @param msdb
+   *          object to get metadata
+   * @param wh
+   *          Hive Warehouse where table data is stored
+   * @param dbname
+   *          database of the table being altered
+   * @param name
+   *          original name of the table being altered. same as
+   *          <i>newTable.tableName</i> if alter op is not a rename.
+   * @param newTable
+   *          new table object
+   * @throws InvalidOperationException
+   *           thrown if the newTable object is invalid
+   * @throws MetaException
+   *           thrown if there is any other error
+   */
+  @Deprecated
+  void alterTable(RawStore msdb, Warehouse wh, String dbname,
+    String name, Table newTable, EnvironmentContext envContext)
+      throws InvalidOperationException, MetaException;
+
+  /**
+   * handles alter table, the changes could be cascaded to partitions if applicable
+   *
+   * @param msdb
+   *          object to get metadata
+   * @param wh
+   *          Hive Warehouse where table data is stored
+   * @param dbname
+   *          database of the table being altered
+   * @param name
+   *          original name of the table being altered. same as
+   *          <i>newTable.tableName</i> if alter op is not a rename.
+   * @param newTable
+   *          new table object
+   * @param handler
+   *          HMSHandle object (required to log event notification)
+   * @throws InvalidOperationException
+   *           thrown if the newTable object is invalid
+   * @throws MetaException
+   *           thrown if there is any other error
+   */
+  void alterTable(RawStore msdb, Warehouse wh, String dbname,
+      String name, Table newTable, EnvironmentContext envContext,
+      IHMSHandler handler) throws InvalidOperationException, MetaException;
+
+  /**
+   * @deprecated As of release 2.2.0.  Replaced by {@link #alterPartition(RawStore, Warehouse, String,
+   * String, List, Partition, EnvironmentContext, IHMSHandler)}
+   *
+   * handles alter partition
+   *
+   * @param msdb
+   *          object to get metadata
+   * @param wh
+   * @param dbname
+   *          database of the partition being altered
+   * @param name
+   *          table of the partition being altered
+   * @param part_vals
+   *          original values of the partition being altered
+   * @param new_part
+   *          new partition object
+   * @return the altered partition
+   * @throws InvalidOperationException
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   */
+  @Deprecated
+  Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
+    final String name, final List<String> part_vals, final Partition new_part,
+    EnvironmentContext environmentContext)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
+
+  /**
+   * handles alter partition
+   *
+   * @param msdb
+   *          object to get metadata
+   * @param wh
+   * @param dbname
+   *          database of the partition being altered
+   * @param name
+   *          table of the partition being altered
+   * @param part_vals
+   *          original values of the partition being altered
+   * @param new_part
+   *          new partition object
+   * @param handler
+   *          HMSHandle object (required to log event notification)
+   * @return the altered partition
+   * @throws InvalidOperationException
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   */
+  Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
+    final String name, final List<String> part_vals, final Partition new_part, EnvironmentContext environmentContext,
+    IHMSHandler handler)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
+
+  /**
+   * @deprecated As of release 2.2.0. Replaced by {@link #alterPartitions(RawStore, Warehouse, String,
+   * String, List, EnvironmentContext, IHMSHandler)}
+   *
+   * handles alter partitions
+   *
+   * @param msdb
+   *          object to get metadata
+   * @param wh
+   * @param dbname
+   *          database of the partition being altered
+   * @param name
+   *          table of the partition being altered
+   * @param new_parts
+   *          new partition list
+   * @return the altered partition list
+   * @throws InvalidOperationException
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   */
+  @Deprecated
+  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh,
+    final String dbname, final String name, final List<Partition> new_parts,
+    EnvironmentContext environmentContext)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
+
+  /**
+   * handles alter partitions
+   *
+   * @param msdb
+   *          object to get metadata
+   * @param wh
+   * @param dbname
+   *          database of the partition being altered
+   * @param name
+   *          table of the partition being altered
+   * @param new_parts
+   *          new partition list
+   * @param handler
+   *          HMSHandle object (required to log event notification)
+   * @return the altered partition list
+   * @throws InvalidOperationException
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   */
+  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh,
+    final String dbname, final String name, final List<Partition> new_parts,
+    EnvironmentContext environmentContext,IHMSHandler handler)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
new file mode 100644
index 0000000..0b8b310
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
@@ -0,0 +1,124 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+
+import org.apache.hadoop.fs.LocatedFileStatus;
+
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.utils.HdfsUtils;
+
+public class FileMetadataManager {
+  private static final Log LOG = LogFactory.getLog(FileMetadataManager.class);
+
+  private final RawStore tlms;
+  private final ExecutorService threadPool;
+  private final Configuration conf;
+
+  private final class CacheUpdateRequest implements Callable<Void> {
+    FileMetadataExprType type;
+    String location;
+
+    public CacheUpdateRequest(FileMetadataExprType type, String location) {
+      this.type = type;
+      this.location = location;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      try {
+        cacheMetadata(type, location);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      } catch (Exception ex) {
+        // Nobody can see this exception on the threadpool; just log it.
+        LOG.error("Failed to cache file metadata in background for " + type + ", " + location, ex);
+      }
+      return null;
+    }
+  }
+
+  public FileMetadataManager(RawStore tlms, Configuration conf) {
+    this.tlms = tlms;
+    this.conf = conf;
+    int numThreads = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.FILE_METADATA_THREADS);
+    this.threadPool = Executors.newFixedThreadPool(numThreads,
+        new ThreadFactoryBuilder().setNameFormat("File-Metadata-%d").setDaemon(true).build());
+  }
+
+  public void queueCacheMetadata(String location, FileMetadataExprType type) {
+    threadPool.submit(new CacheUpdateRequest(type, location));
+  }
+
+  private void cacheMetadata(FileMetadataExprType type, String location)
+      throws MetaException, IOException, InterruptedException {
+    Path path = new Path(location);
+    FileSystem fs = path.getFileSystem(conf);
+    List<Path> files;
+    if (!fs.isDirectory(path)) {
+      files = Lists.newArrayList(path);
+    } else {
+      files = new ArrayList<>();
+      RemoteIterator<LocatedFileStatus> iter = fs.listFiles(path, true);
+      while (iter.hasNext()) {
+        // TODO: use fileId right from the list after HDFS-7878; or get dfs client and do it
+        LocatedFileStatus lfs = iter.next();
+        if (lfs.isDirectory()) continue;
+        files.add(lfs.getPath());
+      }
+    }
+    for (Path file : files) {
+      long fileId;
+      // TODO: use the other HdfsUtils here
+      if (!(fs instanceof DistributedFileSystem)) return;
+      try {
+        fileId = HdfsUtils.getFileId(fs, Path.getPathWithoutSchemeAndAuthority(file).toString());
+      } catch (UnsupportedOperationException ex) {
+        LOG.error("Cannot cache file metadata for " + location + "; "
+            + fs.getClass().getCanonicalName() + " does not support fileId");
+        return;
+      }
+      LOG.info("Caching file metadata for " + file + " (file ID " + fileId + ")");
+      file = HdfsUtils.getFileIdPath(fs, file, fileId);
+      tlms.getFileMetadataHandler(type).cacheFileMetadata(fileId, fs, file);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
new file mode 100644
index 0000000..1ee6d97
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HMSMetricsListener.java
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+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.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Report metrics of metadata added, deleted by this Hive Metastore.
+ */
+public class HMSMetricsListener extends MetaStoreEventListener {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(HMSMetricsListener.class);
+
+  private Counter createdDatabases, deletedDatabases, createdTables, deletedTables, createdParts,
+      deletedParts;
+
+  public HMSMetricsListener(Configuration config) {
+    super(config);
+    createdDatabases = Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_DATABASES);
+    deletedDatabases = Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_DATABASES);
+    createdTables = Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_TABLES);
+    deletedTables = Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_TABLES);
+    createdParts = Metrics.getOrCreateCounter(MetricsConstants.CREATE_TOTAL_PARTITIONS);
+    deletedParts = Metrics.getOrCreateCounter(MetricsConstants.DELETE_TOTAL_PARTITIONS);
+  }
+
+  @Override
+  public void onCreateDatabase(CreateDatabaseEvent dbEvent) throws MetaException {
+    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_DATABASES).incrementAndGet();
+    createdDatabases.inc();
+  }
+
+  @Override
+  public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
+    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_DATABASES).decrementAndGet();
+    deletedDatabases.inc();
+  }
+
+  @Override
+  public void onCreateTable(CreateTableEvent tableEvent) throws MetaException {
+    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_TABLES).incrementAndGet();
+    createdTables.inc();
+  }
+
+  @Override
+  public void onDropTable(DropTableEvent tableEvent) throws MetaException {
+    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_TABLES).decrementAndGet();
+    deletedTables.inc();
+  }
+
+  @Override
+  public void onDropPartition(DropPartitionEvent partitionEvent) throws MetaException {
+    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_PARTITIONS).decrementAndGet();
+    deletedParts.inc();
+  }
+
+  @Override
+  public void onAddPartition(AddPartitionEvent partitionEvent) throws MetaException {
+    Metrics.getOrCreateGauge(MetricsConstants.TOTAL_PARTITIONS).incrementAndGet();
+    createdParts.inc();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
new file mode 100644
index 0000000..ccadac1
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -0,0 +1,897 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+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;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * Hive specific implementation of alter
+ */
+public class HiveAlterHandler implements AlterHandler {
+
+  protected Configuration hiveConf;
+  private static final Logger LOG = LoggerFactory.getLogger(HiveAlterHandler.class
+      .getName());
+
+  @Override
+  public Configuration getConf() {
+    return hiveConf;
+  }
+
+  @Override
+  @SuppressWarnings("nls")
+  public void setConf(Configuration conf) {
+    hiveConf = conf;
+  }
+
+  @Override
+  public void alterTable(RawStore msdb, Warehouse wh, String dbname,
+    String name, Table newt, EnvironmentContext environmentContext)
+      throws InvalidOperationException, MetaException {
+    alterTable(msdb, wh, dbname, name, newt, environmentContext, null);
+  }
+
+  @Override
+  public void alterTable(RawStore msdb, Warehouse wh, String dbname,
+      String name, Table newt, EnvironmentContext environmentContext,
+      IHMSHandler handler) throws InvalidOperationException, MetaException {
+    name = name.toLowerCase();
+    dbname = dbname.toLowerCase();
+
+    final boolean cascade = environmentContext != null
+        && environmentContext.isSetProperties()
+        && StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(
+            StatsSetupConst.CASCADE));
+    if (newt == null) {
+      throw new InvalidOperationException("New table is invalid: " + newt);
+    }
+
+    String newTblName = newt.getTableName().toLowerCase();
+    String newDbName = newt.getDbName().toLowerCase();
+
+    if (!MetaStoreUtils.validateName(newTblName, hiveConf)) {
+      throw new InvalidOperationException(newTblName + " is not a valid object name");
+    }
+    String validate = MetaStoreUtils.validateTblColumns(newt.getSd().getCols());
+    if (validate != null) {
+      throw new InvalidOperationException("Invalid column " + validate);
+    }
+
+    Path srcPath = null;
+    FileSystem srcFs;
+    Path destPath = null;
+    FileSystem destFs = null;
+
+    boolean success = false;
+    boolean dataWasMoved = false;
+    Table oldt;
+    List<TransactionalMetaStoreEventListener> transactionalListeners = null;
+    if (handler != null) {
+      transactionalListeners = handler.getTransactionalListeners();
+    }
+
+    try {
+      boolean rename = false;
+      boolean isPartitionedTable = false;
+      List<Partition> parts;
+
+      // check if table with the new name already exists
+      if (!newTblName.equals(name) || !newDbName.equals(dbname)) {
+        if (msdb.getTable(newDbName, newTblName) != null) {
+          throw new InvalidOperationException("new table " + newDbName
+              + "." + newTblName + " already exists");
+        }
+        rename = true;
+      }
+
+      msdb.openTransaction();
+      // get old table
+      oldt = msdb.getTable(dbname, name);
+      if (oldt == null) {
+        throw new InvalidOperationException("table " + dbname + "." + name + " doesn't exist");
+      }
+
+      if (oldt.getPartitionKeysSize() != 0) {
+        isPartitionedTable = true;
+      }
+
+      // Views derive the column type from the base table definition.  So the view definition
+      // can be altered to change the column types.  The column type compatibility checks should
+      // be done only for non-views.
+      if (MetastoreConf.getBoolVar(hiveConf,
+            MetastoreConf.ConfVars.DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES) &&
+          !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())) {
+        // Throws InvalidOperationException if the new column types are not
+        // compatible with the current column types.
+        checkColTypeChangeCompatible(oldt.getSd().getCols(), newt.getSd().getCols());
+      }
+
+      //check that partition keys have not changed, except for virtual views
+      //however, allow the partition comments to change
+      boolean partKeysPartiallyEqual = checkPartialPartKeysEqual(oldt.getPartitionKeys(),
+          newt.getPartitionKeys());
+
+      if(!oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())){
+        if (!partKeysPartiallyEqual) {
+          throw new InvalidOperationException("partition keys can not be changed.");
+        }
+      }
+
+      // rename needs change the data location and move the data to the new location corresponding
+      // to the new name if:
+      // 1) the table is not a virtual view, and
+      // 2) the table is not an external table, and
+      // 3) the user didn't change the default location (or new location is empty), and
+      // 4) the table was not initially created with a specified location
+      if (rename
+          && !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())
+          && (oldt.getSd().getLocation().compareTo(newt.getSd().getLocation()) == 0
+            || StringUtils.isEmpty(newt.getSd().getLocation()))
+          && !MetaStoreUtils.isExternalTable(oldt)) {
+        Database olddb = msdb.getDatabase(dbname);
+        // if a table was created in a user specified location using the DDL like
+        // create table tbl ... location ...., it should be treated like an external table
+        // in the table rename, its data location should not be changed. We can check
+        // if the table directory was created directly under its database directory to tell
+        // if it is such a table
+        srcPath = new Path(oldt.getSd().getLocation());
+        String oldtRelativePath = (new Path(olddb.getLocationUri()).toUri())
+            .relativize(srcPath.toUri()).toString();
+        boolean tableInSpecifiedLoc = !oldtRelativePath.equalsIgnoreCase(name)
+            && !oldtRelativePath.equalsIgnoreCase(name + Path.SEPARATOR);
+
+        if (!tableInSpecifiedLoc) {
+          srcFs = wh.getFs(srcPath);
+
+          // get new location
+          Database db = msdb.getDatabase(newDbName);
+          Path databasePath = constructRenamedPath(wh.getDatabasePath(db), srcPath);
+          destPath = new Path(databasePath, newTblName);
+          destFs = wh.getFs(destPath);
+
+          newt.getSd().setLocation(destPath.toString());
+
+          // check that destination does not exist otherwise we will be
+          // overwriting data
+          // check that src and dest are on the same file system
+          if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
+            throw new InvalidOperationException("table new location " + destPath
+                + " is on a different file system than the old location "
+                + srcPath + ". This operation is not supported");
+          }
+
+          try {
+            if (destFs.exists(destPath)) {
+              throw new InvalidOperationException("New location for this table "
+                  + newDbName + "." + newTblName + " already exists : " + destPath);
+            }
+            // check that src exists and also checks permissions necessary, rename src to dest
+            if (srcFs.exists(srcPath) && wh.renameDir(srcPath, destPath, true)) {
+              dataWasMoved = true;
+            }
+          } catch (IOException | MetaException e) {
+            LOG.error("Alter Table operation for " + dbname + "." + name + " failed.", e);
+            throw new InvalidOperationException("Alter Table operation for " + dbname + "." + name +
+                " failed to move data due to: '" + getSimpleMessage(e)
+                + "' See hive log file for details.");
+          }
+        }
+
+        if (isPartitionedTable) {
+          String oldTblLocPath = srcPath.toUri().getPath();
+          String newTblLocPath = dataWasMoved ? destPath.toUri().getPath() : null;
+
+          // also the location field in partition
+          parts = msdb.getPartitions(dbname, name, -1);
+          Map<Partition, ColumnStatistics> columnStatsNeedUpdated = new HashMap<>();
+          for (Partition part : parts) {
+            String oldPartLoc = part.getSd().getLocation();
+            if (dataWasMoved && oldPartLoc.contains(oldTblLocPath)) {
+              URI oldUri = new Path(oldPartLoc).toUri();
+              String newPath = oldUri.getPath().replace(oldTblLocPath, newTblLocPath);
+              Path newPartLocPath = new Path(oldUri.getScheme(), oldUri.getAuthority(), newPath);
+              part.getSd().setLocation(newPartLocPath.toString());
+            }
+            part.setDbName(newDbName);
+            part.setTableName(newTblName);
+            ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
+                part.getValues(), part.getSd().getCols(), oldt, part);
+            if (colStats != null) {
+              columnStatsNeedUpdated.put(part, colStats);
+            }
+          }
+          msdb.alterTable(dbname, name, newt);
+          // alterPartition is only for changing the partition location in the table rename
+          if (dataWasMoved) {
+            for (Partition part : parts) {
+              msdb.alterPartition(newDbName, newTblName, part.getValues(), part);
+            }
+          }
+
+          for (Entry<Partition, ColumnStatistics> partColStats : columnStatsNeedUpdated.entrySet()) {
+            ColumnStatistics newPartColStats = partColStats.getValue();
+            newPartColStats.getStatsDesc().setDbName(newDbName);
+            newPartColStats.getStatsDesc().setTableName(newTblName);
+            msdb.updatePartitionColumnStatistics(newPartColStats, partColStats.getKey().getValues());
+          }
+        } else {
+          alterTableUpdateTableColumnStats(msdb, oldt, newt);
+        }
+      } else {
+        // operations other than table rename
+        if (MetaStoreUtils.requireCalStats(hiveConf, null, null, newt, environmentContext) &&
+            !isPartitionedTable) {
+          Database db = msdb.getDatabase(newDbName);
+          // Update table stats. For partitioned table, we update stats in alterPartition()
+          MetaStoreUtils.updateTableStatsFast(db, newt, wh, false, true, environmentContext);
+        }
+
+        if (cascade && isPartitionedTable) {
+          //Currently only column related changes can be cascaded in alter table
+          if(!MetaStoreUtils.areSameColumns(oldt.getSd().getCols(), newt.getSd().getCols())) {
+            parts = msdb.getPartitions(dbname, name, -1);
+            for (Partition part : parts) {
+              List<FieldSchema> oldCols = part.getSd().getCols();
+              part.getSd().setCols(newt.getSd().getCols());
+              ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
+                  part.getValues(), oldCols, oldt, part);
+              assert(colStats == null);
+              msdb.alterPartition(dbname, name, part.getValues(), part);
+            }
+            msdb.alterTable(dbname, name, newt);
+          } else {
+            LOG.warn("Alter table does not cascade changes to its partitions.");
+          }
+        } else {
+          alterTableUpdateTableColumnStats(msdb, oldt, newt);
+        }
+      }
+
+      if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
+        if (oldt.getDbName().equalsIgnoreCase(newt.getDbName())) {
+          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                  EventMessage.EventType.ALTER_TABLE,
+                  new AlterTableEvent(oldt, newt, false, true, handler),
+                  environmentContext);
+        } else {
+          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                  EventMessage.EventType.DROP_TABLE,
+                  new DropTableEvent(oldt, true, false, handler),
+                  environmentContext);
+          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                  EventMessage.EventType.CREATE_TABLE,
+                  new CreateTableEvent(newt, true, handler),
+                  environmentContext);
+          if (isPartitionedTable) {
+            parts = msdb.getPartitions(newt.getDbName(), newt.getTableName(), -1);
+            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventMessage.EventType.ADD_PARTITION,
+                    new AddPartitionEvent(newt, parts, true, handler),
+                    environmentContext);
+          }
+        }
+      }
+      // commit the changes
+      success = msdb.commitTransaction();
+    } catch (InvalidObjectException e) {
+      LOG.debug("Failed to get object from Metastore ", e);
+      throw new InvalidOperationException(
+          "Unable to change partition or table."
+              + " Check metastore logs for detailed stack." + e.getMessage());
+    } catch (InvalidInputException e) {
+        LOG.debug("Accessing Metastore failed due to invalid input ", e);
+        throw new InvalidOperationException(
+            "Unable to change partition or table."
+                + " Check metastore logs for detailed stack." + e.getMessage());
+    } catch (NoSuchObjectException e) {
+      LOG.debug("Object not found in metastore ", e);
+      throw new InvalidOperationException(
+          "Unable to change partition or table. Database " + dbname + " does not exist"
+              + " Check metastore logs for detailed stack." + e.getMessage());
+    } finally {
+      if (!success) {
+        LOG.error("Failed to alter table " + dbname + "." + name);
+        msdb.rollbackTransaction();
+        if (dataWasMoved) {
+          try {
+            if (destFs.exists(destPath)) {
+              if (!destFs.rename(destPath, srcPath)) {
+                LOG.error("Failed to restore data from " + destPath + " to " + srcPath
+                    + " in alter table failure. Manual restore is needed.");
+              }
+            }
+          } catch (IOException e) {
+            LOG.error("Failed to restore data from " + destPath + " to " + srcPath
+                +  " in alter table failure. Manual restore is needed.");
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * MetaException that encapsulates error message from RemoteException from hadoop RPC which wrap
+   * the stack trace into e.getMessage() which makes logs/stack traces confusing.
+   * @param ex
+   * @return
+   */
+  String getSimpleMessage(Exception ex) {
+    if(ex instanceof MetaException) {
+      String msg = ex.getMessage();
+      if(msg == null || !msg.contains("\n")) {
+        return msg;
+      }
+      return msg.substring(0, msg.indexOf('\n'));
+    }
+    return ex.getMessage();
+  }
+
+  @Override
+  public Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
+    final String name, final List<String> part_vals, final Partition new_part,
+    EnvironmentContext environmentContext)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
+    return alterPartition(msdb, wh, dbname, name, part_vals, new_part, environmentContext, null);
+  }
+
+  @Override
+  public Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
+    final String name, final List<String> part_vals, final Partition new_part,
+    EnvironmentContext environmentContext, IHMSHandler handler)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
+    boolean success = false;
+    Partition oldPart;
+    List<TransactionalMetaStoreEventListener> transactionalListeners = null;
+    if (handler != null) {
+      transactionalListeners = handler.getTransactionalListeners();
+    }
+
+    // Set DDL time to now if not specified
+    if (new_part.getParameters() == null ||
+        new_part.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
+        Integer.parseInt(new_part.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
+      new_part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
+          .currentTimeMillis() / 1000));
+    }
+
+    Table tbl = msdb.getTable(dbname, name);
+    if (tbl == null) {
+      throw new InvalidObjectException(
+          "Unable to alter partition because table or database does not exist.");
+    }
+
+    //alter partition
+    if (part_vals == null || part_vals.size() == 0) {
+      try {
+        msdb.openTransaction();
+        oldPart = msdb.getPartition(dbname, name, new_part.getValues());
+        if (MetaStoreUtils.requireCalStats(hiveConf, oldPart, new_part, tbl, environmentContext)) {
+          // if stats are same, no need to update
+          if (MetaStoreUtils.isFastStatsSame(oldPart, new_part)) {
+            MetaStoreUtils.updateBasicState(environmentContext, new_part.getParameters());
+          } else {
+            MetaStoreUtils.updatePartitionStatsFast(new_part, wh, false, true, environmentContext);
+          }
+        }
+
+        // PartitionView does not have SD. We do not need update its column stats
+        if (oldPart.getSd() != null) {
+          updateOrGetPartitionColumnStats(msdb, dbname, name, new_part.getValues(),
+              oldPart.getSd().getCols(), tbl, new_part);
+        }
+        msdb.alterPartition(dbname, name, new_part.getValues(), new_part);
+        if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
+          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                                                EventMessage.EventType.ALTER_PARTITION,
+                                                new AlterPartitionEvent(oldPart, new_part, tbl, false, true, handler),
+                                                environmentContext);
+
+
+        }
+        success = msdb.commitTransaction();
+      } catch (InvalidObjectException e) {
+        throw new InvalidOperationException("alter is not possible");
+      } catch (NoSuchObjectException e){
+        //old partition does not exist
+        throw new InvalidOperationException("alter is not possible");
+      } finally {
+        if(!success) {
+          msdb.rollbackTransaction();
+        }
+      }
+      return oldPart;
+    }
+
+    //rename partition
+    String oldPartLoc;
+    String newPartLoc;
+    Path srcPath = null;
+    Path destPath = null;
+    FileSystem srcFs;
+    FileSystem destFs = null;
+    boolean dataWasMoved = false;
+    try {
+      msdb.openTransaction();
+      try {
+        oldPart = msdb.getPartition(dbname, name, part_vals);
+      } catch (NoSuchObjectException e) {
+        // this means there is no existing partition
+        throw new InvalidObjectException(
+            "Unable to rename partition because old partition does not exist");
+      }
+
+      Partition check_part;
+      try {
+        check_part = msdb.getPartition(dbname, name, new_part.getValues());
+      } catch(NoSuchObjectException e) {
+        // this means there is no existing partition
+        check_part = null;
+      }
+
+      if (check_part != null) {
+        throw new AlreadyExistsException("Partition already exists:" + dbname + "." + name + "." +
+            new_part.getValues());
+      }
+
+      // when renaming a partition, we should update
+      // 1) partition SD Location
+      // 2) partition column stats if there are any because of part_name field in HMS table PART_COL_STATS
+      // 3) rename the partition directory if it is not an external table
+      if (!tbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
+        try {
+          // if tbl location is available use it
+          // else derive the tbl location from database location
+          destPath = wh.getPartitionPath(msdb.getDatabase(dbname), tbl, new_part.getValues());
+          destPath = constructRenamedPath(destPath, new Path(new_part.getSd().getLocation()));
+        } catch (NoSuchObjectException e) {
+          LOG.debug("Didn't find object in metastore ", e);
+          throw new InvalidOperationException(
+            "Unable to change partition or table. Database " + dbname + " does not exist"
+              + " Check metastore logs for detailed stack." + e.getMessage());
+        }
+
+        if (destPath != null) {
+          newPartLoc = destPath.toString();
+          oldPartLoc = oldPart.getSd().getLocation();
+          LOG.info("srcPath:" + oldPartLoc);
+          LOG.info("descPath:" + newPartLoc);
+          srcPath = new Path(oldPartLoc);
+          srcFs = wh.getFs(srcPath);
+          destFs = wh.getFs(destPath);
+          // check that src and dest are on the same file system
+          if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
+            throw new InvalidOperationException("New table location " + destPath
+              + " is on a different file system than the old location "
+              + srcPath + ". This operation is not supported.");
+          }
+
+          try {
+            if (srcFs.exists(srcPath)) {
+              if (newPartLoc.compareTo(oldPartLoc) != 0 && destFs.exists(destPath)) {
+                throw new InvalidOperationException("New location for this table "
+                  + tbl.getDbName() + "." + tbl.getTableName()
+                  + " already exists : " + destPath);
+              }
+              //if destPath's parent path doesn't exist, we should mkdir it
+              Path destParentPath = destPath.getParent();
+              if (!wh.mkdirs(destParentPath)) {
+                  throw new MetaException("Unable to create path " + destParentPath);
+              }
+
+              //rename the data directory
+              wh.renameDir(srcPath, destPath, true);
+              LOG.info("Partition directory rename from " + srcPath + " to " + destPath + " done.");
+              dataWasMoved = true;
+            }
+          } catch (IOException e) {
+            LOG.error("Cannot rename partition directory from " + srcPath + " to " + destPath, e);
+            throw new InvalidOperationException("Unable to access src or dest location for partition "
+                + tbl.getDbName() + "." + tbl.getTableName() + " " + new_part.getValues());
+          } catch (MetaException me) {
+            LOG.error("Cannot rename partition directory from " + srcPath + " to " + destPath, me);
+            throw me;
+          }
+
+          new_part.getSd().setLocation(newPartLoc);
+        }
+      } else {
+        new_part.getSd().setLocation(oldPart.getSd().getLocation());
+      }
+
+      if (MetaStoreUtils.requireCalStats(hiveConf, oldPart, new_part, tbl, environmentContext)) {
+        MetaStoreUtils.updatePartitionStatsFast(new_part, wh, false, true, environmentContext);
+      }
+
+      String newPartName = Warehouse.makePartName(tbl.getPartitionKeys(), new_part.getValues());
+      ColumnStatistics cs = updateOrGetPartitionColumnStats(msdb, dbname, name, oldPart.getValues(),
+          oldPart.getSd().getCols(), tbl, new_part);
+      msdb.alterPartition(dbname, name, part_vals, new_part);
+      if (cs != null) {
+        cs.getStatsDesc().setPartName(newPartName);
+        try {
+          msdb.updatePartitionColumnStatistics(cs, new_part.getValues());
+        } catch (InvalidInputException iie) {
+          throw new InvalidOperationException("Unable to update partition stats in table rename." + iie);
+        } catch (NoSuchObjectException nsoe) {
+          // It is ok, ignore
+        }
+      }
+
+      if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
+        MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                                              EventMessage.EventType.ALTER_PARTITION,
+                                              new AlterPartitionEvent(oldPart, new_part, tbl, false, true, handler),
+                                              environmentContext);
+      }
+
+      success = msdb.commitTransaction();
+    } finally {
+      if (!success) {
+        LOG.error("Failed to rename a partition. Rollback transaction");
+        msdb.rollbackTransaction();
+        if (dataWasMoved) {
+          LOG.error("Revert the data move in renaming a partition.");
+          try {
+            if (destFs.exists(destPath)) {
+              wh.renameDir(destPath, srcPath, false);
+            }
+          } catch (MetaException me) {
+            LOG.error("Failed to restore partition data from " + destPath + " to " + srcPath
+                +  " in alter partition failure. Manual restore is needed.");
+          } catch (IOException ioe) {
+            LOG.error("Failed to restore partition data from " + destPath + " to " + srcPath
+                +  " in alter partition failure. Manual restore is needed.");
+          }
+        }
+      }
+    }
+    return oldPart;
+  }
+
+  @Override
+  public List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String dbname,
+    final String name, final List<Partition> new_parts,
+    EnvironmentContext environmentContext)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
+    return alterPartitions(msdb, wh, dbname, name, new_parts, environmentContext, null);
+  }
+
+  @Override
+  public List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String dbname,
+    final String name, final List<Partition> new_parts, EnvironmentContext environmentContext,
+    IHMSHandler handler)
+      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
+    List<Partition> oldParts = new ArrayList<>();
+    List<List<String>> partValsList = new ArrayList<>();
+    List<TransactionalMetaStoreEventListener> transactionalListeners = null;
+    if (handler != null) {
+      transactionalListeners = handler.getTransactionalListeners();
+    }
+
+    Table tbl = msdb.getTable(dbname, name);
+    if (tbl == null) {
+      throw new InvalidObjectException(
+          "Unable to alter partitions because table or database does not exist.");
+    }
+
+    boolean success = false;
+    try {
+      msdb.openTransaction();
+      for (Partition tmpPart: new_parts) {
+        // Set DDL time to now if not specified
+        if (tmpPart.getParameters() == null ||
+            tmpPart.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
+            Integer.parseInt(tmpPart.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
+          tmpPart.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
+              .currentTimeMillis() / 1000));
+        }
+
+        Partition oldTmpPart = msdb.getPartition(dbname, name, tmpPart.getValues());
+        oldParts.add(oldTmpPart);
+        partValsList.add(tmpPart.getValues());
+
+        if (MetaStoreUtils.requireCalStats(hiveConf, oldTmpPart, tmpPart, tbl, environmentContext)) {
+          // Check if stats are same, no need to update
+          if (MetaStoreUtils.isFastStatsSame(oldTmpPart, tmpPart)) {
+            MetaStoreUtils.updateBasicState(environmentContext, tmpPart.getParameters());
+          } else {
+            MetaStoreUtils.updatePartitionStatsFast(tmpPart, wh, false, true, environmentContext);
+          }
+        }
+
+        // PartitionView does not have SD and we do not need to update its column stats
+        if (oldTmpPart.getSd() != null) {
+          updateOrGetPartitionColumnStats(msdb, dbname, name, oldTmpPart.getValues(),
+              oldTmpPart.getSd().getCols(), tbl, tmpPart);
+        }
+      }
+
+      msdb.alterPartitions(dbname, name, partValsList, new_parts);
+      Iterator<Partition> oldPartsIt = oldParts.iterator();
+      for (Partition newPart : new_parts) {
+        Partition oldPart;
+        if (oldPartsIt.hasNext()) {
+          oldPart = oldPartsIt.next();
+        } else {
+          throw new InvalidOperationException("Missing old partition corresponding to new partition " +
+              "when invoking MetaStoreEventListener for alterPartitions event.");
+        }
+
+        if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
+          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                                                EventMessage.EventType.ALTER_PARTITION,
+                                                new AlterPartitionEvent(oldPart, newPart, tbl, false, true, handler));
+        }
+      }
+
+      success = msdb.commitTransaction();
+    } catch (InvalidObjectException | NoSuchObjectException e) {
+      throw new InvalidOperationException("Alter partition operation failed: " + e);
+    } finally {
+      if(!success) {
+        msdb.rollbackTransaction();
+      }
+    }
+
+    return oldParts;
+  }
+
+  private boolean checkPartialPartKeysEqual(List<FieldSchema> oldPartKeys,
+      List<FieldSchema> newPartKeys) {
+    //return true if both are null, or false if one is null and the other isn't
+    if (newPartKeys == null || oldPartKeys == null) {
+      return oldPartKeys == newPartKeys;
+    }
+    if (oldPartKeys.size() != newPartKeys.size()) {
+      return false;
+    }
+    Iterator<FieldSchema> oldPartKeysIter = oldPartKeys.iterator();
+    Iterator<FieldSchema> newPartKeysIter = newPartKeys.iterator();
+    FieldSchema oldFs;
+    FieldSchema newFs;
+    while (oldPartKeysIter.hasNext()) {
+      oldFs = oldPartKeysIter.next();
+      newFs = newPartKeysIter.next();
+      // Alter table can change the type of partition key now.
+      // So check the column name only.
+      if (!oldFs.getName().equals(newFs.getName())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Uses the scheme and authority of the object's current location and the path constructed
+   * using the object's new name to construct a path for the object's new location.
+   */
+  private Path constructRenamedPath(Path defaultNewPath, Path currentPath) {
+    URI currentUri = currentPath.toUri();
+
+    return new Path(currentUri.getScheme(), currentUri.getAuthority(),
+        defaultNewPath.toUri().getPath());
+  }
+
+  @VisibleForTesting
+  void alterTableUpdateTableColumnStats(RawStore msdb, Table oldTable, Table newTable)
+      throws MetaException, InvalidObjectException {
+    String dbName = oldTable.getDbName().toLowerCase();
+    String tableName = org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier(oldTable.getTableName());
+    String newDbName = newTable.getDbName().toLowerCase();
+    String newTableName = org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier(newTable.getTableName());
+
+    try {
+      List<FieldSchema> oldCols = oldTable.getSd().getCols();
+      List<FieldSchema> newCols = newTable.getSd().getCols();
+      List<ColumnStatisticsObj> newStatsObjs = new ArrayList<>();
+      ColumnStatistics colStats = null;
+      boolean updateColumnStats = true;
+
+      // Nothing to update if everything is the same
+        if (newDbName.equals(dbName) &&
+            newTableName.equals(tableName) &&
+            MetaStoreUtils.columnsIncludedByNameType(oldCols, newCols)) {
+          updateColumnStats = false;
+        }
+
+        if (updateColumnStats) {
+          List<String> oldColNames = new ArrayList<>(oldCols.size());
+          for (FieldSchema oldCol : oldCols) {
+            oldColNames.add(oldCol.getName());
+          }
+
+          // Collect column stats which need to be rewritten and remove old stats
+          colStats = msdb.getTableColumnStatistics(dbName, tableName, oldColNames);
+          if (colStats == null) {
+            updateColumnStats = false;
+          } else {
+            List<ColumnStatisticsObj> statsObjs = colStats.getStatsObj();
+            if (statsObjs != null) {
+              List<String> deletedCols = new ArrayList<>();
+              for (ColumnStatisticsObj statsObj : statsObjs) {
+                boolean found = false;
+                for (FieldSchema newCol : newCols) {
+                  if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
+                      && statsObj.getColType().equalsIgnoreCase(newCol.getType())) {
+                    found = true;
+                    break;
+                  }
+                }
+
+                if (found) {
+                  if (!newDbName.equals(dbName) || !newTableName.equals(tableName)) {
+                    msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
+                    newStatsObjs.add(statsObj);
+                    deletedCols.add(statsObj.getColName());
+                  }
+                } else {
+                  msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
+                  deletedCols.add(statsObj.getColName());
+                }
+              }
+              StatsSetupConst.removeColumnStatsState(newTable.getParameters(), deletedCols);
+            }
+          }
+        }
+
+        // Change to new table and append stats for the new table
+        msdb.alterTable(dbName, tableName, newTable);
+        if (updateColumnStats && !newStatsObjs.isEmpty()) {
+          ColumnStatisticsDesc statsDesc = colStats.getStatsDesc();
+          statsDesc.setDbName(newDbName);
+          statsDesc.setTableName(newTableName);
+          colStats.setStatsObj(newStatsObjs);
+          msdb.updateTableColumnStatistics(colStats);
+        }
+    } catch (NoSuchObjectException nsoe) {
+      LOG.debug("Could not find db entry." + nsoe);
+    } catch (InvalidInputException e) {
+      //should not happen since the input were verified before passed in
+      throw new InvalidObjectException("Invalid inputs to update table column stats: " + e);
+    }
+  }
+
+  private ColumnStatistics updateOrGetPartitionColumnStats(
+      RawStore msdb, String dbname, String tblname, List<String> partVals,
+      List<FieldSchema> oldCols, Table table, Partition part)
+          throws MetaException, InvalidObjectException {
+    ColumnStatistics newPartsColStats = null;
+    try {
+      List<FieldSchema> newCols = part.getSd() == null ?
+          new ArrayList<>() : part.getSd().getCols();
+      String oldPartName = Warehouse.makePartName(table.getPartitionKeys(), partVals);
+      String newPartName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
+      boolean rename = !part.getDbName().equals(dbname) || !part.getTableName().equals(tblname)
+          || !oldPartName.equals(newPartName);
+
+      // do not need to update column stats if alter partition is not for rename or changing existing columns
+      if (!rename && MetaStoreUtils.columnsIncludedByNameType(oldCols, newCols)) {
+        return newPartsColStats;
+      }
+      List<String> oldColNames = new ArrayList<>(oldCols.size());
+      for (FieldSchema oldCol : oldCols) {
+        oldColNames.add(oldCol.getName());
+      }
+      List<String> oldPartNames = Lists.newArrayList(oldPartName);
+      List<ColumnStatistics> partsColStats = msdb.getPartitionColumnStatistics(dbname, tblname,
+          oldPartNames, oldColNames);
+      assert (partsColStats.size() <= 1);
+      for (ColumnStatistics partColStats : partsColStats) { //actually only at most one loop
+        List<ColumnStatisticsObj> newStatsObjs = new ArrayList<>();
+        List<ColumnStatisticsObj> statsObjs = partColStats.getStatsObj();
+        List<String> deletedCols = new ArrayList<>();
+        for (ColumnStatisticsObj statsObj : statsObjs) {
+          boolean found =false;
+          for (FieldSchema newCol : newCols) {
+            if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
+                && statsObj.getColType().equalsIgnoreCase(newCol.getType())) {
+              found = true;
+              break;
+            }
+          }
+          if (found) {
+            if (rename) {
+              msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
+                  partVals, statsObj.getColName());
+              newStatsObjs.add(statsObj);
+            }
+          } else {
+            msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
+                partVals, statsObj.getColName());
+            deletedCols.add(statsObj.getColName());
+          }
+        }
+        StatsSetupConst.removeColumnStatsState(part.getParameters(), deletedCols);
+        if (!newStatsObjs.isEmpty()) {
+          partColStats.setStatsObj(newStatsObjs);
+          newPartsColStats = partColStats;
+        }
+      }
+    } catch (NoSuchObjectException nsoe) {
+      // ignore this exception, actually this exception won't be thrown from getPartitionColumnStatistics
+    } catch (InvalidInputException iie) {
+      throw new InvalidObjectException("Invalid input to delete partition column stats." + iie);
+    }
+
+    return newPartsColStats;
+  }
+
+  private void checkColTypeChangeCompatible(List<FieldSchema> oldCols, List<FieldSchema> newCols)
+      throws InvalidOperationException {
+    List<String> incompatibleCols = new ArrayList<>();
+    int maxCols = Math.min(oldCols.size(), newCols.size());
+    for (int i = 0; i < maxCols; i++) {
+      if (!ColumnType.areColTypesCompatible(
+          ColumnType.getTypeName(oldCols.get(i).getType()),
+          ColumnType.getTypeName(newCols.get(i).getType()))) {
+        incompatibleCols.add(newCols.get(i).getName());
+      }
+    }
+    if (!incompatibleCols.isEmpty()) {
+      throw new InvalidOperationException(
+          "The following columns have types incompatible with the existing " +
+              "columns in their respective positions :\n" +
+              org.apache.commons.lang.StringUtils.join(incompatibleCols, ',')
+      );
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
index 633b3c7..85bdc4d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
@@ -18,11 +18,70 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 
+import java.util.List;
+
+/**
+ * An interface wrapper for HMSHandler.  This interface contains methods that need to be
+ * called by internal classes but that are not part of the thrift interface.
+ */
+@InterfaceAudience.Private
 public interface IHMSHandler extends ThriftHiveMetastore.Iface, Configurable {
 
   void init() throws MetaException;
+
+  /**
+   * Get the id of the thread of this handler.
+   * @return thread id
+   */
+  int getThreadId();
+
+  /**
+   * Get a reference to the underlying RawStore.
+   * @return the RawStore instance.
+   * @throws MetaException if the creation of a new RawStore object is necessary but fails.
+   */
+  RawStore getMS() throws MetaException;
+
+  /**
+   * Get a reference to Hive's warehouse object (the class that does all the physical operations).
+   * @return Warehouse instance.
+   */
+  Warehouse getWh();
+
+  /**
+   * Equivalent to get_database, but does not write to audit logs, or fire pre-event listeners.
+   * Meant to be used for internal hive classes that don't use the thrift interface.
+   * @param name database name
+   * @return database object
+   * @throws NoSuchObjectException If the database does not exist.
+   * @throws MetaException If another error occurs.
+   */
+  Database get_database_core(final String name) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Equivalent of get_table, but does not log audits and fire pre-event listener.
+   * Meant to be used for calls made by other hive classes, that are not using the
+   * thrift interface.
+   * @param dbname database name
+   * @param name table name
+   * @return Table object
+   * @throws NoSuchObjectException If the table does not exist.
+   * @throws MetaException  If another error occurs.
+   */
+  Table get_table_core(final String dbname, final String name) throws MetaException,
+      NoSuchObjectException;
+
+  /**
+   * Get a list of all transactional listeners.
+   * @return list of listeners.
+   */
+  List<TransactionalMetaStoreEventListener> getTransactionalListeners();
 }


[08/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java
deleted file mode 100644
index d254ad9..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.messaging;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-
-public abstract class DropPartitionMessage extends EventMessage {
-
-  protected DropPartitionMessage() {
-    super(EventType.DROP_PARTITION);
-  }
-
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  public abstract Table getTableObj() throws Exception;
-
-  public abstract List<Map<String, String>> getPartitions ();
-
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null)
-      throw new IllegalStateException("Table name unset.");
-    if (getPartitions() == null)
-      throw new IllegalStateException("Partition-list unset.");
-    return super.checkValid();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java
deleted file mode 100644
index 7aed21b..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-
-public abstract class DropTableMessage extends EventMessage {
-
-  protected DropTableMessage() {
-    super(EventType.DROP_TABLE);
-  }
-
-  /**
-   * Getter for the name of the table being dropped.
-   * @return Table-name (String).
-   */
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  public abstract Table getTableObj() throws Exception;
-
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null)
-      throw new IllegalStateException("Table name unset.");
-    return super.checkValid();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
deleted file mode 100644
index 3b37cb3..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
- * 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.messaging;
-
-/**
- * Class representing messages emitted when Metastore operations are done.
- * (E.g. Creation and deletion of databases, tables and partitions.)
- */
-public abstract class EventMessage {
-
-  /**
-   * Enumeration of all supported types of Metastore operations.
-   */
-  public static enum EventType {
-
-    CREATE_DATABASE(MessageFactory.CREATE_DATABASE_EVENT),
-    DROP_DATABASE(MessageFactory.DROP_DATABASE_EVENT),
-    CREATE_TABLE(MessageFactory.CREATE_TABLE_EVENT),
-    DROP_TABLE(MessageFactory.DROP_TABLE_EVENT),
-    ADD_PARTITION(MessageFactory.ADD_PARTITION_EVENT),
-    DROP_PARTITION(MessageFactory.DROP_PARTITION_EVENT),
-    ALTER_TABLE(MessageFactory.ALTER_TABLE_EVENT),
-    ALTER_PARTITION(MessageFactory.ALTER_PARTITION_EVENT),
-    INSERT(MessageFactory.INSERT_EVENT),
-    CREATE_FUNCTION(MessageFactory.CREATE_FUNCTION_EVENT),
-    DROP_FUNCTION(MessageFactory.DROP_FUNCTION_EVENT),
-    CREATE_INDEX(MessageFactory.CREATE_INDEX_EVENT),
-    DROP_INDEX(MessageFactory.DROP_INDEX_EVENT),
-    ALTER_INDEX(MessageFactory.ALTER_INDEX_EVENT),
-    ADD_PRIMARYKEY(MessageFactory.ADD_PRIMARYKEY_EVENT),
-    ADD_FOREIGNKEY(MessageFactory.ADD_FOREIGNKEY_EVENT),
-    ADD_UNIQUECONSTRAINT(MessageFactory.ADD_UNIQUECONSTRAINT_EVENT),
-    ADD_NOTNULLCONSTRAINT(MessageFactory.ADD_NOTNULLCONSTRAINT_EVENT),
-    DROP_CONSTRAINT(MessageFactory.DROP_CONSTRAINT_EVENT);
-
-    private String typeString;
-
-    EventType(String typeString) {
-      this.typeString = typeString;
-    }
-
-    @Override
-    public String toString() { return typeString; }
-  }
-
-  protected EventType eventType;
-
-  protected EventMessage(EventType eventType) {
-    this.eventType = eventType;
-  }
-
-  public EventType getEventType() {
-    return eventType;
-  }
-
-  /**
-   * Getter for HCatalog Server's URL.
-   * (This is where the event originates from.)
-   * @return HCatalog Server's URL (String).
-   */
-  public abstract String getServer();
-
-  /**
-   * Getter for the Kerberos principal of the HCatalog service.
-   * @return HCatalog Service Principal (String).
-   */
-  public abstract String getServicePrincipal();
-
-  /**
-   * Getter for the name of the Database on which the Metastore operation is done.
-   * @return Database-name (String).
-   */
-  public abstract String getDB();
-
-  /**
-   * Getter for the timestamp associated with the operation.
-   * @return Timestamp (Long - seconds since epoch).
-   */
-  public abstract Long   getTimestamp();
-
-  /**
-   * Class invariant. Checked after construction or deserialization.
-   */
-  public EventMessage checkValid() {
-    if (getServer() == null || getServicePrincipal() == null)
-      throw new IllegalStateException("Server-URL/Service-Principal shouldn't be null.");
-    if (getEventType() == null)
-      throw new IllegalStateException("Event-type unset.");
-    if (getDB() == null)
-      throw new IllegalArgumentException("DB-name unset.");
-
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java
deleted file mode 100644
index 01fc0f2..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-/**
- * HCat message sent when an insert is done to a table or partition.
- */
-public abstract class InsertMessage extends EventMessage {
-
-  protected InsertMessage() {
-    super(EventType.INSERT);
-  }
-
-  /**
-   * Getter for the name of the table being insert into.
-   * @return Table-name (String).
-   */
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  /**
-   * Getter for the replace flag being insert into/overwrite
-   * @return Replace flag to represent INSERT INTO or INSERT OVERWRITE (Boolean).
-   */
-  public abstract boolean isReplace();
-
-  /**
-   * Get list of file name and checksum created as a result of this DML operation
-   *
-   * @return The iterable of files
-   */
-  public abstract Iterable<String> getFiles();
-
-  /**
-   * Get the table object associated with the insert
-   *
-   * @return The Json format of Table object
-   */
-  public abstract Table getTableObj() throws Exception;
-
-  /**
-   * Get the partition object associated with the insert
-   *
-   * @return The Json format of Partition object if the table is partitioned else return null.
-   */
-  public abstract Partition getPtnObj() throws Exception;
-
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null)
-      throw new IllegalStateException("Table name unset.");
-    return super.checkValid();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
deleted file mode 100644
index 50a2728..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * 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.messaging;
-
-/**
- * Interface for converting HCat events from String-form back to EventMessage instances.
- */
-public abstract class MessageDeserializer {
-
-  /**
-   * Method to construct EventMessage from string.
-   */
-  public EventMessage getEventMessage(String eventTypeString, String messageBody) {
-
-    switch (EventMessage.EventType.valueOf(eventTypeString)) {
-    case CREATE_DATABASE:
-      return getCreateDatabaseMessage(messageBody);
-    case DROP_DATABASE:
-      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);
-    case CREATE_FUNCTION:
-      return getCreateFunctionMessage(messageBody);
-    case DROP_FUNCTION:
-      return getDropFunctionMessage(messageBody);
-    case CREATE_INDEX:
-      return getCreateIndexMessage(messageBody);
-    case DROP_INDEX:
-      return getDropIndexMessage(messageBody);
-    case ALTER_INDEX:
-      return getAlterIndexMessage(messageBody);
-    case INSERT:
-      return getInsertMessage(messageBody);
-    case ADD_PRIMARYKEY:
-      return getAddPrimaryKeyMessage(messageBody);
-    case ADD_FOREIGNKEY:
-      return getAddForeignKeyMessage(messageBody);
-    case ADD_UNIQUECONSTRAINT:
-      return getAddUniqueConstraintMessage(messageBody);
-    case ADD_NOTNULLCONSTRAINT:
-      return getAddNotNullConstraintMessage(messageBody);
-    case DROP_CONSTRAINT:
-      return getDropConstraintMessage(messageBody);
-    default:
-      throw new IllegalArgumentException("Unsupported event-type: " + eventTypeString);
-    }
-  }
-
-  /**
-   * Method to de-serialize CreateDatabaseMessage instance.
-   */
-  public abstract CreateDatabaseMessage getCreateDatabaseMessage(String messageBody);
-
-  /**
-   * Method to de-serialize DropDatabaseMessage instance.
-   */
-  public abstract DropDatabaseMessage getDropDatabaseMessage(String messageBody);
-
-  /**
-   * Method to de-serialize CreateTableMessage instance.
-   */
-  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);
-
-  /**
-   * Method to de-serialize AddPartitionMessage instance.
-   */
-  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);
-
-  /**
-   * Method to de-serialize CreateFunctionMessage instance.
-   */
-  public abstract CreateFunctionMessage getCreateFunctionMessage(String messageBody);
-
-  /**
-   * Method to de-serialize DropFunctionMessage instance.
-   */
-  public abstract DropFunctionMessage getDropFunctionMessage(String messageBody);
-
-  /**
-   * Method to de-serialize CreateIndexMessage instance.
-   */
-  public abstract CreateIndexMessage getCreateIndexMessage(String messageBody);
-
-  /**
-   * Method to de-serialize DropIndexMessage instance.
-   */
-  public abstract DropIndexMessage getDropIndexMessage(String messageBody);
-
-  /**
-   * Method to de-serialize AlterIndexMessage instance.
-   */
-  public abstract AlterIndexMessage getAlterIndexMessage(String messageBody);
-
-  /**
-   * Method to deserialize InsertMessage
-   * @param messageBody the message in serialized form
-   * @return message in object form
-   */
-  public abstract InsertMessage getInsertMessage(String messageBody);
-
-  /**
-   * Method to de-serialize AddPrimaryKeyMessage instance.
-   */
-  public abstract AddPrimaryKeyMessage getAddPrimaryKeyMessage(String messageBody);
-
-  /**
-   * Method to de-serialize AddForeignKeyMessage instance.
-   */
-  public abstract AddForeignKeyMessage getAddForeignKeyMessage(String messageBody);
-
-  /**
-   * Method to de-serialize AddUniqueConstraintMessage instance.
-   */
-  public abstract AddUniqueConstraintMessage getAddUniqueConstraintMessage(String messageBody);
-
-  /**
-   * Method to de-serialize AddNotNullConstraintMessage instance.
-   */
-  public abstract AddNotNullConstraintMessage getAddNotNullConstraintMessage(String messageBody);
-
-  /**
-   * Method to de-serialize DropConstraintMessage instance.
-   */
-  public abstract DropConstraintMessage getDropConstraintMessage(String messageBody);
-
-  // Protection against construction.
-  protected MessageDeserializer() {}
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
deleted file mode 100644
index e639e48..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.common.JavaUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.util.ReflectionUtils;
-
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * Abstract Factory for the construction of HCatalog message instances.
- */
-public abstract class MessageFactory {
-
-  // Common name constants for event messages
-  public static final String ADD_PARTITION_EVENT = "ADD_PARTITION";
-  public static final String ALTER_PARTITION_EVENT = "ALTER_PARTITION";
-  public static final String DROP_PARTITION_EVENT = "DROP_PARTITION";
-  public static final String CREATE_TABLE_EVENT = "CREATE_TABLE";
-  public static final String ALTER_TABLE_EVENT = "ALTER_TABLE";
-  public static final String DROP_TABLE_EVENT = "DROP_TABLE";
-  public static final String CREATE_DATABASE_EVENT = "CREATE_DATABASE";
-  public static final String DROP_DATABASE_EVENT = "DROP_DATABASE";
-  public static final String INSERT_EVENT = "INSERT";
-  public static final String CREATE_FUNCTION_EVENT = "CREATE_FUNCTION";
-  public static final String DROP_FUNCTION_EVENT = "DROP_FUNCTION";
-  public static final String CREATE_INDEX_EVENT = "CREATE_INDEX";
-  public static final String DROP_INDEX_EVENT = "DROP_INDEX";
-  public static final String ALTER_INDEX_EVENT = "ALTER_INDEX";
-  public static final String ADD_PRIMARYKEY_EVENT = "ADD_PRIMARYKEY";
-  public static final String ADD_FOREIGNKEY_EVENT = "ADD_FOREIGNKEY";
-  public static final String ADD_UNIQUECONSTRAINT_EVENT = "ADD_UNIQUECONSTRAINT";
-  public static final String ADD_NOTNULLCONSTRAINT_EVENT = "ADD_NOTNULLCONSTRAINT";
-  public static final String DROP_CONSTRAINT_EVENT = "DROP_CONSTRAINT";
-
-  private static MessageFactory instance = null;
-
-  protected static final HiveConf hiveConf = new HiveConf();
-  static {
-    hiveConf.addResource("hive-site.xml");
-  }
-
-  // This parameter is retained for legacy reasons, in case someone implemented custom
-  // factories. This, however, should not be the case, since this api was intended to
-  // be internal-only, and we should manage the jms and json implementations without
-  // needing this parameter. Marking as deprecated, for removal by 2.4 - see corresponding
-  // note on the getDeserializer(String,String) method
-  @Deprecated
-  private static final String CONF_LABEL_HCAT_MESSAGE_FACTORY_IMPL_PREFIX = "hcatalog.message.factory.impl.";
-
-  protected static final String MS_SERVER_URL = hiveConf.get(HiveConf.ConfVars.METASTOREURIS.name(), "");
-  protected static final String MS_SERVICE_PRINCIPAL = hiveConf.get(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL.name(), "");
-
-  /**
-   * Getter for MessageFactory instance.
-   */
-  public static MessageFactory getInstance() {
-    if (instance == null) {
-      instance =
-          getInstance(hiveConf.get(HiveConf.ConfVars.METASTORE_EVENT_MESSAGE_FACTORY.varname));
-    }
-    return instance;
-  }
-
-  private static MessageFactory getInstance(String className) {
-    try {
-      return (MessageFactory)ReflectionUtils.newInstance(JavaUtils.loadClass(className), hiveConf);
-    }
-    catch (ClassNotFoundException classNotFound) {
-      throw new IllegalStateException("Could not construct MessageFactory implementation: ", classNotFound);
-    }
-  }
-
-  /**
-   * Getter for MessageDeserializer, corresponding to the specified format and version.
-   * @param format Serialization format for notifications.
-   * @param version Version of serialization format (currently ignored.)
-   * @return MessageDeserializer.
-   */
-  public static MessageDeserializer getDeserializer(String format,
-                            String version) {
-    return getInstance(hiveConf.get(CONF_LABEL_HCAT_MESSAGE_FACTORY_IMPL_PREFIX + format,
-        HiveConf.ConfVars.METASTORE_EVENT_MESSAGE_FACTORY.varname)).getDeserializer();
-    // Note : The reason this method exists outside the no-arg getDeserializer method is in
-    // case there is a user-implemented MessageFactory that's used, and some the messages
-    // are in an older format and the rest in another. Then, what MessageFactory is default
-    // is irrelevant, we should always use the one that was used to create it to deserialize.
-    //
-    // There exist only 2 implementations of this - json and jms
-    //
-    // Additional note : rather than as a config parameter, does it make sense to have
-    // this use jdbc-like semantics that each MessageFactory made available register
-    // itself for discoverability? Might be worth pursuing.
-  }
-
-  public abstract MessageDeserializer getDeserializer();
-
-  /**
-   * Getter for message-format.
-   */
-  public abstract String getMessageFormat();
-
-  /**
-   * Factory method for CreateDatabaseMessage.
-   * @param db The Database being added.
-   * @return CreateDatabaseMessage instance.
-   */
-  public abstract CreateDatabaseMessage buildCreateDatabaseMessage(Database db);
-
-  /**
-   * Factory method for DropDatabaseMessage.
-   * @param db The Database being dropped.
-   * @return DropDatabaseMessage instance.
-   */
-  public abstract DropDatabaseMessage buildDropDatabaseMessage(Database db);
-
-  /**
-   * Factory method for CreateTableMessage.
-   * @param table The Table being created.
-   * @param files Iterator of files
-   * @return CreateTableMessage instance.
-   */
-  public abstract CreateTableMessage buildCreateTableMessage(Table table, Iterator<String> files);
-
-  /**
-   * 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
-   * @param isTruncateOp Flag to denote truncate table
-   * @return
-   */
-  public abstract AlterTableMessage buildAlterTableMessage(Table before, Table after, boolean isTruncateOp);
-
-  /**
-   * Factory method for DropTableMessage.
-   * @param table The Table being dropped.
-   * @return DropTableMessage instance.
-   */
-  public abstract DropTableMessage buildDropTableMessage(Table table);
-
-    /**
-     * Factory method for AddPartitionMessage.
-     * @param table The Table to which the partitions are added.
-     * @param partitions The iterator to set of Partitions being added.
-     * @param partitionFiles The iterator of partition files
-     * @return AddPartitionMessage instance.
-     */
-  public abstract AddPartitionMessage buildAddPartitionMessage(Table table, Iterator<Partition> partitions,
-      Iterator<PartitionFiles> partitionFiles);
-
-  /**
-   * Factory method for building AlterPartitionMessage
-   * @param table The table in which the partition is being altered
-   * @param before The partition before it was altered
-   * @param after The partition after it was altered
-   * @param isTruncateOp Flag to denote truncate partition
-   * @return a new AlterPartitionMessage
-   */
-  public abstract AlterPartitionMessage buildAlterPartitionMessage(Table table, Partition before,
-                                                                   Partition after, boolean isTruncateOp);
-
-  /**
-   * Factory method for DropPartitionMessage.
-   * @param table The Table from which the partition is dropped.
-   * @param partitions The set of partitions being dropped.
-   * @return DropPartitionMessage instance.
-   */
-  public abstract DropPartitionMessage buildDropPartitionMessage(Table table, Iterator<Partition> partitions);
-
-  /**
-   * Factory method for CreateFunctionMessage.
-   * @param fn The Function being added.
-   * @return CreateFunctionMessage instance.
-   */
-  public abstract CreateFunctionMessage buildCreateFunctionMessage(Function fn);
-
-  /**
-   * Factory method for DropFunctionMessage.
-   * @param fn The Function being dropped.
-   * @return DropFunctionMessage instance.
-   */
-  public abstract DropFunctionMessage buildDropFunctionMessage(Function fn);
-
-  /**
-   * Factory method for CreateIndexMessage.
-   * @param idx The Index being added.
-   * @return CreateIndexMessage instance.
-   */
-  public abstract CreateIndexMessage buildCreateIndexMessage(Index idx);
-
-  /**
-   * Factory method for DropIndexMessage.
-   * @param idx The Index being dropped.
-   * @return DropIndexMessage instance.
-   */
-  public abstract DropIndexMessage buildDropIndexMessage(Index idx);
-
-  /**
-   * Factory method for AlterIndexMessage.
-   * @param before The index before the alter
-   * @param after The index after the alter
-   * @return AlterIndexMessage
-   */
-  public abstract AlterIndexMessage buildAlterIndexMessage(Index before, Index after);
-
-  /**
-   * Factory method for building insert message
-   *
-   * @param tableObj Table object where the insert occurred in
-   * @param ptnObj Partition object where the insert occurred in, may be null if
-   *          the insert was done into a non-partitioned table
-   * @param replace Flag to represent if INSERT OVERWRITE or INSERT INTO
-   * @param files Iterator of file created
-   * @return instance of InsertMessage
-   */
-  public abstract InsertMessage buildInsertMessage(Table tableObj, Partition ptnObj,
-                                                   boolean replace, Iterator<String> files);
-
-  /***
-   * Factory method for building add primary key message
-   *
-   * @param pks list of primary keys
-   * @return instance of AddPrimaryKeyMessage
-   */
-  public abstract AddPrimaryKeyMessage buildAddPrimaryKeyMessage(List<SQLPrimaryKey> pks);
-
-  /***
-   * Factory method for building add foreign key message
-   *
-   * @param fks list of foreign keys
-   * @return instance of AddForeignKeyMessage
-   */
-  public abstract AddForeignKeyMessage buildAddForeignKeyMessage(List<SQLForeignKey> fks);
-
-  /***
-   * Factory method for building add unique constraint message
-   *
-   * @param uks list of unique constraints
-   * @return instance of SQLUniqueConstraint
-   */
-  public abstract AddUniqueConstraintMessage buildAddUniqueConstraintMessage(List<SQLUniqueConstraint> uks);
-
-  /***
-   * Factory method for building add not null constraint message
-   *
-   * @param nns list of not null constraints
-   * @return instance of SQLNotNullConstraint
-   */
-  public abstract AddNotNullConstraintMessage buildAddNotNullConstraintMessage(List<SQLNotNullConstraint> nns);
-
-  /***
-   * Factory method for building drop constraint message
-   * @param dbName
-   * @param tableName
-   * @param constraintName
-   * @return
-   */
-  public abstract DropConstraintMessage buildDropConstraintMessage(String dbName, String tableName,
-      String constraintName);
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java
deleted file mode 100644
index 4fd7f8c..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.messaging;
-
-import java.util.Iterator;
-import java.util.List;
-
-import com.google.common.collect.Lists;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-public class PartitionFiles {
-
-  @JsonProperty
-  private String partitionName;
-  @JsonProperty
-  private List<String> files;
-
-  public PartitionFiles(String partitionName, Iterator<String> files) {
-    this.partitionName = partitionName;
-    this.files = Lists.newArrayList(files);
-  }
-
-  public PartitionFiles() {
-  }
-
-  public String getPartitionName() {
-    return partitionName;
-  }
-
-  public void setPartitionName(String partitionName) {
-    this.partitionName = partitionName;
-  }
-
-  public Iterable<String> getFiles() {
-    return files;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java
deleted file mode 100644
index 9c04c15..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * 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.messaging.json;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON implementation of AddForeignKeyMessage
- */
-public class JSONAddForeignKeyMessage extends AddForeignKeyMessage {
-
-  @JsonProperty
-  String server, servicePrincipal;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  List<String> foreignKeyListJson;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONAddForeignKeyMessage() {
-  }
-
-  public JSONAddForeignKeyMessage(String server, String servicePrincipal, List<SQLForeignKey> fks,
-      Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.timestamp = timestamp;
-    this.foreignKeyListJson = new ArrayList<String>();
-    try {
-      for (SQLForeignKey pk : fks) {
-        foreignKeyListJson.add(JSONMessageFactory.createForeignKeyObjJson(pk));
-      }
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return null;
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public List<SQLForeignKey> getForeignKeys() throws Exception {
-    List<SQLForeignKey> fks = new ArrayList<SQLForeignKey>();
-    for (String pkJson : foreignKeyListJson) {
-      fks.add((SQLForeignKey)JSONMessageFactory.getTObj(pkJson, SQLForeignKey.class));
-    }
-    return fks;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java
deleted file mode 100644
index 95ff23a..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * 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.messaging.json;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
-import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-public class JSONAddNotNullConstraintMessage extends AddNotNullConstraintMessage {
-  @JsonProperty
-  String server, servicePrincipal;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  List<String> notNullConstraintListJson;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONAddNotNullConstraintMessage() {
-  }
-
-  public JSONAddNotNullConstraintMessage(String server, String servicePrincipal, List<SQLNotNullConstraint> nns,
-      Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.timestamp = timestamp;
-    this.notNullConstraintListJson = new ArrayList<String>();
-    try {
-      for (SQLNotNullConstraint nn : nns) {
-        notNullConstraintListJson.add(JSONMessageFactory.createNotNullConstraintObjJson(nn));
-      }
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return null;
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public List<SQLNotNullConstraint> getNotNullConstraints() throws Exception {
-    List<SQLNotNullConstraint> nns = new ArrayList<SQLNotNullConstraint>();
-    for (String nnJson : notNullConstraintListJson) {
-      nns.add((SQLNotNullConstraint)JSONMessageFactory.getTObj(nnJson, SQLNotNullConstraint.class));
-    }
-    return nns;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java
deleted file mode 100644
index db3431e..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * 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.messaging.json;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
-import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import javax.annotation.Nullable;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * JSON implementation of AddPartitionMessage.
- */
-public class JSONAddPartitionMessage extends AddPartitionMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, table, tableType, tableObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  List<Map<String, String>> partitions;
-
-  @JsonProperty
-  List<String> partitionListJson;
-
-  @JsonProperty
-  List<PartitionFiles> partitionFiles;
-
-  /**
-   * Default Constructor. Required for Jackson.
-   */
-  public JSONAddPartitionMessage() {
-  }
-
-  /**
-   * Note that we get an Iterator rather than an Iterable here: so we can only walk thru the list once
-   */
-  public JSONAddPartitionMessage(String server, String servicePrincipal, Table tableObj,
-      Iterator<Partition> partitionsIterator, Iterator<PartitionFiles> partitionFileIter,
-      Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = tableObj.getDbName();
-    this.table = tableObj.getTableName();
-    this.tableType = tableObj.getTableType();
-    this.timestamp = timestamp;
-    partitions = new ArrayList<Map<String, String>>();
-    partitionListJson = new ArrayList<String>();
-    Partition partitionObj;
-    try {
-      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
-      while (partitionsIterator.hasNext()) {
-        partitionObj = partitionsIterator.next();
-        partitions.add(JSONMessageFactory.getPartitionKeyValues(tableObj, partitionObj));
-        partitionListJson.add(JSONMessageFactory.createPartitionObjJson(partitionObj));
-      }
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-    this.partitionFiles = Lists.newArrayList(partitionFileIter);
-    checkValid();
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return db;
-  }
-
-  @Override
-  public String getTable() {
-    return table;
-  }
-
-  @Override
-  public String getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public Table getTableObj() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public List<Map<String, String>> getPartitions() {
-    return partitions;
-  }
-
-  @Override
-  public Iterable<Partition> getPartitionObjs() throws Exception {
-    // glorified cast from Iterable<TBase> to Iterable<Partition>
-    return Iterables.transform(
-        JSONMessageFactory.getTObjs(partitionListJson,Partition.class),
-        new Function<Object, Partition>() {
-      @Nullable
-      @Override
-      public Partition apply(@Nullable Object input) {
-        return (Partition) input;
-      }
-    });
-  }
-
-  public String getTableObjJson() {
-    return tableObjJson;
-  }
-
-  public List<String> getPartitionListJson() {
-    return partitionListJson;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-
-  @Override
-  public Iterable<PartitionFiles> getPartitionFilesIter() {
-    return partitionFiles;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java
deleted file mode 100644
index 2551cbf..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * 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.messaging.json;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.messaging.AddPrimaryKeyMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON implementation of AddPrimaryKeyMessage
- */
-public class JSONAddPrimaryKeyMessage extends AddPrimaryKeyMessage {
-
-  @JsonProperty
-  String server, servicePrincipal;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  List<String> primaryKeyListJson;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONAddPrimaryKeyMessage() {
-  }
-
-  public JSONAddPrimaryKeyMessage(String server, String servicePrincipal, List<SQLPrimaryKey> pks,
-      Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.timestamp = timestamp;
-    this.primaryKeyListJson = new ArrayList<String>();
-    try {
-      for (SQLPrimaryKey pk : pks) {
-        primaryKeyListJson.add(JSONMessageFactory.createPrimaryKeyObjJson(pk));
-      }
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return null;
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public List<SQLPrimaryKey> getPrimaryKeys() throws Exception {
-    List<SQLPrimaryKey> pks = new ArrayList<SQLPrimaryKey>();
-    for (String pkJson : primaryKeyListJson) {
-      pks.add((SQLPrimaryKey)JSONMessageFactory.getTObj(pkJson, SQLPrimaryKey.class));
-    }
-    return pks;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java
deleted file mode 100644
index 37f6b81..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * 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.messaging.json;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
-import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-public class JSONAddUniqueConstraintMessage extends AddUniqueConstraintMessage {
-  @JsonProperty
-  String server, servicePrincipal;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  List<String> uniqueConstraintListJson;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONAddUniqueConstraintMessage() {
-  }
-
-  public JSONAddUniqueConstraintMessage(String server, String servicePrincipal, List<SQLUniqueConstraint> uks,
-      Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.timestamp = timestamp;
-    this.uniqueConstraintListJson = new ArrayList<String>();
-    try {
-      for (SQLUniqueConstraint uk : uks) {
-        uniqueConstraintListJson.add(JSONMessageFactory.createUniqueConstraintObjJson(uk));
-      }
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return null;
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public List<SQLUniqueConstraint> getUniqueConstraints() throws Exception {
-    List<SQLUniqueConstraint> uks = new ArrayList<SQLUniqueConstraint>();
-    for (String pkJson : uniqueConstraintListJson) {
-      uks.add((SQLUniqueConstraint)JSONMessageFactory.getTObj(pkJson, SQLUniqueConstraint.class));
-    }
-    return uks;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java
deleted file mode 100644
index 5dffdd1..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.messaging.AlterIndexMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON Implementation of AlterIndexMessage.
- */
-public class JSONAlterIndexMessage extends AlterIndexMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, beforeIndexObjJson, afterIndexObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONAlterIndexMessage() {}
-
-  public JSONAlterIndexMessage(String server, String servicePrincipal, Index before, Index after,
-                               Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = after.getDbName();
-    this.timestamp = timestamp;
-    try {
-      this.beforeIndexObjJson = JSONMessageFactory.createIndexObjJson(before);
-      this.afterIndexObjJson = JSONMessageFactory.createIndexObjJson(after);
-    } catch (TException ex) {
-      throw new IllegalArgumentException("Could not serialize Index object", ex);
-    }
-
-    checkValid();
-  }
-
-  @Override
-  public String getDB() { return db; }
-
-  @Override
-  public String getServer() { return server; }
-
-  @Override
-  public String getServicePrincipal() { return servicePrincipal; }
-
-  @Override
-  public Long getTimestamp() { return timestamp; }
-
-  public String getBeforeIndexObjJson() {
-    return beforeIndexObjJson;
-  }
-
-  public String getAfterIndexObjJson() {
-    return afterIndexObjJson;
-  }
-
-  @Override
-  public Index getIndexObjBefore() throws Exception {
-    return (Index)  JSONMessageFactory.getTObj(beforeIndexObjJson, Index.class);
-  }
-
-  @Override
-  public Index getIndexObjAfter() throws Exception {
-    return (Index)  JSONMessageFactory.getTObj(afterIndexObjJson, Index.class);
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
deleted file mode 100644
index 2991e08..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
-import org.apache.thrift.TException;
-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, tableType, tableObjJson;
-
-  @JsonProperty
-  String isTruncateOp;
-
-  @JsonProperty
-  Long timestamp;
-
-  @JsonProperty
-  Map<String, String> keyValues;
-
-  @JsonProperty
-  String partitionObjBeforeJson, partitionObjAfterJson;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONAlterPartitionMessage() {
-  }
-
-  public JSONAlterPartitionMessage(String server, String servicePrincipal, Table tableObj,
-      Partition partitionObjBefore, Partition partitionObjAfter, boolean isTruncateOp, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = tableObj.getDbName();
-    this.table = tableObj.getTableName();
-    this.tableType = tableObj.getTableType();
-    this.isTruncateOp = Boolean.toString(isTruncateOp);
-    this.timestamp = timestamp;
-    this.keyValues = JSONMessageFactory.getPartitionKeyValues(tableObj, partitionObjBefore);
-    try {
-      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
-      this.partitionObjBeforeJson = JSONMessageFactory.createPartitionObjJson(partitionObjBefore);
-      this.partitionObjAfterJson = JSONMessageFactory.createPartitionObjJson(partitionObjAfter);
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-    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 getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public boolean getIsTruncateOp() { return Boolean.parseBoolean(isTruncateOp); }
-
-  @Override
-  public Map<String, String> getKeyValues() {
-    return keyValues;
-  }
-
-  @Override
-  public Table getTableObj() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
-  }
-
-  @Override
-  public Partition getPtnObjBefore() throws Exception {
-    return (Partition) JSONMessageFactory.getTObj(partitionObjBeforeJson, Partition.class);
-  }
-
-  @Override
-  public Partition getPtnObjAfter() throws Exception {
-    return (Partition) JSONMessageFactory.getTObj(partitionObjAfterJson, Partition.class);
-  }
-
-  public String getTableObjJson() {
-    return tableObjJson;
-  }
-
-  public String getPartitionObjBeforeJson() {
-    return partitionObjBeforeJson;
-  }
-
-  public String getPartitionObjAfterJson() {
-    return partitionObjAfterJson;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
deleted file mode 100644
index 1df9c8b..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON alter table message
- */
-public class JSONAlterTableMessage extends AlterTableMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, table, tableType, tableObjBeforeJson, tableObjAfterJson;
-
-  @JsonProperty
-  String isTruncateOp;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONAlterTableMessage() {
-  }
-
-  public JSONAlterTableMessage(String server, String servicePrincipal, Table tableObjBefore, Table tableObjAfter,
-      boolean isTruncateOp, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = tableObjBefore.getDbName();
-    this.table = tableObjBefore.getTableName();
-    this.tableType = tableObjBefore.getTableType();
-    this.isTruncateOp = Boolean.toString(isTruncateOp);
-    this.timestamp = timestamp;
-    try {
-      this.tableObjBeforeJson = JSONMessageFactory.createTableObjJson(tableObjBefore);
-      this.tableObjAfterJson = JSONMessageFactory.createTableObjJson(tableObjAfter);
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-    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 getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public boolean getIsTruncateOp() { return Boolean.parseBoolean(isTruncateOp); }
-
-  @Override
-  public Table getTableObjBefore() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjBeforeJson,Table.class);
-  }
-
-  @Override
-  public Table getTableObjAfter() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjAfterJson,Table.class);
-  }
-
-  public String getTableObjBeforeJson() {
-    return tableObjBeforeJson;
-  }
-
-  public String getTableObjAfterJson() {
-    return tableObjAfterJson ;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java
deleted file mode 100644
index f8717b2..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON Implementation of CreateDatabaseMessage.
- */
-public class JSONCreateDatabaseMessage extends CreateDatabaseMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONCreateDatabaseMessage() {}
-
-  public JSONCreateDatabaseMessage(String server, String servicePrincipal, String db, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = db;
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  @Override
-  public String getDB() { return db; }
-
-  @Override
-  public String getServer() { return server; }
-
-  @Override
-  public String getServicePrincipal() { return servicePrincipal; }
-
-  @Override
-  public Long getTimestamp() { return timestamp; }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java
deleted file mode 100644
index 3646d85..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON Implementation of CreateFunctionMessage.
- */
-public class JSONCreateFunctionMessage extends CreateFunctionMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, functionObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONCreateFunctionMessage() {}
-
-  public JSONCreateFunctionMessage(String server, String servicePrincipal, Function fn, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = fn.getDbName();
-    this.timestamp = timestamp;
-    try {
-      this.functionObjJson = JSONMessageFactory.createFunctionObjJson(fn);
-    } catch (TException ex) {
-      throw new IllegalArgumentException("Could not serialize Function object", ex);
-    }
-    checkValid();
-  }
-
-  @Override
-  public String getDB() { return db; }
-
-  @Override
-  public String getServer() { return server; }
-
-  @Override
-  public String getServicePrincipal() { return servicePrincipal; }
-
-  @Override
-  public Long getTimestamp() { return timestamp; }
-
-  public String getFunctionObjJson() {
-    return functionObjJson;
-  }
-
-  @Override
-  public Function getFunctionObj() throws Exception {
-    return (Function) JSONMessageFactory.getTObj(functionObjJson,Function.class);
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java
deleted file mode 100644
index a2e2fc0..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.messaging.CreateIndexMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON Implementation of CreateIndexMessage.
- */
-public class JSONCreateIndexMessage extends CreateIndexMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, indexObjJson;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONCreateIndexMessage() {}
-
-  public JSONCreateIndexMessage(String server, String servicePrincipal, Index index, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = index.getDbName();
-    try {
-      this.indexObjJson = JSONMessageFactory.createIndexObjJson(index);
-    } catch (TException ex) {
-      throw new IllegalArgumentException("Could not serialize Index object", ex);
-    }
-
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  @Override
-  public String getDB() { return db; }
-
-  @Override
-  public String getServer() { return server; }
-
-  @Override
-  public String getServicePrincipal() { return servicePrincipal; }
-
-  @Override
-  public Long getTimestamp() { return timestamp; }
-
-  public String getIndexObjJson() {
-    return indexObjJson;
-  }
-
-  @Override
-  public Index getIndexObj() throws Exception {
-    return (Index)  JSONMessageFactory.getTObj(indexObjJson, Index.class);
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java
deleted file mode 100644
index 9dd7a74..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * 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.messaging.json;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import com.google.common.collect.Lists;
-
-/**
- * JSON implementation of CreateTableMessage.
- */
-public class JSONCreateTableMessage extends CreateTableMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, table, tableType, tableObjJson;
-  @JsonProperty
-  Long timestamp;
-  @JsonProperty
-  List<String> files;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONCreateTableMessage() {
-  }
-
-  public JSONCreateTableMessage(String server, String servicePrincipal, String db, String table,
-      String tableType, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = db;
-    this.table = table;
-    this.tableType = tableType;
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  public JSONCreateTableMessage(String server, String servicePrincipal, String db, String table,
-      Long timestamp) {
-    this(server, servicePrincipal, db, table, null, timestamp);
-  }
-
-  public JSONCreateTableMessage(String server, String servicePrincipal, Table tableObj,
-      Iterator<String> fileIter, Long timestamp) {
-    this(server, servicePrincipal, tableObj.getDbName(), tableObj.getTableName(),
-        tableObj.getTableType(), timestamp);
-    try {
-      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
-    } catch (TException e) {
-      throw new IllegalArgumentException("Could not serialize: ", e);
-    }
-    this.files = Lists.newArrayList(fileIter);
-  }
-
-  @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 getTableType() {
-    if (tableType != null) return tableType; else return "";
-  }
-
-  @Override
-  public Table getTableObj() throws Exception {
-    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
-  }
-
-  public String getTableObjJson() {
-    return tableObjJson;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-
-  @Override
-  public Iterable<String> getFiles() {
-    return files;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java
deleted file mode 100644
index 4d3422b..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON implementation of DropConstraintMessage
- */
-public class JSONDropConstraintMessage extends DropConstraintMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, dbName, tableName, constraintName;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, needed for Jackson.
-   */
-  public JSONDropConstraintMessage() {
-  }
-
-  public JSONDropConstraintMessage(String server, String servicePrincipal, String dbName,
-      String tableName, String constraintName, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.timestamp = timestamp;
-    this.dbName = dbName;
-    this.tableName = tableName;
-    this.constraintName = constraintName;
-  }
-
-  @Override
-  public String getServer() {
-    return server;
-  }
-
-  @Override
-  public String getServicePrincipal() {
-    return servicePrincipal;
-  }
-
-  @Override
-  public String getDB() {
-    return dbName;
-  }
-
-  @Override
-  public String getTable() {
-    return tableName;
-  }
-
-  @Override
-  public String getConstraint() {
-    return constraintName;
-  }
-
-  @Override
-  public Long getTimestamp() {
-    return timestamp;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    } catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java
deleted file mode 100644
index be17e6d..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON implementation of DropDatabaseMessage.
- */
-public class JSONDropDatabaseMessage extends DropDatabaseMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONDropDatabaseMessage() {}
-
-  public JSONDropDatabaseMessage(String server, String servicePrincipal, String db, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = db;
-    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 toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java
deleted file mode 100644
index b9ee4c4..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * 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.messaging.json;
-
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * JSON Implementation of CreateDatabaseMessage.
- */
-public class JSONDropFunctionMessage extends DropFunctionMessage {
-
-  @JsonProperty
-  String server, servicePrincipal, db, functionName;
-
-  @JsonProperty
-  Long timestamp;
-
-  /**
-   * Default constructor, required for Jackson.
-   */
-  public JSONDropFunctionMessage() {}
-
-  public JSONDropFunctionMessage(String server, String servicePrincipal, Function fn, Long timestamp) {
-    this.server = server;
-    this.servicePrincipal = servicePrincipal;
-    this.db = fn.getDbName();
-    this.functionName = fn.getFunctionName();
-    this.timestamp = timestamp;
-    checkValid();
-  }
-
-  @Override
-  public String getDB() { return db; }
-
-  @Override
-  public String getServer() { return server; }
-
-  @Override
-  public String getServicePrincipal() { return servicePrincipal; }
-
-  @Override
-  public Long getTimestamp() { return timestamp; }
-
-  @Override
-  public String toString() {
-    try {
-      return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
-      throw new IllegalArgumentException("Could not serialize: ", exception);
-    }
-  }
-
-  @Override
-  public String getFunctionName() {
-    return functionName;
-  }
-
-}
\ No newline at end of file


[05/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
new file mode 100644
index 0000000..fc4f4d7
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -0,0 +1,227 @@
+/*
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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.AddForeignKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
+import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
+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;
+import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropConstraintEvent;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
+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.InsertEvent;
+import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
+
+/**
+ * This abstract class needs to be extended to  provide implementation of actions that needs
+ * to be performed when a particular event occurs on a metastore. These methods
+ * are called whenever an event occurs on metastore. Status of the event whether
+ * it was successful or not is contained in container event object.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class MetaStoreEventListener implements Configurable {
+
+  private Configuration conf;
+
+  public MetaStoreEventListener(Configuration config){
+    this.conf = config;
+  }
+
+  /**
+   * @param tableEvent table event.
+   * @throws MetaException
+   */
+  public void onConfigChange(ConfigChangeEvent tableEvent) throws MetaException {
+  }
+
+  /**
+   * @param tableEvent table event.
+   * @throws MetaException
+   */
+  public void onCreateTable (CreateTableEvent tableEvent) throws MetaException {
+  }
+
+  /**
+   * @param tableEvent table event.
+   * @throws MetaException
+   */
+  public void onDropTable (DropTableEvent tableEvent)  throws MetaException {
+  }
+
+  /**
+   * @param tableEvent alter table event
+   * @throws MetaException
+   */
+  public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
+  }
+
+  /**
+   * @param partitionEvent add partition event
+   * @throws MetaException
+   */
+  public void onAddPartition (AddPartitionEvent partitionEvent) throws MetaException {
+  }
+
+  /**
+   * @param partitionEvent partition event
+   * @throws MetaException
+   */
+  public void onDropPartition (DropPartitionEvent partitionEvent)  throws MetaException {
+  }
+
+  /**
+   * @param partitionEvent partition event
+   * @throws MetaException
+   */
+  public void onAlterPartition (AlterPartitionEvent partitionEvent)  throws MetaException {
+  }
+
+  /**
+   * @param dbEvent database event
+   * @throws MetaException
+   */
+  public void onCreateDatabase (CreateDatabaseEvent dbEvent) throws MetaException {
+  }
+
+  /**
+   * @param dbEvent database event
+   * @throws MetaException
+   */
+  public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
+  }
+
+  /**
+   * @param partSetDoneEvent
+   * @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 {
+  }
+
+  /**
+   * @param fnEvent function event
+   * @throws MetaException
+   */
+  public void onCreateFunction (CreateFunctionEvent fnEvent) throws MetaException {
+  }
+
+  /**
+   * @param fnEvent function event
+   * @throws MetaException
+   */
+  public void onDropFunction (DropFunctionEvent fnEvent) throws MetaException {
+  }
+
+  /**
+   * This will be called when an insert is executed that does not cause a partition to be added.
+   * If an insert causes a partition to be added it will cause {@link #onAddPartition} to be
+   * called instead.
+   * @param insertEvent
+   * @throws MetaException
+   */
+  public void onInsert(InsertEvent insertEvent) throws MetaException {
+
+  }
+
+  /**
+   * @param addPrimaryKeyEvent add primary key event
+   * @throws MetaException
+   */
+  public void onAddPrimaryKey(AddPrimaryKeyEvent addPrimaryKeyEvent) throws MetaException {
+  }
+
+  /**
+   * @param addForeignKeyEvent add foreign key event
+   * @throws MetaException
+   */
+  public void onAddForeignKey(AddForeignKeyEvent addForeignKeyEvent) throws MetaException {
+  }
+
+  /**
+   * @param addUniqueConstraintEvent add unique constraint event
+   * @throws MetaException
+   */
+  public void onAddUniqueConstraint(AddUniqueConstraintEvent addUniqueConstraintEvent) throws MetaException {
+  }
+
+  /**
+   * @param addNotNullConstraintEvent add not null constraint event
+   * @throws MetaException
+   */
+  public void onAddNotNullConstraint(AddNotNullConstraintEvent addNotNullConstraintEvent) throws MetaException {
+  }
+
+  /**
+   * @param dropConstraintEvent drop constraint event
+   * @throws MetaException
+   */
+  public void onDropConstraint(DropConstraintEvent dropConstraintEvent) throws MetaException {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    this.conf = config;
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
new file mode 100644
index 0000000..0b3de49
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Keeps a list of reserved keys used by Hive listeners when updating the ListenerEvent
+ * parameters.
+ */
+public class MetaStoreEventListenerConstants {
+  /*
+   * DbNotificationListener keys reserved for updating ListenerEvent parameters.
+   *
+   * DB_NOTIFICATION_EVENT_ID_KEY_NAME This key will have the event identifier that DbNotificationListener
+   *                                   processed during an event. This event identifier might be shared
+   *                                   across other MetaStoreEventListener implementations.
+   */
+  public static final String DB_NOTIFICATION_EVENT_ID_KEY_NAME = "DB_NOTIFICATION_EVENT_ID_KEY_NAME";
+
+  /*
+   * HiveMetaStore keys reserved for updating ListenerEvent parameters.
+   *
+   * HIVE_METASTORE_TRANSACTION_ACTIVE This key is used to check if a listener event is run inside a current
+   *                                   transaction. A boolean value is used for active (true) or no active (false).
+   */
+  public static final String HIVE_METASTORE_TRANSACTION_ACTIVE = "HIVE_METASTORE_TRANSACTION_ACTIVE";
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java
new file mode 100644
index 0000000..0968f42
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+/**
+ * Base class which provides context to implementations of MetaStoreInitListener
+ */
+
+public class MetaStoreInitContext {
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java
new file mode 100644
index 0000000..12130ea
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+
+/**
+ * This abstract class needs to be extended to  provide implementation of actions
+ * that needs to be performed when HMSHandler is initialized
+ */
+
+public abstract class MetaStoreInitListener implements Configurable {
+
+  private Configuration conf;
+
+  public MetaStoreInitListener(Configuration config){
+    this.conf = config;
+  }
+
+  public abstract void onInit(MetaStoreInitContext context) throws MetaException;
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    this.conf = config;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
new file mode 100644
index 0000000..f6e25c6
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -0,0 +1,266 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
+import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
+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.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
+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.InsertEvent;
+import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants.HIVE_METASTORE_TRANSACTION_ACTIVE;
+import static org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
+
+/**
+ * This class is used to notify a list of listeners about specific MetaStore events.
+ */
+@InterfaceAudience.Private
+public class MetaStoreListenerNotifier {
+
+  private interface EventNotifier {
+    void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException;
+  }
+
+  private static Map<EventType, EventNotifier> notificationEvents = Maps.newHashMap(
+      ImmutableMap.<EventType, EventNotifier>builder()
+          .put(EventType.CREATE_DATABASE, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onCreateDatabase((CreateDatabaseEvent)event);
+            }
+          })
+          .put(EventType.DROP_DATABASE, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropDatabase((DropDatabaseEvent)event);
+            }
+          })
+          .put(EventType.CREATE_TABLE, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onCreateTable((CreateTableEvent)event);
+            }
+          })
+          .put(EventType.DROP_TABLE, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropTable((DropTableEvent)event);
+            }
+          })
+          .put(EventType.ADD_PARTITION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddPartition((AddPartitionEvent)event);
+            }
+          })
+          .put(EventType.DROP_PARTITION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropPartition((DropPartitionEvent)event);
+            }
+          })
+          .put(EventType.ALTER_TABLE, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAlterTable((AlterTableEvent)event);
+            }
+          })
+          .put(EventType.ALTER_PARTITION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAlterPartition((AlterPartitionEvent)event);
+            }
+          })
+          .put(EventType.INSERT, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onInsert((InsertEvent)event);
+            }
+          })
+          .put(EventType.CREATE_FUNCTION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onCreateFunction((CreateFunctionEvent)event);
+            }
+          })
+          .put(EventType.DROP_FUNCTION, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropFunction((DropFunctionEvent)event);
+            }
+          })
+          .put(EventType.CREATE_INDEX, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddIndex((AddIndexEvent)event);
+            }
+          })
+          .put(EventType.DROP_INDEX, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onDropIndex((DropIndexEvent)event);
+            }
+          })
+          .put(EventType.ALTER_INDEX, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAlterIndex((AlterIndexEvent)event);
+            }
+          })
+          .put(EventType.ADD_PRIMARYKEY, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddPrimaryKey((AddPrimaryKeyEvent)event);
+            }
+          })
+          .put(EventType.ADD_FOREIGNKEY, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddForeignKey((AddForeignKeyEvent)event);
+            }
+          })
+          .put(EventType.ADD_UNIQUECONSTRAINT, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddUniqueConstraint((AddUniqueConstraintEvent)event);
+            }
+          })
+          .put(EventType.ADD_NOTNULLCONSTRAINT, new EventNotifier() {
+            @Override
+            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
+              listener.onAddNotNullConstraint((AddNotNullConstraintEvent)event);
+            }
+          })
+          .build()
+  );
+
+  /**
+   * Notify a list of listeners about a specific metastore event. Each listener notified might update
+   * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
+   * be returned to the caller.
+   *
+   * @param listeners List of MetaStoreEventListener listeners.
+   * @param eventType Type of the notification event.
+   * @param event The ListenerEvent with information about the event.
+   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
+   *         map if no parameters were updated or if no listeners were notified.
+   * @throws MetaException If an error occurred while calling the listeners.
+   */
+  public static Map<String, String> notifyEvent(List<? extends MetaStoreEventListener> listeners,
+                                                EventType eventType,
+                                                ListenerEvent event) throws MetaException {
+
+    Preconditions.checkNotNull(listeners, "Listeners must not be null.");
+    Preconditions.checkNotNull(event, "The event must not be null.");
+
+    for (MetaStoreEventListener listener : listeners) {
+        notificationEvents.get(eventType).notify(listener, event);
+    }
+
+    // Each listener called above might set a different parameter on the event.
+    // This write permission is allowed on the listener side to avoid breaking compatibility if we change the API
+    // method calls.
+    return event.getParameters();
+  }
+
+  /**
+   * Notify a list of listeners about a specific metastore event. Each listener notified might update
+   * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
+   * be returned to the caller.
+   *
+   * @param listeners List of MetaStoreEventListener listeners.
+   * @param eventType Type of the notification event.
+   * @param event The ListenerEvent with information about the event.
+   * @param environmentContext An EnvironmentContext object with parameters sent by the HMS client.
+   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
+   *         map if no parameters were updated or if no listeners were notified.
+   * @throws MetaException If an error occurred while calling the listeners.
+   */
+  public static Map<String, String> notifyEvent(List<? extends MetaStoreEventListener> listeners,
+                                                EventType eventType,
+                                                ListenerEvent event,
+                                                EnvironmentContext environmentContext) throws MetaException {
+
+    Preconditions.checkNotNull(event, "The event must not be null.");
+
+    event.setEnvironmentContext(environmentContext);
+    return notifyEvent(listeners, eventType, event);
+  }
+
+  /**
+   * Notify a list of listeners about a specific metastore event. Each listener notified might update
+   * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
+   * be returned to the caller.
+   *
+   * Sometimes these events are run inside a DB transaction and might cause issues with the listeners,
+   * for instance, Sentry blocks the HMS until an event is seen committed on the DB. To notify the listener about this,
+   * a new parameter to verify if a transaction is active is added to the ListenerEvent, and is up to the listener
+   * to skip this notification if so.
+   *
+   * @param listeners List of MetaStoreEventListener listeners.
+   * @param eventType Type of the notification event.
+   * @param event The ListenerEvent with information about the event.
+   * @param environmentContext An EnvironmentContext object with parameters sent by the HMS client.
+   * @param parameters A list of key/value pairs with the new parameters to add.
+   * @param ms The RawStore object from where to check if a transaction is active.
+   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
+   *         map if no parameters were updated or if no listeners were notified.
+   * @throws MetaException If an error occurred while calling the listeners.
+   */
+  public static Map<String, String> notifyEvent(List<? extends MetaStoreEventListener> listeners,
+                                                EventType eventType,
+                                                ListenerEvent event,
+                                                EnvironmentContext environmentContext,
+                                                Map<String, String> parameters,
+                                                final RawStore ms) throws MetaException {
+
+    Preconditions.checkNotNull(event, "The event must not be null.");
+
+    event.putParameters(parameters);
+
+    if (ms != null) {
+      event.putParameter(HIVE_METASTORE_TRANSACTION_ACTIVE, Boolean.toString(ms.isActiveTransaction()));
+    }
+
+    return notifyEvent(listeners, eventType, event, environmentContext);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java
new file mode 100644
index 0000000..a757dff
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+
+/**
+ * This abstract class needs to be extended to  provide implementation of actions that needs
+ * to be performed before a particular event occurs on a metastore. These methods
+ * are called before an event occurs on metastore.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class MetaStorePreEventListener implements Configurable {
+
+  private Configuration conf;
+
+  public MetaStorePreEventListener(Configuration config){
+    this.conf = config;
+  }
+
+  public abstract void onEvent(PreEventContext context)
+      throws MetaException, NoSuchObjectException, InvalidOperationException;
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    this.conf = config;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
new file mode 100644
index 0000000..559ff34
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
@@ -0,0 +1,232 @@
+/*
+ * 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;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.datanucleus.exceptions.NucleusException;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RetryingHMSHandler implements InvocationHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RetryingHMSHandler.class);
+  private static final String CLASS_NAME = RetryingHMSHandler.class.getName();
+
+  private static class Result {
+    private final Object result;
+    private final int numRetries;
+
+    public Result(Object result, int numRetries) {
+      this.result = result;
+      this.numRetries = numRetries;
+    }
+  }
+
+  private final IHMSHandler baseHandler;
+  private final MetaStoreInit.MetaStoreInitData metaStoreInitData =
+    new MetaStoreInit.MetaStoreInitData();
+
+  private final Configuration origConf;            // base configuration
+  private final Configuration activeConf;  // active configuration
+
+  private RetryingHMSHandler(Configuration origConf, IHMSHandler baseHandler, boolean local) throws MetaException {
+    this.origConf = origConf;
+    this.baseHandler = baseHandler;
+    if (local) {
+      baseHandler.setConf(origConf); // tests expect configuration changes applied directly to metastore
+    }
+    activeConf = baseHandler.getConf();
+    // This has to be called before initializing the instance of HMSHandler
+    // Using the hook on startup ensures that the hook always has priority
+    // over settings in *.xml.  The thread local conf needs to be used because at this point
+    // it has already been initialized using hiveConf.
+    MetaStoreInit.updateConnectionURL(origConf, getActiveConf(), null, metaStoreInitData);
+    try {
+      //invoking init method of baseHandler this way since it adds the retry logic
+      //in case of transient failures in init method
+      invoke(baseHandler, baseHandler.getClass().getDeclaredMethod("init", (Class<?>[]) null),
+          null);
+    } catch (Throwable e) {
+      LOG.error("HMSHandler Fatal error: " + ExceptionUtils.getStackTrace(e));
+      MetaException me = new MetaException(e.getMessage());
+      me.initCause(e);
+      throw me;
+    }
+  }
+
+  public static IHMSHandler getProxy(Configuration conf, IHMSHandler baseHandler, boolean local)
+      throws MetaException {
+
+    RetryingHMSHandler handler = new RetryingHMSHandler(conf, baseHandler, local);
+
+    return (IHMSHandler) Proxy.newProxyInstance(
+      RetryingHMSHandler.class.getClassLoader(),
+      new Class[] { IHMSHandler.class }, handler);
+  }
+
+  @Override
+  public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
+    int retryCount = -1;
+    int threadId = baseHandler.getThreadId();
+    boolean error = true;
+    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
+    perfLogger.PerfLogBegin(CLASS_NAME, method.getName());
+    try {
+      Result result = invokeInternal(proxy, method, args);
+      retryCount = result.numRetries;
+      error = false;
+      return result.result;
+    } finally {
+      StringBuilder additionalInfo = new StringBuilder();
+      additionalInfo.append("threadId=").append(threadId).append(" retryCount=").append(retryCount)
+        .append(" error=").append(error);
+      perfLogger.PerfLogEnd(CLASS_NAME, method.getName(), additionalInfo.toString());
+    }
+  }
+
+  public Result invokeInternal(final Object proxy, final Method method, final Object[] args) throws Throwable {
+
+    boolean gotNewConnectUrl = false;
+    boolean reloadConf = MetastoreConf.getBoolVar(origConf, ConfVars.HMSHANDLERFORCERELOADCONF);
+    long retryInterval = MetastoreConf.getTimeVar(origConf,
+        ConfVars.HMSHANDLERINTERVAL, TimeUnit.MILLISECONDS);
+    int retryLimit = MetastoreConf.getIntVar(origConf, ConfVars.HMSHANDLERATTEMPTS);
+    long timeout = MetastoreConf.getTimeVar(origConf,
+        ConfVars.CLIENT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS);
+
+    Deadline.registerIfNot(timeout);
+
+    if (reloadConf) {
+      MetaStoreInit.updateConnectionURL(origConf, getActiveConf(),
+        null, metaStoreInitData);
+    }
+
+    int retryCount = 0;
+    Throwable caughtException = null;
+    while (true) {
+      try {
+        if (reloadConf || gotNewConnectUrl) {
+          baseHandler.setConf(getActiveConf());
+        }
+        Object object = null;
+        boolean isStarted = Deadline.startTimer(method.getName());
+        try {
+          object = method.invoke(baseHandler, args);
+        } finally {
+          if (isStarted) {
+            Deadline.stopTimer();
+          }
+        }
+        return new Result(object, retryCount);
+
+      } catch (UndeclaredThrowableException e) {
+        if (e.getCause() != null) {
+          if (e.getCause() instanceof javax.jdo.JDOException) {
+            // Due to reflection, the jdo exception is wrapped in
+            // invocationTargetException
+            caughtException = e.getCause();
+          } else if (e.getCause() instanceof MetaException && e.getCause().getCause() != null
+              && e.getCause().getCause() instanceof javax.jdo.JDOException) {
+            // The JDOException may be wrapped further in a MetaException
+            caughtException = e.getCause().getCause();
+          } else {
+            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
+            throw e.getCause();
+          }
+        } else {
+          LOG.error(ExceptionUtils.getStackTrace(e));
+          throw e;
+        }
+      } catch (InvocationTargetException e) {
+        if (e.getCause() instanceof javax.jdo.JDOException) {
+          // Due to reflection, the jdo exception is wrapped in
+          // invocationTargetException
+          caughtException = e.getCause();
+        } else if (e.getCause() instanceof NoSuchObjectException || e.getTargetException().getCause() instanceof NoSuchObjectException) {
+          String methodName = method.getName();
+          if (!methodName.startsWith("get_database") && !methodName.startsWith("get_table")
+              && !methodName.startsWith("get_partition") && !methodName.startsWith("get_function")) {
+            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
+          }
+          throw e.getCause();
+        } else if (e.getCause() instanceof MetaException && e.getCause().getCause() != null) {
+          if (e.getCause().getCause() instanceof javax.jdo.JDOException ||
+              e.getCause().getCause() instanceof NucleusException) {
+            // The JDOException or the Nucleus Exception may be wrapped further in a MetaException
+            caughtException = e.getCause().getCause();
+          } else if (e.getCause().getCause() instanceof DeadlineException) {
+            // The Deadline Exception needs no retry and be thrown immediately.
+            Deadline.clear();
+            LOG.error("Error happens in method " + method.getName() + ": " +
+                ExceptionUtils.getStackTrace(e.getCause()));
+            throw e.getCause();
+          } else {
+            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
+            throw e.getCause();
+          }
+        } else {
+          LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
+          throw e.getCause();
+        }
+      }
+
+      if (retryCount >= retryLimit) {
+        LOG.error("HMSHandler Fatal error: " + ExceptionUtils.getStackTrace(caughtException));
+        MetaException me = new MetaException(caughtException.getMessage());
+        me.initCause(caughtException);
+        throw me;
+      }
+
+      assert (retryInterval >= 0);
+      retryCount++;
+      LOG.error(
+        String.format(
+          "Retrying HMSHandler after %d ms (attempt %d of %d)", retryInterval, retryCount, retryLimit) +
+          " with error: " + ExceptionUtils.getStackTrace(caughtException));
+
+      Thread.sleep(retryInterval);
+      // If we have a connection error, the JDO connection URL hook might
+      // provide us with a new URL to access the datastore.
+      String lastUrl = MetaStoreInit.getConnectionURL(getActiveConf());
+      gotNewConnectUrl = MetaStoreInit.updateConnectionURL(origConf, getActiveConf(),
+        lastUrl, metaStoreInitData);
+    }
+  }
+
+  public Configuration getActiveConf() {
+    return activeConf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java
new file mode 100644
index 0000000..af4f0f2
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java
@@ -0,0 +1,46 @@
+/**
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * It handles the changed properties in the change event.
+ */
+public class SessionPropertiesListener extends MetaStoreEventListener {
+
+  public SessionPropertiesListener(Configuration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public void onConfigChange(ConfigChangeEvent changeEvent) throws MetaException {
+    if (changeEvent.getKey().equals(MetastoreConf.ConfVars.CLIENT_SOCKET_TIMEOUT.getVarname()) ||
+        changeEvent.getKey().equals(MetastoreConf.ConfVars.CLIENT_SOCKET_TIMEOUT.getHiveName())) {
+      // TODO: this only applies to current thread, so it's not useful at all.
+      Deadline.resetTimeout(MetastoreConf.convertTimeStr(changeEvent.getNewValue(), TimeUnit.SECONDS,
+          TimeUnit.MILLISECONDS));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java
new file mode 100644
index 0000000..b6a67ac
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Special type of MetaStoreEventListener which should only be called in a transactional context
+ * and only if the transaction is successful.
+ * The events are expected to have a success status.
+ */
+public abstract class TransactionalMetaStoreEventListener extends MetaStoreEventListener {
+
+  /**
+   * Constructor
+   *
+   * @param config configuration object
+   */
+  public TransactionalMetaStoreEventListener(Configuration config) {
+    super(config);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
new file mode 100644
index 0000000..f7962a4
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
@@ -0,0 +1,360 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class TransactionalValidationListener extends MetaStorePreEventListener {
+  public static final Logger LOG = LoggerFactory.getLogger(TransactionalValidationListener.class);
+
+  // These constants are also imported by org.apache.hadoop.hive.ql.io.AcidUtils.
+  public static final String DEFAULT_TRANSACTIONAL_PROPERTY = "default";
+  public static final String INSERTONLY_TRANSACTIONAL_PROPERTY = "insert_only";
+
+  TransactionalValidationListener(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public void onEvent(PreEventContext context) throws MetaException, NoSuchObjectException,
+      InvalidOperationException {
+    switch (context.getEventType()) {
+      case CREATE_TABLE:
+        handle((PreCreateTableEvent) context);
+        break;
+      case ALTER_TABLE:
+        handle((PreAlterTableEvent) context);
+        break;
+      default:
+        //no validation required..
+    }
+  }
+
+  private void handle(PreAlterTableEvent context) throws MetaException {
+    handleAlterTableTransactionalProp(context);
+  }
+
+  private void handle(PreCreateTableEvent context) throws MetaException {
+    handleCreateTableTransactionalProp(context);
+  }
+
+  /**
+   * once a table is marked transactional, you cannot go back.  Enforce this.
+   * Also in current version, 'transactional_properties' of the table cannot be altered after
+   * the table is created. Any attempt to alter it will throw a MetaException.
+   */
+  private void handleAlterTableTransactionalProp(PreAlterTableEvent context) throws MetaException {
+    Table newTable = context.getNewTable();
+    Map<String, String> parameters = newTable.getParameters();
+    if (parameters == null || parameters.isEmpty()) {
+      return;
+    }
+    Set<String> keys = new HashSet<>(parameters.keySet());
+    String transactionalValue = null;
+    boolean transactionalValuePresent = false;
+    boolean isTransactionalPropertiesPresent = false;
+    String transactionalPropertiesValue = null;
+    boolean hasValidTransactionalValue = false;
+
+    for (String key : keys) {
+      if(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) {
+        transactionalValuePresent = true;
+        transactionalValue = parameters.get(key);
+        parameters.remove(key);
+      }
+      if(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
+        isTransactionalPropertiesPresent = true;
+        transactionalPropertiesValue = parameters.get(key);
+        // Do not remove the parameter yet, because we have separate initialization routine
+        // that will use it down below.
+      }
+    }
+    Table oldTable = context.getOldTable();
+    String oldTransactionalValue = null;
+    String oldTransactionalPropertiesValue = null;
+    for (String key : oldTable.getParameters().keySet()) {
+      if (hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) {
+        oldTransactionalValue = oldTable.getParameters().get(key);
+      }
+      if (hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
+        oldTransactionalPropertiesValue = oldTable.getParameters().get(key);
+      }
+    }
+
+    if (transactionalValuePresent) {
+      //normalize prop name
+      parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, transactionalValue);
+    }
+    if ("true".equalsIgnoreCase(transactionalValue) && !"true".equalsIgnoreCase(oldTransactionalValue)) {
+      //only need to check conformance if alter table enabled aicd
+      if (!conformToAcid(newTable)) {
+        // INSERT_ONLY tables don't have to conform to ACID requirement like ORC or bucketing
+        if (transactionalPropertiesValue == null || !"insert_only".equalsIgnoreCase(transactionalPropertiesValue)) {
+          throw new MetaException("The table must be stored using an ACID compliant format (such as ORC)");
+        }
+      }
+
+      if (newTable.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
+        throw new MetaException(getTableName(newTable) +
+            " cannot be declared transactional because it's an external table");
+      }
+      validateTableStructure(context.getHandler(), newTable);
+      hasValidTransactionalValue = true;
+    }
+
+
+
+    if (oldTransactionalValue == null ? transactionalValue == null
+                                     : oldTransactionalValue.equalsIgnoreCase(transactionalValue)) {
+      //this covers backward compat cases where this prop may have been set already
+      hasValidTransactionalValue = true;
+    }
+
+    if (!hasValidTransactionalValue && !MetaStoreUtils.isInsertOnlyTableParam(oldTable.getParameters())) {
+      // if here, there is attempt to set transactional to something other than 'true'
+      // and NOT the same value it was before
+      throw new MetaException("TBLPROPERTIES with 'transactional'='true' cannot be unset");
+    }
+
+    if (isTransactionalPropertiesPresent) {
+      // Now validate transactional_properties for the table.
+      if (oldTransactionalValue == null) {
+        // If this is the first time the table is being initialized to 'transactional=true',
+        // any valid value can be set for the 'transactional_properties'.
+        initializeTransactionalProperties(newTable);
+      } else {
+        // If the table was already marked as 'transactional=true', then the new value of
+        // 'transactional_properties' must match the old value. Any attempt to alter the previous
+        // value will throw an error. An exception will still be thrown if the previous value was
+        // null and an attempt is made to set it. This behaviour can be changed in the future.
+        if ((oldTransactionalPropertiesValue == null
+            || !oldTransactionalPropertiesValue.equalsIgnoreCase(transactionalPropertiesValue))
+            && !MetaStoreUtils.isInsertOnlyTableParam(oldTable.getParameters())) {
+          throw new MetaException("TBLPROPERTIES with 'transactional_properties' cannot be "
+              + "altered after the table is created");
+        }
+      }
+    }
+  }
+
+  /**
+   * Normalize case and make sure:
+   * 1. 'true' is the only value to be set for 'transactional' (if set at all)
+   * 2. If set to 'true', we should also enforce bucketing and ORC format
+   */
+  private void handleCreateTableTransactionalProp(PreCreateTableEvent context) throws MetaException {
+    Table newTable = context.getTable();
+    Map<String, String> parameters = newTable.getParameters();
+    if (parameters == null || parameters.isEmpty()) {
+      return;
+    }
+    String transactional = null;
+    String transactionalProperties = null;
+    Set<String> keys = new HashSet<>(parameters.keySet());
+    for(String key : keys) {
+      // Get the "transactional" tblproperties value
+      if (hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) {
+        transactional = parameters.get(key);
+        parameters.remove(key);
+      }
+
+      // Get the "transactional_properties" tblproperties value
+      if (hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
+        transactionalProperties = parameters.get(key);
+      }
+    }
+
+    if (transactional == null) {
+      return;
+    }
+
+    if ("false".equalsIgnoreCase(transactional)) {
+      // just drop transactional=false.  For backward compatibility in case someone has scripts
+      // with transactional=false
+      LOG.info("'transactional'='false' is no longer a valid property and will be ignored");
+      return;
+    }
+
+    if ("true".equalsIgnoreCase(transactional)) {
+      if (!conformToAcid(newTable)) {
+        // INSERT_ONLY tables don't have to conform to ACID requirement like ORC or bucketing
+        if (transactionalProperties == null || !"insert_only".equalsIgnoreCase(transactionalProperties)) {
+          throw new MetaException("The table must be stored using an ACID compliant format (such as ORC)");
+        }
+      }
+
+      if (newTable.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
+        throw new MetaException(newTable.getDbName() + "." + newTable.getTableName() +
+            " cannot be declared transactional because it's an external table");
+      }
+
+      // normalize prop name
+      parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, Boolean.TRUE.toString());
+      initializeTransactionalProperties(newTable);
+      return;
+    }
+
+    // transactional is found, but the value is not in expected range
+    throw new MetaException("'transactional' property of TBLPROPERTIES may only have value 'true'");
+  }
+
+  /**
+   * Check that InputFormatClass/OutputFormatClass should implement
+   * AcidInputFormat/AcidOutputFormat
+   */
+  private boolean conformToAcid(Table table) throws MetaException {
+    StorageDescriptor sd = table.getSd();
+    try {
+      Class inputFormatClass = Class.forName(sd.getInputFormat());
+      Class outputFormatClass = Class.forName(sd.getOutputFormat());
+
+      if (inputFormatClass == null || outputFormatClass == null ||
+          !Class.forName("org.apache.hadoop.hive.ql.io.AcidInputFormat").isAssignableFrom(inputFormatClass) ||
+          !Class.forName("org.apache.hadoop.hive.ql.io.AcidOutputFormat").isAssignableFrom(outputFormatClass)) {
+        return false;
+      }
+    } catch (ClassNotFoundException e) {
+      throw new MetaException("Invalid input/output format for table");
+    }
+
+    return true;
+  }
+
+  private void initializeTransactionalProperties(Table table) throws MetaException {
+    // All new versions of Acid tables created after the introduction of Acid version/type system
+    // can have TRANSACTIONAL_PROPERTIES property defined. This parameter can be used to change
+    // the operational behavior of ACID. However if this parameter is not defined, the new Acid
+    // tables will still behave as the old ones. This is done so to preserve the behavior
+    // in case of rolling downgrade.
+
+    // Initialize transaction table properties with default string value.
+    String tableTransactionalProperties = null;
+
+    Map<String, String> parameters = table.getParameters();
+    if (parameters != null) {
+      Set<String> keys = parameters.keySet();
+      for (String key : keys) {
+        if (hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
+          tableTransactionalProperties = parameters.get(key).toLowerCase();
+          parameters.remove(key);
+          String validationError = validateTransactionalProperties(tableTransactionalProperties);
+          if (validationError != null) {
+            throw new MetaException("Invalid transactional properties specified for the "
+                + "table with the error " + validationError);
+          }
+          break;
+        }
+      }
+    }
+
+    if (tableTransactionalProperties != null) {
+      parameters.put(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES,
+              tableTransactionalProperties);
+    }
+  }
+
+  private String validateTransactionalProperties(String transactionalProperties) {
+    boolean isValid = false;
+    switch (transactionalProperties) {
+      case DEFAULT_TRANSACTIONAL_PROPERTY:
+      case INSERTONLY_TRANSACTIONAL_PROPERTY:
+        isValid = true;
+        break;
+      default:
+        isValid = false;
+    }
+    if (!isValid) {
+      return "unknown value " + transactionalProperties +  " for transactional_properties";
+    }
+    return null; // All checks passed, return null.
+  }
+  private final Pattern ORIGINAL_PATTERN = Pattern.compile("[0-9]+_[0-9]+");
+  /**
+   * @see org.apache.hadoop.hive.ql.exec.Utilities#COPY_KEYWORD
+   */
+  private static final Pattern ORIGINAL_PATTERN_COPY =
+    Pattern.compile("[0-9]+_[0-9]+" + "_copy_" + "[0-9]+");
+
+  /**
+   * It's assumed everywhere that original data files are named according to
+   * {@link #ORIGINAL_PATTERN} or{@link #ORIGINAL_PATTERN_COPY}
+   * This checks that when transaction=true is set and throws if it finds any files that don't
+   * follow convention.
+   */
+  private void validateTableStructure(IHMSHandler hmsHandler, Table table)
+    throws MetaException {
+    Path tablePath;
+    try {
+      Warehouse wh = hmsHandler.getWh();
+      if (table.getSd().getLocation() == null || table.getSd().getLocation().isEmpty()) {
+        tablePath = wh.getDefaultTablePath(hmsHandler.getMS().getDatabase(table.getDbName()),
+          table.getTableName());
+      } else {
+        tablePath = wh.getDnsPath(new Path(table.getSd().getLocation()));
+      }
+      FileSystem fs = wh.getFs(tablePath);
+      //FileSystem fs = FileSystem.get(getConf());
+      RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(tablePath, true);
+      while (iterator.hasNext()) {
+        LocatedFileStatus fileStatus = iterator.next();
+        if (!fileStatus.isFile()) {
+          continue;
+        }
+        boolean validFile =
+          (ORIGINAL_PATTERN.matcher(fileStatus.getPath().getName()).matches() ||
+            ORIGINAL_PATTERN_COPY.matcher(fileStatus.getPath().getName()).matches()
+          );
+        if (!validFile) {
+          throw new IllegalStateException("Unexpected data file name format.  Cannot convert " +
+            getTableName(table) + " to transactional table.  File: " + fileStatus.getPath());
+        }
+      }
+    } catch (IOException|NoSuchObjectException e) {
+      String msg = "Unable to list files for " + getTableName(table);
+      LOG.error(msg, e);
+      MetaException e1 = new MetaException(msg);
+      e1.initCause(e);
+      throw e1;
+    }
+  }
+  private static String getTableName(Table table) {
+    return table.getDbName() + "." + table.getTableName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index ac81912..f8ebc12 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -166,6 +166,7 @@ public class MetastoreConf {
       ConfVars.AGGREGATE_STATS_CACHE_MAX_FULL,
       ConfVars.AGGREGATE_STATS_CACHE_CLEAN_UNTIL,
       ConfVars.DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES,
+      ConfVars.FILE_METADATA_THREADS
   };
 
   /**
@@ -446,6 +447,9 @@ public class MetastoreConf {
     EXPRESSION_PROXY_CLASS("metastore.expression.proxy", "hive.metastore.expression.proxy",
         "org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore",
         "Class to use to process expressions in partition pruning."),
+    FILE_METADATA_THREADS("metastore.file.metadata.threads",
+        "hive.metastore.hbase.file.metadata.threads", 1,
+        "Number of threads to use to read file metadata in background to cache it."),
     FILTER_HOOK("metastore.filter.hook", "hive.metastore.filter.hook",
         "org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl",
         "Metastore hook class for filtering the metadata read results. If hive.security.authorization.manager"
@@ -501,6 +505,18 @@ public class MetastoreConf {
         "javax.jdo.PersistenceManagerFactoryClass",
         "org.datanucleus.api.jdo.JDOPersistenceManagerFactory",
         "class implementing the jdo persistence"),
+    // Parameters for exporting metadata on table drop (requires the use of the)
+    // org.apache.hadoop.hive.ql.parse.MetaDataExportListener preevent listener
+    METADATA_EXPORT_LOCATION("metastore.metadata.export.location", "hive.metadata.export.location",
+        "",
+        "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" +
+            "it is the location to which the metadata will be exported. The default is an empty string, which results in the \n" +
+            "metadata being exported to the current user's home directory on HDFS."),
+    MOVE_EXPORTED_METADATA_TO_TRASH("metastore.metadata.move.exported.metadata.to.trash",
+        "hive.metadata.move.exported.metadata.to.trash", true,
+        "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" +
+            "this setting determines if the metadata that is exported will subsequently be moved to the user's trash directory \n" +
+            "alongside the dropped table data. This ensures that the metadata will be cleaned up along with the dropped table data."),
     METRICS_ENABLED("metastore.metrics.enabled", "hive.metastore.metrics.enabled", false,
         "Enable metrics on the metastore."),
     METRICS_JSON_FILE_INTERVAL("metastore.metrics.file.frequency",

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java
new file mode 100644
index 0000000..f9c951c
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java
@@ -0,0 +1,41 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+
+@InterfaceStability.Stable
+@InterfaceAudience.Public
+public class AddForeignKeyEvent extends ListenerEvent {
+  private final List<SQLForeignKey> fks;
+
+  public AddForeignKeyEvent(List<SQLForeignKey> fks, boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.fks = fks;
+  }
+
+  public List<SQLForeignKey> getForeignKeyCols() {
+    return fks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
new file mode 100644
index 0000000..0d6cb56
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
@@ -0,0 +1,40 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AddIndexEvent extends ListenerEvent {
+
+  private final Index index;
+
+  public AddIndexEvent(Index index, boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.index = index;
+  }
+
+  public Index getIndex() {
+    return index;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java
new file mode 100644
index 0000000..6b4cf0b
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AddNotNullConstraintEvent extends ListenerEvent {
+  private final List<SQLNotNullConstraint> nns;
+
+  public AddNotNullConstraintEvent(List<SQLNotNullConstraint> nns, boolean status,
+                                   IHMSHandler handler) {
+    super(status, handler);
+    this.nns = nns;
+  }
+
+  public List<SQLNotNullConstraint> getNotNullConstraintCols() {
+    return nns;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
new file mode 100644
index 0000000..d4542d7
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
@@ -0,0 +1,84 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+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 java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AddPartitionEvent extends ListenerEvent {
+
+  private final Table table;
+  private final List<Partition> partitions;
+  private PartitionSpecProxy partitionSpecProxy;
+
+  public AddPartitionEvent(Table table, List<Partition> partitions, boolean status,
+                           IHMSHandler handler) {
+    super(status, handler);
+    this.table = table;
+    this.partitions = partitions;
+    this.partitionSpecProxy = null;
+  }
+
+  public AddPartitionEvent(Table table, Partition partition, boolean status, IHMSHandler handler) {
+    this(table, Arrays.asList(partition), status, handler);
+  }
+
+  /**
+   * Alternative constructor to use PartitionSpec APIs.
+   */
+  public AddPartitionEvent(Table table, PartitionSpecProxy partitionSpec, boolean status,
+                           IHMSHandler handler) {
+    super(status, handler);
+    this.table = table;
+    this.partitions = null;
+    this.partitionSpecProxy = partitionSpec;
+  }
+
+  /**
+   * @return The table.
+   */
+  public Table getTable() {
+    return table;
+  }
+
+
+  // Note : List<Partition> getPartitions() removed with HIVE-9609 because it will result in OOM errors with large add_partitions.
+
+  /**
+   * @return Iterator for partitions.
+   */
+  public Iterator<Partition> getPartitionIterator() {
+    if (partitions != null){
+      return partitions.iterator();
+    } else {
+      return partitionSpecProxy == null ? null : partitionSpecProxy.getPartitionIterator();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java
new file mode 100644
index 0000000..de7978c
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AddPrimaryKeyEvent extends ListenerEvent {
+
+  private final List<SQLPrimaryKey> pks;
+
+  public AddPrimaryKeyEvent(List<SQLPrimaryKey> pks, boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.pks = pks;
+  }
+
+  public List<SQLPrimaryKey> getPrimaryKeyCols() {
+    return pks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java
new file mode 100644
index 0000000..b677ee6
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AddUniqueConstraintEvent extends ListenerEvent {
+  private final List<SQLUniqueConstraint> uks;
+
+  public AddUniqueConstraintEvent(List<SQLUniqueConstraint> uks, boolean status,
+                                  IHMSHandler handler) {
+    super(status, handler);
+    this.uks = uks;
+  }
+
+  public List<SQLUniqueConstraint> getUniqueConstraintCols() {
+    return uks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
new file mode 100644
index 0000000..2c72b83
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
@@ -0,0 +1,46 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AlterIndexEvent extends ListenerEvent {
+
+  private final Index newIndex;
+  private final Index oldIndex;
+
+  public AlterIndexEvent(Index oldIndex, Index newIndex, boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.oldIndex = oldIndex;
+    this.newIndex = newIndex;
+  }
+
+  public Index getOldIndex() {
+    return oldIndex;
+  }
+
+  public Index getNewIndex() {
+    return newIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
new file mode 100644
index 0000000..09cad85
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
@@ -0,0 +1,75 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AlterPartitionEvent extends ListenerEvent {
+
+  private final Partition oldPart;
+  private final Partition newPart;
+  private final Table table;
+  private final boolean isTruncateOp;
+
+  public AlterPartitionEvent(Partition oldPart, Partition newPart, Table table, boolean isTruncateOp,
+                             boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.oldPart = oldPart;
+    this.newPart = newPart;
+    this.table = table;
+    this.isTruncateOp = isTruncateOp;
+  }
+
+  /**
+   * @return the old partition
+   */
+  public Partition getOldPartition() {
+    return oldPart;
+  }
+
+  /**
+   *
+   * @return the new partition
+   */
+  public Partition getNewPartition() {
+    return newPart;
+  }
+
+  /**
+   * Get the table this partition is in
+   * @return
+   */
+  public Table getTable() {
+    return table;
+  }
+
+  /**
+   * Get the truncate table flag
+   * @return
+   */
+  public boolean getIsTruncateOp() {
+    return isTruncateOp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
new file mode 100644
index 0000000..2e3f6f0
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
@@ -0,0 +1,63 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AlterTableEvent extends ListenerEvent {
+
+  private final Table newTable;
+  private final Table oldTable;
+  private final boolean isTruncateOp;
+
+  public AlterTableEvent (Table oldTable, Table newTable, boolean isTruncateOp, boolean status,
+                          IHMSHandler handler) {
+    super (status, handler);
+    this.oldTable = oldTable;
+    this.newTable = newTable;
+    this.isTruncateOp = isTruncateOp;
+  }
+
+  /**
+   * @return the old table
+   */
+  public Table getOldTable() {
+    return oldTable;
+  }
+
+  /**
+   * @return the new table
+   */
+  public Table getNewTable() {
+    return newTable;
+  }
+
+  /**
+   * @return the flag for truncate
+   */
+  public boolean getIsTruncateOp() {
+    return isTruncateOp;
+  }
+}
\ No newline at end of file


[09/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java
deleted file mode 100644
index 7f99f18..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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 java.util.TimerTask;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.RawStore;
-
-public class EventCleanerTask extends TimerTask{
-
-  public static final Logger LOG = LoggerFactory.getLogger(EventCleanerTask.class);
-  private final HMSHandler handler;
-
-  public EventCleanerTask(HMSHandler handler) {
-    super();
-    this.handler = handler;
-  }
-
-  @Override
-  public void run() {
-
-    try {
-      RawStore ms = handler.getMS();
-      long deleteCnt = ms.cleanupEvents();
-
-      if (deleteCnt > 0L){
-        LOG.info("Number of events deleted from event Table: "+deleteCnt);
-      }
-    } catch (Exception e) {
-      LOG.error("Exception while trying to delete events ", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
deleted file mode 100644
index d451122..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.api.GetTableRequest;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-import java.util.ArrayList;
-import java.util.List;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class InsertEvent extends ListenerEvent {
-
-  private final Table tableObj;
-  private final Partition ptnObj;
-  private final boolean replace;
-  private final List<String> files;
-  private List<String> fileChecksums = new ArrayList<String>();
-
-  /**
-   *
-   * @param db name of the database the table is in
-   * @param table name of the table being inserted into
-   * @param partVals list of partition values, can be null
-   * @param insertData the inserted files and their checksums
-   * @param status status of insert, true = success, false = failure
-   * @param handler handler that is firing the event
-   */
-  public InsertEvent(String db, String table, List<String> partVals,
-      InsertEventRequestData insertData, boolean status, HMSHandler handler) throws MetaException,
-      NoSuchObjectException {
-    super(status, handler);
-
-    GetTableRequest req = new GetTableRequest(db, table);
-    req.setCapabilities(HiveMetaStoreClient.TEST_VERSION);
-    this.tableObj = handler.get_table_req(req).getTable();
-    if (partVals != null) {
-      this.ptnObj = handler.get_partition(db, table, partVals);
-    } else {
-      this.ptnObj = null;
-    }
-
-    // If replace flag is not set by caller, then by default set it to true to maintain backward compatibility
-    this.replace = (insertData.isSetReplace() ? insertData.isReplace() : true);
-    this.files = insertData.getFilesAdded();
-    if (insertData.isSetFilesAddedChecksum()) {
-      fileChecksums = insertData.getFilesAddedChecksum();
-    }
-  }
-
-  /**
-   * @return Table object
-   */
-  public Table getTableObj() {
-    return tableObj;
-  }
-
-  /**
-   * @return Partition object
-   */
-  public Partition getPartitionObj() {
-    return ptnObj;
-  }
-
-  /**
-   * @return The replace flag.
-   */
-  public boolean isReplace() {
-    return replace;
-  }
-
-  /**
-   * Get list of files created as a result of this DML operation
-   *
-   * @return list of new files
-   */
-  public List<String> getFiles() {
-    return files;
-  }
-
-  /**
-   * Get a list of file checksums corresponding to the files created (if available)
-   *
-   * @return
-   */
-  public List<String> getFileChecksums() {
-    return fileChecksums;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
deleted file mode 100644
index cd66f10..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-
-import javax.annotation.concurrent.NotThreadSafe;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Base class for all the events which are defined for metastore.
- *
- * This class is not thread-safe and not expected to be called in parallel.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-@NotThreadSafe
-public abstract class ListenerEvent {
-
-  /**
-   * status of the event, whether event was successful or not.
-   */
-  private final boolean status;
-  private final HMSHandler handler;
-
-  /**
-   * Key/value parameters used by listeners to store notifications results
-   * i.e. DbNotificationListener sets a DB_NOTIFICATION_EVENT_ID.
-   *
-   * NotThreadSafe: The parameters map is not expected to be access in parallel by Hive, so keep it thread-unsafe
-   * to avoid locking overhead.
-   */
-  private Map<String, String> parameters;
-
-  /** For performance concerns, it is preferable to cache the unmodifiable parameters variable that will be returned on the
-   * {@link #getParameters()} method. It is expected that {@link #putParameter(String, String)} is called less times
-   * than {@link #getParameters()}, so performance may be better by using this cache.
-   */
-  private Map<String, String> unmodifiableParameters;
-
-  // Listener parameters aren't expected to have many values. So far only
-  // DbNotificationListener will add a parameter; let's set a low initial capacity for now.
-  // If we find out many parameters are added, then we can adjust or remove this initial capacity.
-  private static final int PARAMETERS_INITIAL_CAPACITY = 1;
-
-  // Properties passed by the client, to be used in execution hooks.
-  private EnvironmentContext environmentContext = null;
-
-  public ListenerEvent(boolean status, HMSHandler handler) {
-    super();
-    this.status = status;
-    this.handler = handler;
-    this.parameters = new HashMap<>(PARAMETERS_INITIAL_CAPACITY);
-    updateUnmodifiableParameters();
-  }
-
-  /**
-   * @return the status of event.
-   */
-  public boolean getStatus() {
-    return status;
-  }
-
-  /**
-   * Set the environment context of the event.
-   *
-   * @param environmentContext An EnvironmentContext object that contains environment parameters sent from
-   *                           the HMS client.
-   */
-  public void setEnvironmentContext(EnvironmentContext environmentContext) {
-    this.environmentContext = environmentContext;
-  }
-
-  /**
-   * @return environment properties of the event
-   */
-  public EnvironmentContext getEnvironmentContext() {
-    return environmentContext;
-  }
-
-  /**
-   * @return the handler
-   */
-  public HMSHandler getHandler() {
-    return handler;
-  }
-
-  /**
-   * Return all parameters of the listener event. Parameters are read-only (unmodifiable map). If a new parameter
-   * must be added, please use the putParameter() method.
-   *
-   *
-   * @return A map object with all parameters.
-   */
-  public final Map<String, String> getParameters() {
-    return unmodifiableParameters;
-  }
-
-  /**
-   * Put a new parameter to the listener event.
-   *
-   * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration
-   * between listeners setting the same parameters.
-   *
-   * @param name Name of the parameter.
-   * @param value Value of the parameter.
-   * @throws IllegalStateException if a parameter already exists.
-   */
-  public void putParameter(String name, String value) {
-    putParameterIfAbsent(name, value);
-    updateUnmodifiableParameters();
-  }
-
-  /**
-   * Put a new set the parameters to the listener event.
-   *
-   * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration
-   * between listeners setting the same parameters.
-   *
-   * @param parameters A Map object with the a set of parameters.
-   * @throws IllegalStateException if a parameter already exists.
-   */
-  public void putParameters(final Map<String, String> parameters) {
-    if (parameters != null) {
-      for (Map.Entry<String, String> entry : parameters.entrySet()) {
-        putParameterIfAbsent(entry.getKey(), entry.getValue());
-      }
-
-      updateUnmodifiableParameters();
-    }
-  }
-
-  /**
-   * Put a parameter to the listener event only if the parameter is absent.
-   *
-   * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration
-   * between listeners setting the same parameters.
-   *
-   * @param name Name of the parameter.
-   * @param value Value of the parameter.
-   * @throws IllegalStateException if a parameter already exists.
-   */
-  private void putParameterIfAbsent(String name, String value) {
-    if (parameters.containsKey(name)) {
-      throw new IllegalStateException("Invalid attempt to overwrite a read-only parameter: " + name);
-    }
-
-    parameters.put(name, value);
-  }
-
-  /**
-   * Keeps a cache of unmodifiable parameters returned by the getParameters() method.
-   */
-  private void updateUnmodifiableParameters() {
-    unmodifiableParameters = Collections.unmodifiableMap(parameters);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
deleted file mode 100644
index b56b3bb..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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 java.util.Map;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class LoadPartitionDoneEvent extends ListenerEvent {
-
-  private final Table table;
-
-  private final Map<String,String> partSpec;
-
-  public LoadPartitionDoneEvent(boolean status, Table table,
-      Map<String,String> partSpec, HMSHandler handler) {
-    super(status, handler);
-    this.table = table;
-    this.partSpec = partSpec;
-  }
-
-  /**
-   * @return the tblName
-   */
-  public Table getTable() {
-    return table;
-  }
-
-  /**
-   * @return the partition Name
-   */
-  public Map<String,String> getPartitionName() {
-    return partSpec;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
deleted file mode 100644
index 94a8836..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.api.Index;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-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;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
deleted file mode 100644
index 99105f3..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-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 org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreAddPartitionEvent extends PreEventContext {
-
-  private final Table table;
-  private final List<Partition> partitions;
-  private PartitionSpecProxy partitionSpecProxy;
-
-  public PreAddPartitionEvent (Table table, List<Partition> partitions, HMSHandler handler) {
-    super(PreEventType.ADD_PARTITION, handler);
-    this.table = table;
-    this.partitions = partitions;
-    this.partitionSpecProxy = null;
-  }
-
-  public PreAddPartitionEvent(Table table, Partition partition, HMSHandler handler) {
-    this(table, Arrays.asList(partition), handler);
-  }
-
-  /**
-   * Alternative constructor, using
-   */
-  public PreAddPartitionEvent(Table table, PartitionSpecProxy partitionSpecProxy, HMSHandler handler) {
-    this(table, (List<Partition>)null, handler);
-    this.partitionSpecProxy = partitionSpecProxy;
-  }
-
-  /**
-   * @return the partitions
-   */
-  public List<Partition> getPartitions() {
-    return partitions;
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable() {
-    return table ;
-  }
-
-  /**
-   * @return Iterator over partition-list.
-   */
-  public Iterator<Partition> getPartitionIterator() {
-    return partitionSpecProxy == null ? null : partitionSpecProxy.getPartitionIterator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java
deleted file mode 100644
index 62e2674..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Database;
-
-public class PreAlterDatabaseEvent extends PreEventContext {
-
-  private final Database oldDB, newDB;
-
-  public PreAlterDatabaseEvent(Database oldDB, Database newDB, HMSHandler handler) {
-    super (PreEventType.ALTER_DATABASE, handler);
-    this.oldDB = oldDB;
-    this.newDB = newDB;
-  }
-
-  /**
-   * @return the old db
-   */
-  public Database getOldDatabase () {
-    return oldDB;
-  }
-
-  /**
-   * @return the new db
-   */
-  public Database getNewDatabase() {
-    return newDB;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
deleted file mode 100644
index a2b7662..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.api.Index;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-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;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java
deleted file mode 100644
index 5f29699..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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 java.util.List;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Partition;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreAlterPartitionEvent extends PreEventContext {
-
-  private final String dbName;
-  private final String tableName;
-  private final List<String> oldPartVals;
-  private final Partition newPart;
-
-  public PreAlterPartitionEvent(String dbName, String tableName, List<String> oldPartVals,
-      Partition newPart, HMSHandler handler) {
-    super(PreEventType.ALTER_PARTITION, handler);
-    this.dbName = dbName;
-    this.tableName = tableName;
-    this.oldPartVals = oldPartVals;
-    this.newPart = newPart;
-  }
-
-  public String getDbName() {
-    return dbName;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public List<String> getOldPartVals() {
-    return oldPartVals;
-  }
-
-  /**
-   *
-   * @return the new partition
-   */
-  public Partition getNewPartition() {
-    return newPart;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java
deleted file mode 100644
index 4e82ad1..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreAlterTableEvent extends PreEventContext {
-
-  private final Table newTable;
-  private final Table oldTable;
-
-  public PreAlterTableEvent (Table oldTable, Table newTable, HMSHandler handler) {
-    super (PreEventType.ALTER_TABLE, handler);
-    this.oldTable = oldTable;
-    this.newTable = newTable;
-  }
-
-  /**
-   * @return the old table
-   */
-  public Table getOldTable() {
-    return oldTable;
-  }
-
-  /**
-   * @return the new table
-   */
-  public Table getNewTable() {
-    return newTable;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java
deleted file mode 100644
index 54f6dee..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreAuthorizationCallEvent extends PreEventContext {
-
-  public PreAuthorizationCallEvent (HMSHandler handler) {
-    super(PreEventType.AUTHORIZATION_API_CALL, handler);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java
deleted file mode 100644
index a5f87e1..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Database;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreCreateDatabaseEvent extends PreEventContext {
-
-  private final Database db;
-
-  public PreCreateDatabaseEvent (Database db, HMSHandler handler) {
-    super (PreEventType.CREATE_DATABASE, handler);
-    this.db = db;
-  }
-
-  /**
-   * @return the db
-   */
-  public Database getDatabase () {
-    return db;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
deleted file mode 100644
index 799ad97..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreCreateTableEvent extends PreEventContext {
-
-  private final Table table;
-
-  public PreCreateTableEvent(Table table, HMSHandler handler) {
-    super(PreEventType.CREATE_TABLE, handler);
-    this.table = table;
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable() {
-    return table;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
deleted file mode 100644
index adcca6f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Database;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreDropDatabaseEvent extends PreEventContext {
-
-  private final Database db;
-
-  public PreDropDatabaseEvent(Database db, HMSHandler handler) {
-    super(PreEventType.DROP_DATABASE, handler);
-    this.db = db;
-  }
-
-  /**
-   * @return the db
-   */
-  public Database getDatabase() {
-    return db;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
deleted file mode 100644
index c7fd44b..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.api.Index;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java
deleted file mode 100644
index e890ded..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-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.Collections;
-import java.util.Iterator;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreDropPartitionEvent extends PreEventContext {
-
-  private final Iterable<Partition> partitions;
-  private final Table table;
-  private final boolean deleteData;
-
-  public PreDropPartitionEvent (Table table,
-      Partition partition, boolean deleteData, HMSHandler handler) {
-    super (PreEventType.DROP_PARTITION, handler);
-    this.partitions = Collections.singletonList(partition);
-    this.table = table;
-    this.deleteData = deleteData;
-  }
-
-  /**
-   * @return the partitions
-   */
-  public Iterator<Partition> getPartitionIterator() {
-    return partitions.iterator();
-  }
-
- /**
-  * @return the table
-  */
-  public Table getTable() {
-    return table;
-  }
-
-  /**
-   * @return the deleteData flag
-   */
-  public boolean getDeleteData() {
-
-    return deleteData;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
deleted file mode 100644
index 6c7f169..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreDropTableEvent extends PreEventContext {
-
-  private final Table table;
-  private final boolean deleteData;
-
-  public PreDropTableEvent(Table table, boolean deleteData, HMSHandler handler) {
-    super(PreEventType.DROP_TABLE, handler);
-    this.table = table;
-    // In HiveMetaStore, the deleteData flag indicates whether DFS data should be
-    // removed on a drop.
-    this.deleteData = deleteData;
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable() {
-    return table;
-  }
-
-  /**
-   * @return the deleteData flag
-   */
-  public boolean getDeleteData() {
-    return deleteData;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
deleted file mode 100644
index 0bd2ea6..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-
-
-/**
- * Base class for all the events which are defined for metastore.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class PreEventContext {
-
-  public static enum PreEventType {
-    CREATE_TABLE,
-    DROP_TABLE,
-    ALTER_TABLE,
-    ADD_PARTITION,
-    DROP_PARTITION,
-    ALTER_PARTITION,
-    CREATE_DATABASE,
-    DROP_DATABASE,
-    LOAD_PARTITION_DONE,
-    AUTHORIZATION_API_CALL,
-    READ_TABLE,
-    READ_DATABASE,
-    ADD_INDEX,
-    ALTER_INDEX,
-    DROP_INDEX,
-    ALTER_DATABASE
-  }
-
-  private final PreEventType eventType;
-  private final HMSHandler handler;
-
-  public PreEventContext(PreEventType eventType, HMSHandler  handler) {
-    this.eventType = eventType;
-    this.handler = handler;
-  }
-
-  /**
-   * @return the event type
-   */
-  public PreEventType getEventType() {
-    return eventType;
-  }
-
-  /**
-   * @return the handler
-   */
-  public HMSHandler getHandler() {
-    return handler;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
deleted file mode 100644
index 82b72ed..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * 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 java.util.Map;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreLoadPartitionDoneEvent extends PreEventContext {
-
-  private final String dbName;
-  private final String tableName;
-  private final Map<String,String> partSpec;
-
-  public PreLoadPartitionDoneEvent(String dbName, String tableName,
-      Map<String, String> partSpec, HMSHandler handler) {
-    super(PreEventType.LOAD_PARTITION_DONE, handler);
-    this.dbName = dbName;
-    this.tableName = tableName;
-    this.partSpec = partSpec;
-  }
-
-  public String getDbName() {
-    return dbName;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * @return the partition Name
-   */
-  public Map<String,String> getPartitionName() {
-    return partSpec;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
deleted file mode 100644
index f223960..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Database;
-
-/**
- * Database read event
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreReadDatabaseEvent extends PreEventContext {
-  private final Database db;
-
-  public PreReadDatabaseEvent(Database db, HMSHandler handler) {
-    super(PreEventType.READ_DATABASE, handler);
-    this.db = db;
-  }
-
-  /**
-   * @return the db
-   */
-  public Database getDatabase() {
-    return db;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
deleted file mode 100644
index e9d353a..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-/**
- * Table read event
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class PreReadTableEvent extends PreEventContext {
-
-  private final Table table;
-
-  public PreReadTableEvent(Table table, HMSHandler handler) {
-    super(PreEventType.READ_TABLE, handler);
-    this.table = table;
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable() {
-    return table;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java
deleted file mode 100644
index 2eb14a1..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * 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.messaging;
-
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-
-public abstract class AddForeignKeyMessage extends EventMessage {
-  protected AddForeignKeyMessage() {
-    super(EventType.ADD_FOREIGNKEY);
-  }
-
-  /**
-   * Getter for list of foreign keys.
-   * @return List of SQLForeignKey
-   */
-  public abstract List<SQLForeignKey> getForeignKeys() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java
deleted file mode 100644
index 28ee610..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * 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.messaging;
-
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
-
-public abstract class AddNotNullConstraintMessage extends EventMessage {
-  protected AddNotNullConstraintMessage() {
-    super(EventType.ADD_NOTNULLCONSTRAINT);
-  }
-
-  /**
-   * Getter for list of not null constraints.
-   * @return List of SQLNotNullConstraint
-   */
-  public abstract List<SQLNotNullConstraint> getNotNullConstraints() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java
deleted file mode 100644
index 774c020..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-import java.util.List;
-import java.util.Map;
-
-public abstract class AddPartitionMessage extends EventMessage {
-
-  protected AddPartitionMessage() {
-    super(EventType.ADD_PARTITION);
-  }
-
-  /**
-   * Getter for name of table (where partitions are added).
-   * @return Table-name (String).
-   */
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  public abstract Table getTableObj() throws Exception;
-
-  /**
-   * Getter for list of partitions added.
-   * @return List of maps, where each map identifies values for each partition-key, for every added partition.
-   */
-  public abstract List<Map<String, String>> getPartitions ();
-
-  public abstract Iterable<Partition> getPartitionObjs() throws Exception;
-
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null)
-      throw new IllegalStateException("Table name unset.");
-    if (getPartitions() == null)
-      throw new IllegalStateException("Partition-list unset.");
-    return super.checkValid();
-  }
-
-  /**
-   * Get iterable of partition name and file lists created as a result of this DDL operation
-   *
-   * @return The iterable of partition PartitionFiles
-   */
-  public abstract Iterable<PartitionFiles> getPartitionFilesIter();
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java
deleted file mode 100644
index 0e899ad..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * 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.messaging;
-
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-
-public abstract class AddPrimaryKeyMessage extends EventMessage {
-  protected AddPrimaryKeyMessage() {
-    super(EventType.ADD_PRIMARYKEY);
-  }
-
-  /**
-   * Getter for list of primary keys.
-   * @return List of SQLPrimaryKey
-   */
-  public abstract List<SQLPrimaryKey> getPrimaryKeys() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java
deleted file mode 100644
index 8072d84f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * 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.messaging;
-
-import java.util.List;
-
-import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
-
-public abstract class AddUniqueConstraintMessage extends EventMessage {
-  protected AddUniqueConstraintMessage() {
-    super(EventType.ADD_UNIQUECONSTRAINT);
-  }
-
-  /**
-   * Getter for list of unique constraints.
-   * @return List of SQLUniqueConstraint
-   */
-  public abstract List<SQLUniqueConstraint> getUniqueConstraints() throws Exception;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java
deleted file mode 100644
index e72a94b..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Index;
-
-public abstract class AlterIndexMessage extends EventMessage {
-
-  public abstract Index getIndexObjBefore() throws Exception;
-
-  public abstract Index getIndexObjAfter() throws Exception;
-
-  protected AlterIndexMessage() {
-    super(EventType.ALTER_INDEX);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
deleted file mode 100644
index 077c9f7..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-import java.util.Map;
-
-public abstract class AlterPartitionMessage extends EventMessage {
-
-  protected AlterPartitionMessage() {
-    super(EventType.ALTER_PARTITION);
-  }
-
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  public abstract boolean getIsTruncateOp();
-
-  public abstract Map<String,String> getKeyValues();
-
-  public abstract Table getTableObj() throws Exception;
-
-  public abstract Partition getPtnObjBefore() throws Exception;
-
-  public abstract Partition getPtnObjAfter() throws Exception;
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null) throw new IllegalStateException("Table name unset.");
-    if (getKeyValues() == null) throw new IllegalStateException("Partition values unset");
-    try {
-      if (getTableObj() == null){
-        throw new IllegalStateException("Table object not set.");
-      }
-      if (getPtnObjAfter() == null){
-        throw new IllegalStateException("Partition object(after) not set.");
-      }
-      if (getPtnObjBefore() == null){
-        throw new IllegalStateException("Partition object(before) not set.");
-      }
-    } catch (Exception e) {
-      if (! (e instanceof IllegalStateException)){
-        throw new IllegalStateException("Event not set up correctly",e);
-      } else {
-        throw (IllegalStateException) e;
-      }
-    }
-    return super.checkValid();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
deleted file mode 100644
index 58f01fe..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-
-public abstract class AlterTableMessage extends EventMessage {
-
-  protected AlterTableMessage() {
-    super(EventType.ALTER_TABLE);
-  }
-
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  public abstract boolean getIsTruncateOp();
-
-  public abstract Table getTableObjBefore() throws Exception;
-
-  public abstract Table getTableObjAfter() throws Exception;
-
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null) throw new IllegalStateException("Table name unset.");
-    try {
-      if (getTableObjAfter() == null){
-        throw new IllegalStateException("Table object(after) not set.");
-      }
-      if (getTableObjBefore() == null){
-        throw new IllegalStateException("Table object(before) not set.");
-      }
-    } catch (Exception e) {
-      if (! (e instanceof IllegalStateException)){
-        throw new IllegalStateException("Event not set up correctly",e);
-      } else {
-        throw (IllegalStateException) e;
-      }
-    }
-    return super.checkValid();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java
deleted file mode 100644
index 7614298..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * 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.messaging;
-
-public abstract class CreateDatabaseMessage extends EventMessage {
-
-  protected CreateDatabaseMessage() {
-    super(EventType.CREATE_DATABASE);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java
deleted file mode 100644
index d94a3f0..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Function;
-
-public abstract class CreateFunctionMessage extends EventMessage {
-
-  protected CreateFunctionMessage() {
-    super(EventType.CREATE_FUNCTION);
-  }
-
-  public abstract Function getFunctionObj() throws Exception;
-
-  @Override
-  public EventMessage checkValid() {
-    try {
-      if (getFunctionObj() == null)
-        throw new IllegalStateException("Function object unset.");
-    } catch (Exception e) {
-      if (! (e instanceof IllegalStateException)){
-        throw new IllegalStateException("Event not set up correctly", e);
-      } else {
-        throw (IllegalStateException) e;
-      }
-    }
-    return super.checkValid();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java
deleted file mode 100644
index 3ce0d62..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Index;
-
-public abstract class CreateIndexMessage extends EventMessage {
-
-  protected CreateIndexMessage() {
-    super(EventType.CREATE_INDEX);
-  }
-
-  public abstract Index getIndexObj() throws Exception;
-
-  @Override
-  public EventMessage checkValid() {
-    try {
-      if (getIndexObj() == null)
-        throw new IllegalStateException("Function object unset.");
-    } catch (Exception e) {
-      if (! (e instanceof IllegalStateException)){
-        throw new IllegalStateException("Event not set up correctly", e);
-      } else {
-        throw (IllegalStateException) e;
-      }
-    }
-    return super.checkValid();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java
deleted file mode 100644
index b75caa6..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.messaging;
-
-import org.apache.hadoop.hive.metastore.api.Table;
-
-public abstract class CreateTableMessage extends EventMessage {
-
-  protected CreateTableMessage() {
-    super(EventType.CREATE_TABLE);
-  }
-
-  /**
-   * Getter for the name of table created
-   * @return Table-name (String).
-   */
-  public abstract String getTable();
-
-  public abstract String getTableType();
-
-  public abstract Table getTableObj() throws Exception;
-
-  /**
-   * Get list of files created as a result of this DML operation
-   *
-   * @return The iterable of files
-   */
-  public abstract Iterable<String> getFiles();
-
-  @Override
-  public EventMessage checkValid() {
-    if (getTable() == null)
-      throw new IllegalStateException("Table name unset.");
-    return super.checkValid();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java
deleted file mode 100644
index 6e691e9..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * 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.messaging;
-
-public abstract class DropConstraintMessage extends EventMessage {
-  protected DropConstraintMessage() {
-    super(EventType.DROP_CONSTRAINT);
-  }
-
-  public abstract String getTable();
-
-  public abstract String getConstraint();
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java
deleted file mode 100644
index fa6da38..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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.messaging;
-
-public abstract class DropDatabaseMessage extends EventMessage {
-
-  protected DropDatabaseMessage() {
-    super(EventType.DROP_DATABASE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java
deleted file mode 100644
index 2b45d40..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * 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.messaging;
-
-public abstract class DropFunctionMessage extends EventMessage {
-
-  public abstract String getFunctionName();
-
-  protected DropFunctionMessage() {
-    super(EventType.DROP_FUNCTION);
-  }
-
-  @Override
-  public EventMessage checkValid() {
-    if (getFunctionName() == null){
-      throw new IllegalStateException("Function name unset.");
-    }
-    return super.checkValid();
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java
deleted file mode 100644
index 5997f92..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * 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.messaging;
-
-public abstract class DropIndexMessage extends EventMessage {
-
-  public abstract String getIndexName();
-  public abstract String getOrigTableName();
-  public abstract String getIndexTableName();
-
-  protected DropIndexMessage() {
-    super(EventType.DROP_INDEX);
-  }
-
-  @Override
-  public EventMessage checkValid() {
-    if (getIndexName() == null){
-      throw new IllegalStateException("Index name unset.");
-    }
-    if (getOrigTableName() == null){
-      throw new IllegalStateException("Index original table name unset.");
-    }
-    // NOTE: we do not do a not-null check on getIndexTableName,
-    // since, per the index design wiki, it can actually be null.
-
-    return super.checkValid();
-  }
-
-}
\ No newline at end of file


[04/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java
new file mode 100644
index 0000000..090d175
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java
@@ -0,0 +1,52 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ConfigChangeEvent extends ListenerEvent {
+
+  private final String key;
+  private final String oldValue;
+  private final String newValue;
+
+  public ConfigChangeEvent(IHMSHandler handler, String key,
+                           String oldValue, String newValue) {
+    super(true, handler);
+    this.key = key;
+    this.oldValue = oldValue;
+    this.newValue = newValue;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public String getOldValue() {
+    return oldValue;
+  }
+
+  public String getNewValue() {
+    return newValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
new file mode 100644
index 0000000..e2c3ee3
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CreateDatabaseEvent extends ListenerEvent {
+
+  private final Database db;
+
+  public CreateDatabaseEvent (Database db, boolean status, IHMSHandler handler) {
+    super (status, handler);
+    this.db = db;
+  }
+
+  /**
+   * @return the db
+   */
+  public Database getDatabase () {
+    return db;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
new file mode 100644
index 0000000..c1403ec
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Function;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CreateFunctionEvent extends ListenerEvent {
+
+  private final Function function;
+
+  public CreateFunctionEvent (Function function, boolean status, IHMSHandler handler) {
+    super (status, handler);
+    this.function = function;
+  }
+
+  /**
+   * @return the function
+   */
+  public Function getFunction () {
+    return function;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
new file mode 100644
index 0000000..4f5e887
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CreateTableEvent extends ListenerEvent {
+
+  private final Table table;
+
+  public CreateTableEvent (Table table, boolean status, IHMSHandler handler) {
+    super (status, handler);
+    this.table = table;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable () {
+    return table;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
new file mode 100644
index 0000000..4c5918f
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
@@ -0,0 +1,51 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropConstraintEvent extends ListenerEvent {
+
+  private final String dbName;
+  private final String tableName;
+  private final String constraintName;
+  public DropConstraintEvent(String dbName, String tableName, String constraintName,
+      boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.constraintName = constraintName;
+  }
+
+  public String getDbName() {
+    return dbName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getConstraintName() {
+    return constraintName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
new file mode 100644
index 0000000..94fe264
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropDatabaseEvent extends ListenerEvent {
+
+  private final Database db;
+
+  public DropDatabaseEvent(Database db, boolean status, IHMSHandler handler) {
+    super (status, handler);
+    this.db = db;
+  }
+
+  /**
+   * @return the db
+   */
+  public Database getDatabase() {
+    return db;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
new file mode 100644
index 0000000..e8728a5
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Function;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropFunctionEvent extends ListenerEvent {
+
+  private final Function function;
+
+  public DropFunctionEvent(Function function, boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.function = function;
+  }
+
+  /**
+   * @return the function
+   */
+  public Function getFunction() {
+    return function;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
new file mode 100644
index 0000000..4d69834
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
@@ -0,0 +1,40 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropIndexEvent extends ListenerEvent {
+
+  private final Index index;
+
+  public DropIndexEvent(Index index, boolean status, IHMSHandler handler) {
+    super(status, handler);
+    this.index = index;
+  }
+
+  public Index getIndex() {
+    return index;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
new file mode 100644
index 0000000..dce9df3
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
@@ -0,0 +1,70 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropPartitionEvent extends ListenerEvent {
+
+  private final Table table;
+  private final Iterable<Partition> partitions;
+  private final boolean deleteData;
+
+  public DropPartitionEvent (Table table,
+      Partition partition, boolean status, boolean deleteData, IHMSHandler handler) {
+    super (status, handler);
+    this.table = table;
+    this.partitions = Collections.singletonList(partition);
+    // In HiveMetaStore, the deleteData flag indicates whether DFS data should be
+    // removed on a drop.
+    this.deleteData = deleteData;
+  }
+
+  /**
+   * @return the partitions
+   */
+  public Iterator<Partition> getPartitionIterator() {
+    return partitions.iterator();
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+
+    return table;
+  }
+
+  /**
+   * @return the deleteData flag
+   */
+  public boolean getDeleteData() {
+
+    return deleteData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
new file mode 100644
index 0000000..9152232
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
@@ -0,0 +1,54 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropTableEvent extends ListenerEvent {
+
+  private final Table table;
+  private final boolean deleteData;
+
+  public DropTableEvent(Table table, boolean status, boolean deleteData, IHMSHandler handler) {
+    super(status, handler);
+    this.table = table;
+    // In HiveMetaStore, the deleteData flag indicates whether DFS data should be
+    // removed on a drop.
+    this.deleteData = deleteData;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+    return table;
+  }
+
+  /**
+   * @return the deleteData flag
+   */
+  public boolean getDeleteData() {
+    return deleteData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java
new file mode 100644
index 0000000..230c0d3
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/EventCleanerTask.java
@@ -0,0 +1,52 @@
+/*
+ * 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 java.util.TimerTask;
+
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.metastore.RawStore;
+
+public class EventCleanerTask extends TimerTask{
+
+  public static final Logger LOG = LoggerFactory.getLogger(EventCleanerTask.class);
+  private final IHMSHandler handler;
+
+  public EventCleanerTask(IHMSHandler handler) {
+    super();
+    this.handler = handler;
+  }
+
+  @Override
+  public void run() {
+
+    try {
+      RawStore ms = handler.getMS();
+      long deleteCnt = ms.cleanupEvents();
+
+      if (deleteCnt > 0L){
+        LOG.info("Number of events deleted from event Table: "+deleteCnt);
+      }
+    } catch (Exception e) {
+      LOG.error("Exception while trying to delete events ", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
new file mode 100644
index 0000000..b963f78
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
@@ -0,0 +1,129 @@
+/*
+ * 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 com.google.common.collect.Lists;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.ClientCapabilities;
+import org.apache.hadoop.hive.metastore.api.ClientCapability;
+import org.apache.hadoop.hive.metastore.api.GetTableRequest;
+import org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.thrift.TException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InsertEvent extends ListenerEvent {
+
+  private final Table tableObj;
+  private final Partition ptnObj;
+  private final boolean replace;
+  private final List<String> files;
+  private List<String> fileChecksums = new ArrayList<>();
+
+  /**
+   *
+   * @param db name of the database the table is in
+   * @param table name of the table being inserted into
+   * @param partVals list of partition values, can be null
+   * @param insertData the inserted files and their checksums
+   * @param status status of insert, true = success, false = failure
+   * @param handler handler that is firing the event
+   */
+  public InsertEvent(String db, String table, List<String> partVals,
+      InsertEventRequestData insertData, boolean status, IHMSHandler handler) throws MetaException,
+      NoSuchObjectException {
+    super(status, handler);
+
+    GetTableRequest req = new GetTableRequest(db, table);
+    // TODO MS-SPLIT Switch this back once HiveMetaStoreClient is moved.
+    //req.setCapabilities(HiveMetaStoreClient.TEST_VERSION);
+    req.setCapabilities(new ClientCapabilities(
+      Lists.newArrayList(ClientCapability.TEST_CAPABILITY)));
+    try {
+      this.tableObj = handler.get_table_req(req).getTable();
+      if (partVals != null) {
+        this.ptnObj = handler.get_partition(db, table, partVals);
+      } else {
+        this.ptnObj = null;
+      }
+    } catch (NoSuchObjectException e) {
+      // This is to mimic previous behavior where NoSuchObjectException was thrown through this
+      // method.
+      throw e;
+    } catch (TException e) {
+      throw MetaStoreUtils.newMetaException(e);
+    }
+
+    // If replace flag is not set by caller, then by default set it to true to maintain backward compatibility
+    this.replace = (insertData.isSetReplace() ? insertData.isReplace() : true);
+    this.files = insertData.getFilesAdded();
+    if (insertData.isSetFilesAddedChecksum()) {
+      fileChecksums = insertData.getFilesAddedChecksum();
+    }
+  }
+
+  /**
+   * @return Table object
+   */
+  public Table getTableObj() {
+    return tableObj;
+  }
+
+  /**
+   * @return Partition object
+   */
+  public Partition getPartitionObj() {
+    return ptnObj;
+  }
+
+  /**
+   * @return The replace flag.
+   */
+  public boolean isReplace() {
+    return replace;
+  }
+
+  /**
+   * Get list of files created as a result of this DML operation
+   *
+   * @return list of new files
+   */
+  public List<String> getFiles() {
+    return files;
+  }
+
+  /**
+   * Get a list of file checksums corresponding to the files created (if available)
+   *
+   * @return
+   */
+  public List<String> getFileChecksums() {
+    return fileChecksums;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
new file mode 100644
index 0000000..e031dbb
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
@@ -0,0 +1,178 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Base class for all the events which are defined for metastore.
+ *
+ * This class is not thread-safe and not expected to be called in parallel.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+@NotThreadSafe
+public abstract class ListenerEvent {
+
+  /**
+   * status of the event, whether event was successful or not.
+   */
+  private final boolean status;
+  private final IHMSHandler handler;
+
+  /**
+   * Key/value parameters used by listeners to store notifications results
+   * i.e. DbNotificationListener sets a DB_NOTIFICATION_EVENT_ID.
+   *
+   * NotThreadSafe: The parameters map is not expected to be access in parallel by Hive, so keep it thread-unsafe
+   * to avoid locking overhead.
+   */
+  private Map<String, String> parameters;
+
+  /** For performance concerns, it is preferable to cache the unmodifiable parameters variable that will be returned on the
+   * {@link #getParameters()} method. It is expected that {@link #putParameter(String, String)} is called less times
+   * than {@link #getParameters()}, so performance may be better by using this cache.
+   */
+  private Map<String, String> unmodifiableParameters;
+
+  // Listener parameters aren't expected to have many values. So far only
+  // DbNotificationListener will add a parameter; let's set a low initial capacity for now.
+  // If we find out many parameters are added, then we can adjust or remove this initial capacity.
+  private static final int PARAMETERS_INITIAL_CAPACITY = 1;
+
+  // Properties passed by the client, to be used in execution hooks.
+  private EnvironmentContext environmentContext = null;
+
+  public ListenerEvent(boolean status, IHMSHandler handler) {
+    super();
+    this.status = status;
+    this.handler = handler;
+    this.parameters = new HashMap<>(PARAMETERS_INITIAL_CAPACITY);
+    updateUnmodifiableParameters();
+  }
+
+  /**
+   * @return the status of event.
+   */
+  public boolean getStatus() {
+    return status;
+  }
+
+  /**
+   * Set the environment context of the event.
+   *
+   * @param environmentContext An EnvironmentContext object that contains environment parameters sent from
+   *                           the HMS client.
+   */
+  public void setEnvironmentContext(EnvironmentContext environmentContext) {
+    this.environmentContext = environmentContext;
+  }
+
+  /**
+   * @return environment properties of the event
+   */
+  public EnvironmentContext getEnvironmentContext() {
+    return environmentContext;
+  }
+
+  /**
+   * @return the handler
+   */
+  public IHMSHandler getIHMSHandler() {
+    return handler;
+  }
+
+  /**
+   * Return all parameters of the listener event. Parameters are read-only (unmodifiable map). If a new parameter
+   * must be added, please use the putParameter() method.
+   *
+   *
+   * @return A map object with all parameters.
+   */
+  public final Map<String, String> getParameters() {
+    return unmodifiableParameters;
+  }
+
+  /**
+   * Put a new parameter to the listener event.
+   *
+   * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration
+   * between listeners setting the same parameters.
+   *
+   * @param name Name of the parameter.
+   * @param value Value of the parameter.
+   * @throws IllegalStateException if a parameter already exists.
+   */
+  public void putParameter(String name, String value) {
+    putParameterIfAbsent(name, value);
+    updateUnmodifiableParameters();
+  }
+
+  /**
+   * Put a new set the parameters to the listener event.
+   *
+   * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration
+   * between listeners setting the same parameters.
+   *
+   * @param parameters A Map object with the a set of parameters.
+   * @throws IllegalStateException if a parameter already exists.
+   */
+  public void putParameters(final Map<String, String> parameters) {
+    if (parameters != null) {
+      for (Map.Entry<String, String> entry : parameters.entrySet()) {
+        putParameterIfAbsent(entry.getKey(), entry.getValue());
+      }
+
+      updateUnmodifiableParameters();
+    }
+  }
+
+  /**
+   * Put a parameter to the listener event only if the parameter is absent.
+   *
+   * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration
+   * between listeners setting the same parameters.
+   *
+   * @param name Name of the parameter.
+   * @param value Value of the parameter.
+   * @throws IllegalStateException if a parameter already exists.
+   */
+  private void putParameterIfAbsent(String name, String value) {
+    if (parameters.containsKey(name)) {
+      throw new IllegalStateException("Invalid attempt to overwrite a read-only parameter: " + name);
+    }
+
+    parameters.put(name, value);
+  }
+
+  /**
+   * Keeps a cache of unmodifiable parameters returned by the getParameters() method.
+   */
+  private void updateUnmodifiableParameters() {
+    unmodifiableParameters = Collections.unmodifiableMap(parameters);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
new file mode 100644
index 0000000..3128a8f
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
@@ -0,0 +1,57 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class LoadPartitionDoneEvent extends ListenerEvent {
+
+  private final Table table;
+
+  private final Map<String,String> partSpec;
+
+  public LoadPartitionDoneEvent(boolean status, Table table,
+      Map<String,String> partSpec, IHMSHandler handler) {
+    super(status, handler);
+    this.table = table;
+    this.partSpec = partSpec;
+  }
+
+  /**
+   * @return the tblName
+   */
+  public Table getTable() {
+    return table;
+  }
+
+  /**
+   * @return the partition Name
+   */
+  public Map<String,String> getPartitionName() {
+    return partSpec;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
new file mode 100644
index 0000000..d9a53f8
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddIndexEvent.java
@@ -0,0 +1,41 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAddIndexEvent extends PreEventContext {
+
+private final Index table;
+
+  public PreAddIndexEvent(Index table, IHMSHandler handler) {
+    super(PreEventType.ADD_INDEX, handler);
+    this.table = table;
+  }
+
+  public Index getIndex() {
+    return table;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
new file mode 100644
index 0000000..b5c4607
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAddPartitionEvent.java
@@ -0,0 +1,79 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+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 java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAddPartitionEvent extends PreEventContext {
+
+  private final Table table;
+  private final List<Partition> partitions;
+  private PartitionSpecProxy partitionSpecProxy;
+
+  public PreAddPartitionEvent (Table table, List<Partition> partitions, IHMSHandler handler) {
+    super(PreEventType.ADD_PARTITION, handler);
+    this.table = table;
+    this.partitions = partitions;
+    this.partitionSpecProxy = null;
+  }
+
+  public PreAddPartitionEvent(Table table, Partition partition, IHMSHandler handler) {
+    this(table, Arrays.asList(partition), handler);
+  }
+
+  /**
+   * Alternative constructor, using
+   */
+  public PreAddPartitionEvent(Table table, PartitionSpecProxy partitionSpecProxy, IHMSHandler handler) {
+    this(table, (List<Partition>)null, handler);
+    this.partitionSpecProxy = partitionSpecProxy;
+  }
+
+  /**
+   * @return the partitions
+   */
+  public List<Partition> getPartitions() {
+    return partitions;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+    return table ;
+  }
+
+  /**
+   * @return Iterator over partition-list.
+   */
+  public Iterator<Partition> getPartitionIterator() {
+    return partitionSpecProxy == null ? null : partitionSpecProxy.getPartitionIterator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java
new file mode 100644
index 0000000..6ac1a4e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterDatabaseEvent.java
@@ -0,0 +1,47 @@
+/*
+ * 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.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+public class PreAlterDatabaseEvent extends PreEventContext {
+
+  private final Database oldDB, newDB;
+
+  public PreAlterDatabaseEvent(Database oldDB, Database newDB, IHMSHandler handler) {
+    super (PreEventType.ALTER_DATABASE, handler);
+    this.oldDB = oldDB;
+    this.newDB = newDB;
+  }
+
+  /**
+   * @return the old db
+   */
+  public Database getOldDatabase () {
+    return oldDB;
+  }
+
+  /**
+   * @return the new db
+   */
+  public Database getNewDatabase() {
+    return newDB;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
new file mode 100644
index 0000000..2e9d9d7
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterIndexEvent.java
@@ -0,0 +1,47 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAlterIndexEvent extends PreEventContext {
+
+  private final Index newIndex;
+  private final Index oldIndex;
+
+  public PreAlterIndexEvent(Index oldIndex, Index newIndex, IHMSHandler handler) {
+    super(PreEventType.ALTER_INDEX, handler);
+    this.oldIndex = oldIndex;
+    this.newIndex = newIndex;
+  }
+
+  public Index getOldIndex() {
+    return oldIndex;
+  }
+
+  public Index getNewIndex() {
+    return newIndex;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java
new file mode 100644
index 0000000..a269563
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterPartitionEvent.java
@@ -0,0 +1,65 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Partition;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAlterPartitionEvent extends PreEventContext {
+
+  private final String dbName;
+  private final String tableName;
+  private final List<String> oldPartVals;
+  private final Partition newPart;
+
+  public PreAlterPartitionEvent(String dbName, String tableName, List<String> oldPartVals,
+      Partition newPart, IHMSHandler handler) {
+    super(PreEventType.ALTER_PARTITION, handler);
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.oldPartVals = oldPartVals;
+    this.newPart = newPart;
+  }
+
+  public String getDbName() {
+    return dbName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public List<String> getOldPartVals() {
+    return oldPartVals;
+  }
+
+  /**
+   *
+   * @return the new partition
+   */
+  public Partition getNewPartition() {
+    return newPart;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java
new file mode 100644
index 0000000..31db0e9
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAlterTableEvent.java
@@ -0,0 +1,53 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAlterTableEvent extends PreEventContext {
+
+  private final Table newTable;
+  private final Table oldTable;
+
+  public PreAlterTableEvent (Table oldTable, Table newTable, IHMSHandler handler) {
+    super (PreEventType.ALTER_TABLE, handler);
+    this.oldTable = oldTable;
+    this.newTable = newTable;
+  }
+
+  /**
+   * @return the old table
+   */
+  public Table getOldTable() {
+    return oldTable;
+  }
+
+  /**
+   * @return the new table
+   */
+  public Table getNewTable() {
+    return newTable;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java
new file mode 100644
index 0000000..1205fc9
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreAuthorizationCallEvent.java
@@ -0,0 +1,33 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreAuthorizationCallEvent extends PreEventContext {
+
+  public PreAuthorizationCallEvent (IHMSHandler handler) {
+    super(PreEventType.AUTHORIZATION_API_CALL, handler);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java
new file mode 100644
index 0000000..86ff1a0
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateDatabaseEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreCreateDatabaseEvent extends PreEventContext {
+
+  private final Database db;
+
+  public PreCreateDatabaseEvent (Database db, IHMSHandler handler) {
+    super (PreEventType.CREATE_DATABASE, handler);
+    this.db = db;
+  }
+
+  /**
+   * @return the db
+   */
+  public Database getDatabase () {
+    return db;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
new file mode 100644
index 0000000..bd3aa3d
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreCreateTableEvent extends PreEventContext {
+
+  private final Table table;
+
+  public PreCreateTableEvent(Table table, IHMSHandler handler) {
+    super(PreEventType.CREATE_TABLE, handler);
+    this.table = table;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+    return table;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
new file mode 100644
index 0000000..f5ebbcf
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropDatabaseEvent extends PreEventContext {
+
+  private final Database db;
+
+  public PreDropDatabaseEvent(Database db, IHMSHandler handler) {
+    super(PreEventType.DROP_DATABASE, handler);
+    this.db = db;
+  }
+
+  /**
+   * @return the db
+   */
+  public Database getDatabase() {
+    return db;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
new file mode 100644
index 0000000..ad570db
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropIndexEvent.java
@@ -0,0 +1,40 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Index;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropIndexEvent extends PreEventContext {
+
+  private final Index index;
+
+  public PreDropIndexEvent(Index index, IHMSHandler handler) {
+    super(PreEventType.DROP_INDEX, handler);
+    this.index = index;
+  }
+
+  public Index getIndex() {
+    return index;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java
new file mode 100644
index 0000000..a0daf31
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropPartitionEvent.java
@@ -0,0 +1,67 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropPartitionEvent extends PreEventContext {
+
+  private final Iterable<Partition> partitions;
+  private final Table table;
+  private final boolean deleteData;
+
+  public PreDropPartitionEvent (Table table, Partition partition, boolean deleteData,
+                                IHMSHandler handler) {
+    super (PreEventType.DROP_PARTITION, handler);
+    this.partitions = Collections.singletonList(partition);
+    this.table = table;
+    this.deleteData = deleteData;
+  }
+
+  /**
+   * @return the partitions
+   */
+  public Iterator<Partition> getPartitionIterator() {
+    return partitions.iterator();
+  }
+
+ /**
+  * @return the table
+  */
+  public Table getTable() {
+    return table;
+  }
+
+  /**
+   * @return the deleteData flag
+   */
+  public boolean getDeleteData() {
+
+    return deleteData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
new file mode 100644
index 0000000..f06fdcd
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
@@ -0,0 +1,55 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropTableEvent extends PreEventContext {
+
+  private final Table table;
+  private final boolean deleteData;
+
+  public PreDropTableEvent(Table table, boolean deleteData, IHMSHandler handler) {
+    super(PreEventType.DROP_TABLE, handler);
+    this.table = table;
+    // In HiveMetaStore, the deleteData flag indicates whether DFS data should be
+    // removed on a drop.
+    this.deleteData = deleteData;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+    return table;
+  }
+
+  /**
+   * @return the deleteData flag
+   */
+  public boolean getDeleteData() {
+    return deleteData;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
new file mode 100644
index 0000000..a3f4836
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
@@ -0,0 +1,73 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+
+/**
+ * Base class for all the events which are defined for metastore.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class PreEventContext {
+
+  public enum PreEventType {
+    CREATE_TABLE,
+    DROP_TABLE,
+    ALTER_TABLE,
+    ADD_PARTITION,
+    DROP_PARTITION,
+    ALTER_PARTITION,
+    CREATE_DATABASE,
+    DROP_DATABASE,
+    LOAD_PARTITION_DONE,
+    AUTHORIZATION_API_CALL,
+    READ_TABLE,
+    READ_DATABASE,
+    ADD_INDEX,
+    ALTER_INDEX,
+    DROP_INDEX,
+    ALTER_DATABASE
+  }
+
+  private final PreEventType eventType;
+  private final IHMSHandler handler;
+
+  public PreEventContext(PreEventType eventType, IHMSHandler  handler) {
+    this.eventType = eventType;
+    this.handler = handler;
+  }
+
+  /**
+   * @return the event type
+   */
+  public PreEventType getEventType() {
+    return eventType;
+  }
+
+  /**
+   * @return the handler
+   */
+  public IHMSHandler getHandler() {
+    return handler;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
new file mode 100644
index 0000000..999ec31
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
@@ -0,0 +1,58 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+
+import java.util.Map;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreLoadPartitionDoneEvent extends PreEventContext {
+
+  private final String dbName;
+  private final String tableName;
+  private final Map<String,String> partSpec;
+
+  public PreLoadPartitionDoneEvent(String dbName, String tableName,
+      Map<String, String> partSpec, IHMSHandler handler) {
+    super(PreEventType.LOAD_PARTITION_DONE, handler);
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.partSpec = partSpec;
+  }
+
+  public String getDbName() {
+    return dbName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  /**
+   * @return the partition Name
+   */
+  public Map<String,String> getPartitionName() {
+    return partSpec;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
new file mode 100644
index 0000000..88f3855
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
@@ -0,0 +1,46 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+/**
+ * Database read event
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreReadDatabaseEvent extends PreEventContext {
+  private final Database db;
+
+  public PreReadDatabaseEvent(Database db, IHMSHandler handler) {
+    super(PreEventType.READ_DATABASE, handler);
+    this.db = db;
+  }
+
+  /**
+   * @return the db
+   */
+  public Database getDatabase() {
+    return db;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
new file mode 100644
index 0000000..beec72b
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
@@ -0,0 +1,47 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+/**
+ * Table read event
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreReadTableEvent extends PreEventContext {
+
+  private final Table table;
+
+  public PreReadTableEvent(Table table, IHMSHandler handler) {
+    super(PreEventType.READ_TABLE, handler);
+    this.table = table;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+    return table;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java
new file mode 100644
index 0000000..db8a7d7
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddForeignKeyMessage.java
@@ -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.messaging;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+
+public abstract class AddForeignKeyMessage extends EventMessage {
+  protected AddForeignKeyMessage() {
+    super(EventType.ADD_FOREIGNKEY);
+  }
+
+  /**
+   * Getter for list of foreign keys.
+   * @return List of SQLForeignKey
+   */
+  public abstract List<SQLForeignKey> getForeignKeys() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java
new file mode 100644
index 0000000..07fbe64
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddNotNullConstraintMessage.java
@@ -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.messaging;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+
+public abstract class AddNotNullConstraintMessage extends EventMessage {
+  protected AddNotNullConstraintMessage() {
+    super(EventType.ADD_NOTNULLCONSTRAINT);
+  }
+
+  /**
+   * Getter for list of not null constraints.
+   * @return List of SQLNotNullConstraint
+   */
+  public abstract List<SQLNotNullConstraint> getNotNullConstraints() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java
new file mode 100644
index 0000000..3262b52
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPartitionMessage.java
@@ -0,0 +1,68 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.List;
+import java.util.Map;
+
+public abstract class AddPartitionMessage extends EventMessage {
+
+  protected AddPartitionMessage() {
+    super(EventType.ADD_PARTITION);
+  }
+
+  /**
+   * Getter for name of table (where partitions are added).
+   * @return Table-name (String).
+   */
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  public abstract Table getTableObj() throws Exception;
+
+  /**
+   * Getter for list of partitions added.
+   * @return List of maps, where each map identifies values for each partition-key, for every added partition.
+   */
+  public abstract List<Map<String, String>> getPartitions ();
+
+  public abstract Iterable<Partition> getPartitionObjs() throws Exception;
+
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null)
+      throw new IllegalStateException("Table name unset.");
+    if (getPartitions() == null)
+      throw new IllegalStateException("Partition-list unset.");
+    return super.checkValid();
+  }
+
+  /**
+   * Get iterable of partition name and file lists created as a result of this DDL operation
+   *
+   * @return The iterable of partition PartitionFiles
+   */
+  public abstract Iterable<PartitionFiles> getPartitionFilesIter();
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java
new file mode 100644
index 0000000..37f2b46
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddPrimaryKeyMessage.java
@@ -0,0 +1,35 @@
+/*
+ * 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.messaging;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+
+public abstract class AddPrimaryKeyMessage extends EventMessage {
+  protected AddPrimaryKeyMessage() {
+    super(EventType.ADD_PRIMARYKEY);
+  }
+
+  /**
+   * Getter for list of primary keys.
+   * @return List of SQLPrimaryKey
+   */
+  public abstract List<SQLPrimaryKey> getPrimaryKeys() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java
new file mode 100644
index 0000000..5c71f22
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AddUniqueConstraintMessage.java
@@ -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.messaging;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+
+public abstract class AddUniqueConstraintMessage extends EventMessage {
+  protected AddUniqueConstraintMessage() {
+    super(EventType.ADD_UNIQUECONSTRAINT);
+  }
+
+  /**
+   * Getter for list of unique constraints.
+   * @return List of SQLUniqueConstraint
+   */
+  public abstract List<SQLUniqueConstraint> getUniqueConstraints() throws Exception;
+}
\ No newline at end of file


[10/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index ef097ac..a491789 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -22,13 +22,8 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
-import java.math.BigDecimal;
-import java.math.BigInteger;
 import java.net.URL;
 import java.net.URLClassLoader;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -40,30 +35,16 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.google.common.base.Predicates;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.collections.ListUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.api.Decimal;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -81,15 +62,12 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregator;
-import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregatorFactory;
 import org.apache.hadoop.hive.metastore.columnstats.merge.ColumnStatsMerger;
 import org.apache.hadoop.hive.metastore.columnstats.merge.ColumnStatsMergerFactory;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
@@ -194,7 +172,7 @@ public class MetaStoreUtils {
    * @param partParams
    * @return True if the passed Parameters Map contains values for all "Fast Stats".
    */
-  public static boolean containsAllFastStats(Map<String, String> partParams) {
+  private static boolean containsAllFastStats(Map<String, String> partParams) {
     for (String stat : StatsSetupConst.fastStats) {
       if (!partParams.containsKey(stat)) {
         return false;
@@ -203,12 +181,12 @@ public class MetaStoreUtils {
     return true;
   }
 
-  public static boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh,
+  static boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh,
       boolean madeDir, EnvironmentContext environmentContext) throws MetaException {
     return updateTableStatsFast(db, tbl, wh, madeDir, false, environmentContext);
   }
 
-  public static boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh,
+  private static boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh,
       boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
     if (tbl.getPartitionKeysSize() == 0) {
       // Update stats only when unpartitioned
@@ -288,70 +266,12 @@ public class MetaStoreUtils {
     params.put(StatsSetupConst.TOTAL_SIZE, Long.toString(tableSize));
   }
 
-  // check if stats need to be (re)calculated
-  public static boolean requireCalStats(Configuration hiveConf, Partition oldPart,
-    Partition newPart, Table tbl, EnvironmentContext environmentContext) {
-
-    if (environmentContext != null
-        && environmentContext.isSetProperties()
-        && StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(
-            StatsSetupConst.DO_NOT_UPDATE_STATS))) {
-      return false;
-    }
-
-    if (MetaStoreUtils.isView(tbl)) {
-      return false;
-    }
-
-    if  (oldPart == null && newPart == null) {
-      return true;
-    }
-
-    // requires to calculate stats if new partition doesn't have it
-    if ((newPart == null) || (newPart.getParameters() == null)
-        || !containsAllFastStats(newPart.getParameters())) {
-      return true;
-    }
-
-    if (environmentContext != null && environmentContext.isSetProperties()) {
-      String statsType = environmentContext.getProperties().get(StatsSetupConst.STATS_GENERATED);
-      // no matter STATS_GENERATED is USER or TASK, all need to re-calculate the stats:
-      // USER: alter table .. update statistics
-      // TASK: from some sql operation which could collect and compute stats
-      if (StatsSetupConst.TASK.equals(statsType) || StatsSetupConst.USER.equals(statsType)) {
-        return true;
-      }
-    }
-
-    // requires to calculate stats if new and old have different fast stats
-    return !isFastStatsSame(oldPart, newPart);
-  }
-
-  static boolean isFastStatsSame(Partition oldPart, Partition newPart) {
-    // requires to calculate stats if new and old have different fast stats
-    if ((oldPart != null) && (oldPart.getParameters() != null)) {
-      for (String stat : StatsSetupConst.fastStats) {
-        if (oldPart.getParameters().containsKey(stat)) {
-          Long oldStat = Long.parseLong(oldPart.getParameters().get(stat));
-          Long newStat = Long.parseLong(newPart.getParameters().get(stat));
-          if (!oldStat.equals(newStat)) {
-            return false;
-          }
-        } else {
-          return false;
-        }
-      }
-      return true;
-    }
-    return false;
-  }
-
-  public static boolean updatePartitionStatsFast(Partition part, Warehouse wh, EnvironmentContext environmentContext)
+  static boolean updatePartitionStatsFast(Partition part, Warehouse wh, EnvironmentContext environmentContext)
       throws MetaException {
     return updatePartitionStatsFast(part, wh, false, false, environmentContext);
   }
 
-  public static boolean updatePartitionStatsFast(Partition part, Warehouse wh, boolean madeDir, EnvironmentContext environmentContext)
+  static boolean updatePartitionStatsFast(Partition part, Warehouse wh, boolean madeDir, EnvironmentContext environmentContext)
       throws MetaException {
     return updatePartitionStatsFast(part, wh, madeDir, false, environmentContext);
   }
@@ -366,7 +286,7 @@ public class MetaStoreUtils {
    * these parameters set
    * @return true if the stats were updated, false otherwise
    */
-  public static boolean updatePartitionStatsFast(Partition part, Warehouse wh,
+  private static boolean updatePartitionStatsFast(Partition part, Warehouse wh,
       boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
     return updatePartitionStatsFast(new PartitionSpecProxy.SimplePartitionWrapperIterator(part),
                                     wh, madeDir, forceRecompute, environmentContext);
@@ -382,7 +302,7 @@ public class MetaStoreUtils {
    * these parameters set
    * @return true if the stats were updated, false otherwise
    */
-  public static boolean updatePartitionStatsFast(PartitionSpecProxy.PartitionIterator part, Warehouse wh,
+  static boolean updatePartitionStatsFast(PartitionSpecProxy.PartitionIterator part, Warehouse wh,
       boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) throws MetaException {
     Map<String,String> params = part.getParameters();
     boolean updated = false;
@@ -407,7 +327,8 @@ public class MetaStoreUtils {
     return updated;
   }
 
-  static void updateBasicState(EnvironmentContext environmentContext, Map<String,String> params) {
+  private static void updateBasicState(EnvironmentContext environmentContext, Map<String,String>
+      params) {
     if (params == null) {
       return;
     }
@@ -612,7 +533,7 @@ public class MetaStoreUtils {
   /*
    * At the Metadata level there are no restrictions on Column Names.
    */
-  public static final boolean validateColumnName(String name) {
+  public static boolean validateColumnName(String name) {
     return true;
   }
 
@@ -629,53 +550,6 @@ public class MetaStoreUtils {
     return null;
   }
 
-  static void throwExceptionIfIncompatibleColTypeChange(
-      List<FieldSchema> oldCols, List<FieldSchema> newCols)
-      throws InvalidOperationException {
-
-    List<String> incompatibleCols = new ArrayList<String>();
-    int maxCols = Math.min(oldCols.size(), newCols.size());
-    for (int i = 0; i < maxCols; i++) {
-      if (!areColTypesCompatible(oldCols.get(i).getType(), newCols.get(i).getType())) {
-        incompatibleCols.add(newCols.get(i).getName());
-      }
-    }
-    if (!incompatibleCols.isEmpty()) {
-      throw new InvalidOperationException(
-          "The following columns have types incompatible with the existing " +
-          "columns in their respective positions :\n" +
-          StringUtils.join(incompatibleCols, ',')
-        );
-    }
-  }
-
-  static boolean areSameColumns(List<FieldSchema> oldCols, List<FieldSchema> newCols) {
-    return ListUtils.isEqualList(oldCols, newCols);
-  }
-
-  /*
-   * This method is to check if the new column list includes all the old columns with same name and
-   * type. The column comment does not count.
-   */
-  static boolean columnsIncludedByNameType(List<FieldSchema> oldCols, List<FieldSchema> newCols) {
-    if (oldCols.size() > newCols.size()) {
-      return false;
-    }
-
-    Map<String, String> columnNameTypePairMap = new HashMap<String, String>(newCols.size());
-    for (FieldSchema newCol : newCols) {
-      columnNameTypePairMap.put(newCol.getName().toLowerCase(), newCol.getType());
-    }
-    for (final FieldSchema oldCol : oldCols) {
-      if (!columnNameTypePairMap.containsKey(oldCol.getName())
-          || !columnNameTypePairMap.get(oldCol.getName()).equalsIgnoreCase(oldCol.getType())) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
   /**
    * @return true if oldType and newType are compatible.
    * Two types are compatible if we have internal functions to cast one to another.
@@ -821,7 +695,7 @@ public class MetaStoreUtils {
         org.apache.hadoop.hive.serde.serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME);
   }
 
-  static Set<String> hiveThriftTypeMap; //for validation
+  private static Set<String> hiveThriftTypeMap; //for validation
   static {
     hiveThriftTypeMap = new HashSet<String>();
     hiveThriftTypeMap.addAll(serdeConstants.PrimitiveTypes);
@@ -1676,31 +1550,6 @@ public class MetaStoreUtils {
   };
 
   /**
-   * We have aneed to sanity-check the map before conversion from persisted objects to
-   * metadata thrift objects because null values in maps will cause a NPE if we send
-   * across thrift. Pruning is appropriate for most cases except for databases such as
-   * Oracle where Empty strings are stored as nulls, in which case we need to handle that.
-   * See HIVE-8485 for motivations for this.
-   */
-  public static Map<String,String> trimMapNulls(
-      Map<String,String> dnMap, boolean retrieveMapNullsAsEmptyStrings){
-    if (dnMap == null){
-      return null;
-    }
-    // Must be deterministic order map - see HIVE-8707
-    //   => we use Maps.newLinkedHashMap instead of Maps.newHashMap
-    if (retrieveMapNullsAsEmptyStrings) {
-      // convert any nulls present in map values to empty strings - this is done in the case
-      // of backing dbs like oracle which persist empty strings as nulls.
-      return Maps.newLinkedHashMap(Maps.transformValues(dnMap, transFormNullsToEmptyString));
-    } else {
-      // prune any nulls present in map values - this is the typical case.
-      return Maps.newLinkedHashMap(Maps.filterValues(dnMap, Predicates.notNull()));
-    }
-  }
-
-
-  /**
    * Create a URL from a string representing a path to a local file.
    * The path string can be just a path, or can start with file:/, file:///
    * @param onestr  path string
@@ -1780,20 +1629,6 @@ public class MetaStoreUtils {
     csNew.setStatsObj(list);
   }
 
-  /**
-   * convert Exception to MetaException, which sets the cause to such exception
-   * @param errorMessage  the error message for this MetaException
-   * @param e             cause of the exception
-   * @return  the MetaException with the specified exception as the cause
-   */
-  public static MetaException newMetaException(String errorMessage, Exception e) {
-    MetaException metaException = new MetaException(errorMessage);
-    if (e != null) {
-      metaException.initCause(e);
-    }
-    return metaException;
-  }
-
   public static List<String> getColumnNames(List<FieldSchema> schema) {
     List<String> cols = new ArrayList<>(schema.size());
     for (FieldSchema fs : schema) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
deleted file mode 100644
index affb38f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/**
- * 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;
-
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.datanucleus.exceptions.NucleusException;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RetryingHMSHandler implements InvocationHandler {
-
-  private static final Logger LOG = LoggerFactory.getLogger(RetryingHMSHandler.class);
-  private static final String CLASS_NAME = RetryingHMSHandler.class.getName();
-
-  private static class Result {
-    private final Object result;
-    private final int numRetries;
-
-    public Result(Object result, int numRetries) {
-      this.result = result;
-      this.numRetries = numRetries;
-    }
-  }
-
-  private final IHMSHandler baseHandler;
-  private final MetaStoreInit.MetaStoreInitData metaStoreInitData =
-    new MetaStoreInit.MetaStoreInitData();
-
-  private final HiveConf origConf;            // base configuration
-  private final Configuration activeConf;  // active configuration
-
-  private RetryingHMSHandler(HiveConf hiveConf, IHMSHandler baseHandler, boolean local) throws MetaException {
-    this.origConf = hiveConf;
-    this.baseHandler = baseHandler;
-    if (local) {
-      baseHandler.setConf(hiveConf); // tests expect configuration changes applied directly to metastore
-    }
-    activeConf = baseHandler.getConf();
-    // This has to be called before initializing the instance of HMSHandler
-    // Using the hook on startup ensures that the hook always has priority
-    // over settings in *.xml.  The thread local conf needs to be used because at this point
-    // it has already been initialized using hiveConf.
-    MetaStoreInit.updateConnectionURL(hiveConf, getActiveConf(), null, metaStoreInitData);
-    try {
-      //invoking init method of baseHandler this way since it adds the retry logic
-      //in case of transient failures in init method
-      invoke(baseHandler, baseHandler.getClass().getDeclaredMethod("init", (Class<?>[]) null),
-          null);
-    } catch (Throwable e) {
-      LOG.error("HMSHandler Fatal error: " + ExceptionUtils.getStackTrace(e));
-      MetaException me = new MetaException(e.getMessage());
-      me.initCause(e);
-      throw me;
-    }
-  }
-
-  public static IHMSHandler getProxy(HiveConf hiveConf, IHMSHandler baseHandler, boolean local)
-      throws MetaException {
-
-    RetryingHMSHandler handler = new RetryingHMSHandler(hiveConf, baseHandler, local);
-
-    return (IHMSHandler) Proxy.newProxyInstance(
-      RetryingHMSHandler.class.getClassLoader(),
-      new Class[] { IHMSHandler.class }, handler);
-  }
-
-  @Override
-  public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
-    int retryCount = -1;
-    int threadId = HiveMetaStore.HMSHandler.get();
-    boolean error = true;
-    PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
-    perfLogger.PerfLogBegin(CLASS_NAME, method.getName());
-    try {
-      Result result = invokeInternal(proxy, method, args);
-      retryCount = result.numRetries;
-      error = false;
-      return result.result;
-    } finally {
-      StringBuilder additionalInfo = new StringBuilder();
-      additionalInfo.append("threadId=").append(threadId).append(" retryCount=").append(retryCount)
-        .append(" error=").append(error);
-      perfLogger.PerfLogEnd(CLASS_NAME, method.getName(), additionalInfo.toString());
-    }
-  }
-
-  public Result invokeInternal(final Object proxy, final Method method, final Object[] args) throws Throwable {
-
-    boolean gotNewConnectUrl = false;
-    boolean reloadConf = HiveConf.getBoolVar(origConf,
-        HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF);
-    long retryInterval = HiveConf.getTimeVar(origConf,
-        HiveConf.ConfVars.HMSHANDLERINTERVAL, TimeUnit.MILLISECONDS);
-    int retryLimit = HiveConf.getIntVar(origConf,
-        HiveConf.ConfVars.HMSHANDLERATTEMPTS);
-    long timeout = HiveConf.getTimeVar(origConf,
-        HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS);
-
-    Deadline.registerIfNot(timeout);
-
-    if (reloadConf) {
-      MetaStoreInit.updateConnectionURL(origConf, getActiveConf(),
-        null, metaStoreInitData);
-    }
-
-    int retryCount = 0;
-    Throwable caughtException = null;
-    while (true) {
-      try {
-        if (reloadConf || gotNewConnectUrl) {
-          baseHandler.setConf(getActiveConf());
-        }
-        Object object = null;
-        boolean isStarted = Deadline.startTimer(method.getName());
-        try {
-          object = method.invoke(baseHandler, args);
-        } finally {
-          if (isStarted) {
-            Deadline.stopTimer();
-          }
-        }
-        return new Result(object, retryCount);
-
-      } catch (UndeclaredThrowableException e) {
-        if (e.getCause() != null) {
-          if (e.getCause() instanceof javax.jdo.JDOException) {
-            // Due to reflection, the jdo exception is wrapped in
-            // invocationTargetException
-            caughtException = e.getCause();
-          } else if (e.getCause() instanceof MetaException && e.getCause().getCause() != null
-              && e.getCause().getCause() instanceof javax.jdo.JDOException) {
-            // The JDOException may be wrapped further in a MetaException
-            caughtException = e.getCause().getCause();
-          } else {
-            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
-            throw e.getCause();
-          }
-        } else {
-          LOG.error(ExceptionUtils.getStackTrace(e));
-          throw e;
-        }
-      } catch (InvocationTargetException e) {
-        if (e.getCause() instanceof javax.jdo.JDOException) {
-          // Due to reflection, the jdo exception is wrapped in
-          // invocationTargetException
-          caughtException = e.getCause();
-        } else if (e.getCause() instanceof NoSuchObjectException || e.getTargetException().getCause() instanceof NoSuchObjectException) {
-          String methodName = method.getName();
-          if (!methodName.startsWith("get_database") && !methodName.startsWith("get_table")
-              && !methodName.startsWith("get_partition") && !methodName.startsWith("get_function")) {
-            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
-          }
-          throw e.getCause();
-        } else if (e.getCause() instanceof MetaException && e.getCause().getCause() != null) {
-          if (e.getCause().getCause() instanceof javax.jdo.JDOException ||
-              e.getCause().getCause() instanceof NucleusException) {
-            // The JDOException or the Nucleus Exception may be wrapped further in a MetaException
-            caughtException = e.getCause().getCause();
-          } else if (e.getCause().getCause() instanceof DeadlineException) {
-            // The Deadline Exception needs no retry and be thrown immediately.
-            Deadline.clear();
-            LOG.error("Error happens in method " + method.getName() + ": " +
-                ExceptionUtils.getStackTrace(e.getCause()));
-            throw e.getCause();
-          } else {
-            LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
-            throw e.getCause();
-          }
-        } else {
-          LOG.error(ExceptionUtils.getStackTrace(e.getCause()));
-          throw e.getCause();
-        }
-      }
-
-      if (retryCount >= retryLimit) {
-        LOG.error("HMSHandler Fatal error: " + ExceptionUtils.getStackTrace(caughtException));
-        MetaException me = new MetaException(caughtException.getMessage());
-        me.initCause(caughtException);
-        throw me;
-      }
-
-      assert (retryInterval >= 0);
-      retryCount++;
-      LOG.error(
-        String.format(
-          "Retrying HMSHandler after %d ms (attempt %d of %d)", retryInterval, retryCount, retryLimit) +
-          " with error: " + ExceptionUtils.getStackTrace(caughtException));
-
-      Thread.sleep(retryInterval);
-      // If we have a connection error, the JDO connection URL hook might
-      // provide us with a new URL to access the datastore.
-      String lastUrl = MetaStoreInit.getConnectionURL(getActiveConf());
-      gotNewConnectUrl = MetaStoreInit.updateConnectionURL(origConf, getActiveConf(),
-        lastUrl, metaStoreInitData);
-    }
-  }
-
-  public Configuration getActiveConf() {
-    return activeConf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java
deleted file mode 100644
index ee96678..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/SessionPropertiesListener.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * It handles the changed properties in the change event.
- */
-public class SessionPropertiesListener extends MetaStoreEventListener {
-
-  public SessionPropertiesListener(Configuration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public void onConfigChange(ConfigChangeEvent changeEvent) throws MetaException {
-    if (changeEvent.getKey().equals(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT.varname)) {
-      // TODO: this only applies to current thread, so it's not useful at all.
-      Deadline.resetTimeout(HiveConf.toTime(changeEvent.getNewValue(), TimeUnit.SECONDS,
-          TimeUnit.MILLISECONDS));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java
deleted file mode 100644
index d7f1c64..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalMetaStoreEventListener.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Special type of MetaStoreEventListener which should only be called in a transactional context
- * and only if the transaction is successful.
- * The events are expected to have a success status.
- */
-public abstract class TransactionalMetaStoreEventListener extends MetaStoreEventListener {
-
-  /**
-   * Constructor
-   *
-   * @param config
-   */
-  public TransactionalMetaStoreEventListener(Configuration config) {
-    super(config);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
deleted file mode 100644
index 49c8cbb..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
+++ /dev/null
@@ -1,359 +0,0 @@
-/**
- * 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;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
-import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.PreEventContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public final class TransactionalValidationListener extends MetaStorePreEventListener {
-  public static final Logger LOG = LoggerFactory.getLogger(TransactionalValidationListener.class);
-
-  // These constants are also imported by org.apache.hadoop.hive.ql.io.AcidUtils.
-  public static final String DEFAULT_TRANSACTIONAL_PROPERTY = "default";
-  public static final String INSERTONLY_TRANSACTIONAL_PROPERTY = "insert_only";
-
-  TransactionalValidationListener(Configuration conf) {
-    super(conf);
-  }
-
-  @Override
-  public void onEvent(PreEventContext context) throws MetaException, NoSuchObjectException,
-      InvalidOperationException {
-    switch (context.getEventType()) {
-      case CREATE_TABLE:
-        handle((PreCreateTableEvent) context);
-        break;
-      case ALTER_TABLE:
-        handle((PreAlterTableEvent) context);
-        break;
-      default:
-        //no validation required..
-    }
-  }
-
-  private void handle(PreAlterTableEvent context) throws MetaException {
-    handleAlterTableTransactionalProp(context);
-  }
-
-  private void handle(PreCreateTableEvent context) throws MetaException {
-    handleCreateTableTransactionalProp(context);
-  }
-
-  /**
-   * once a table is marked transactional, you cannot go back.  Enforce this.
-   * Also in current version, 'transactional_properties' of the table cannot be altered after
-   * the table is created. Any attempt to alter it will throw a MetaException.
-   */
-  private void handleAlterTableTransactionalProp(PreAlterTableEvent context) throws MetaException {
-    Table newTable = context.getNewTable();
-    Map<String, String> parameters = newTable.getParameters();
-    if (parameters == null || parameters.isEmpty()) {
-      return;
-    }
-    Set<String> keys = new HashSet<>(parameters.keySet());
-    String transactionalValue = null;
-    boolean transactionalValuePresent = false;
-    boolean isTransactionalPropertiesPresent = false;
-    String transactionalPropertiesValue = null;
-    boolean hasValidTransactionalValue = false;
-
-    for (String key : keys) {
-      if(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) {
-        transactionalValuePresent = true;
-        transactionalValue = parameters.get(key);
-        parameters.remove(key);
-      }
-      if(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
-        isTransactionalPropertiesPresent = true;
-        transactionalPropertiesValue = parameters.get(key);
-        // Do not remove the parameter yet, because we have separate initialization routine
-        // that will use it down below.
-      }
-    }
-    Table oldTable = context.getOldTable();
-    String oldTransactionalValue = null;
-    String oldTransactionalPropertiesValue = null;
-    for (String key : oldTable.getParameters().keySet()) {
-      if (hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) {
-        oldTransactionalValue = oldTable.getParameters().get(key);
-      }
-      if (hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
-        oldTransactionalPropertiesValue = oldTable.getParameters().get(key);
-      }
-    }
-
-    if (transactionalValuePresent) {
-      //normalize prop name
-      parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, transactionalValue);
-    }
-    if ("true".equalsIgnoreCase(transactionalValue) && !"true".equalsIgnoreCase(oldTransactionalValue)) {
-      //only need to check conformance if alter table enabled aicd
-      if (!conformToAcid(newTable)) {
-        // INSERT_ONLY tables don't have to conform to ACID requirement like ORC or bucketing
-        if (transactionalPropertiesValue == null || !"insert_only".equalsIgnoreCase(transactionalPropertiesValue)) {
-          throw new MetaException("The table must be stored using an ACID compliant format (such as ORC)");
-        }
-      }
-
-      if (newTable.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
-        throw new MetaException(getTableName(newTable) +
-            " cannot be declared transactional because it's an external table");
-      }
-      validateTableStructure(context.getHandler(), newTable);
-      hasValidTransactionalValue = true;
-    }
-
-
-
-    if (oldTransactionalValue == null ? transactionalValue == null
-                                     : oldTransactionalValue.equalsIgnoreCase(transactionalValue)) {
-      //this covers backward compat cases where this prop may have been set already
-      hasValidTransactionalValue = true;
-    }
-
-    if (!hasValidTransactionalValue && !MetaStoreUtils.isInsertOnlyTableParam(oldTable.getParameters())) {
-      // if here, there is attempt to set transactional to something other than 'true'
-      // and NOT the same value it was before
-      throw new MetaException("TBLPROPERTIES with 'transactional'='true' cannot be unset");
-    }
-
-    if (isTransactionalPropertiesPresent) {
-      // Now validate transactional_properties for the table.
-      if (oldTransactionalValue == null) {
-        // If this is the first time the table is being initialized to 'transactional=true',
-        // any valid value can be set for the 'transactional_properties'.
-        initializeTransactionalProperties(newTable);
-      } else {
-        // If the table was already marked as 'transactional=true', then the new value of
-        // 'transactional_properties' must match the old value. Any attempt to alter the previous
-        // value will throw an error. An exception will still be thrown if the previous value was
-        // null and an attempt is made to set it. This behaviour can be changed in the future.
-        if ((oldTransactionalPropertiesValue == null
-            || !oldTransactionalPropertiesValue.equalsIgnoreCase(transactionalPropertiesValue))
-            && !MetaStoreUtils.isInsertOnlyTableParam(oldTable.getParameters())) {
-          throw new MetaException("TBLPROPERTIES with 'transactional_properties' cannot be "
-              + "altered after the table is created");
-        }
-      }
-    }
-  }
-
-  /**
-   * Normalize case and make sure:
-   * 1. 'true' is the only value to be set for 'transactional' (if set at all)
-   * 2. If set to 'true', we should also enforce bucketing and ORC format
-   */
-  private void handleCreateTableTransactionalProp(PreCreateTableEvent context) throws MetaException {
-    Table newTable = context.getTable();
-    Map<String, String> parameters = newTable.getParameters();
-    if (parameters == null || parameters.isEmpty()) {
-      return;
-    }
-    String transactional = null;
-    String transactionalProperties = null;
-    Set<String> keys = new HashSet<>(parameters.keySet());
-    for(String key : keys) {
-      // Get the "transactional" tblproperties value
-      if (hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) {
-        transactional = parameters.get(key);
-        parameters.remove(key);
-      }
-
-      // Get the "transactional_properties" tblproperties value
-      if (hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
-        transactionalProperties = parameters.get(key);
-      }
-    }
-
-    if (transactional == null) {
-      return;
-    }
-
-    if ("false".equalsIgnoreCase(transactional)) {
-      // just drop transactional=false.  For backward compatibility in case someone has scripts
-      // with transactional=false
-      LOG.info("'transactional'='false' is no longer a valid property and will be ignored");
-      return;
-    }
-
-    if ("true".equalsIgnoreCase(transactional)) {
-      if (!conformToAcid(newTable)) {
-        // INSERT_ONLY tables don't have to conform to ACID requirement like ORC or bucketing
-        if (transactionalProperties == null || !"insert_only".equalsIgnoreCase(transactionalProperties)) {
-          throw new MetaException("The table must be stored using an ACID compliant format (such as ORC)");
-        }
-      }
-
-      if (newTable.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
-        throw new MetaException(newTable.getDbName() + "." + newTable.getTableName() +
-            " cannot be declared transactional because it's an external table");
-      }
-
-      // normalize prop name
-      parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, Boolean.TRUE.toString());
-      initializeTransactionalProperties(newTable);
-      return;
-    }
-
-    // transactional is found, but the value is not in expected range
-    throw new MetaException("'transactional' property of TBLPROPERTIES may only have value 'true'");
-  }
-
-  /**
-   * Check that InputFormatClass/OutputFormatClass should implement
-   * AcidInputFormat/AcidOutputFormat
-   */
-  private boolean conformToAcid(Table table) throws MetaException {
-    StorageDescriptor sd = table.getSd();
-    try {
-      Class inputFormatClass = Class.forName(sd.getInputFormat());
-      Class outputFormatClass = Class.forName(sd.getOutputFormat());
-
-      if (inputFormatClass == null || outputFormatClass == null ||
-          !Class.forName("org.apache.hadoop.hive.ql.io.AcidInputFormat").isAssignableFrom(inputFormatClass) ||
-          !Class.forName("org.apache.hadoop.hive.ql.io.AcidOutputFormat").isAssignableFrom(outputFormatClass)) {
-        return false;
-      }
-    } catch (ClassNotFoundException e) {
-      throw new MetaException("Invalid input/output format for table");
-    }
-
-    return true;
-  }
-
-  private void initializeTransactionalProperties(Table table) throws MetaException {
-    // All new versions of Acid tables created after the introduction of Acid version/type system
-    // can have TRANSACTIONAL_PROPERTIES property defined. This parameter can be used to change
-    // the operational behavior of ACID. However if this parameter is not defined, the new Acid
-    // tables will still behave as the old ones. This is done so to preserve the behavior
-    // in case of rolling downgrade.
-
-    // Initialize transaction table properties with default string value.
-    String tableTransactionalProperties = null;
-
-    Map<String, String> parameters = table.getParameters();
-    if (parameters != null) {
-      Set<String> keys = parameters.keySet();
-      for (String key : keys) {
-        if (hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES.equalsIgnoreCase(key)) {
-          tableTransactionalProperties = parameters.get(key).toLowerCase();
-          parameters.remove(key);
-          String validationError = validateTransactionalProperties(tableTransactionalProperties);
-          if (validationError != null) {
-            throw new MetaException("Invalid transactional properties specified for the "
-                + "table with the error " + validationError);
-          }
-          break;
-        }
-      }
-    }
-
-    if (tableTransactionalProperties != null) {
-      parameters.put(hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES,
-              tableTransactionalProperties);
-    }
-  }
-
-  private String validateTransactionalProperties(String transactionalProperties) {
-    boolean isValid = false;
-    switch (transactionalProperties) {
-      case DEFAULT_TRANSACTIONAL_PROPERTY:
-      case INSERTONLY_TRANSACTIONAL_PROPERTY:
-        isValid = true;
-        break;
-      default:
-        isValid = false;
-    }
-    if (!isValid) {
-      return "unknown value " + transactionalProperties +  " for transactional_properties";
-    }
-    return null; // All checks passed, return null.
-  }
-  private final Pattern ORIGINAL_PATTERN = Pattern.compile("[0-9]+_[0-9]+");
-  /**
-   * @see org.apache.hadoop.hive.ql.exec.Utilities#COPY_KEYWORD
-   */
-  private static final Pattern ORIGINAL_PATTERN_COPY =
-    Pattern.compile("[0-9]+_[0-9]+" + "_copy_" + "[0-9]+");
-
-  /**
-   * It's assumed everywhere that original data files are named according to
-   * {@link #ORIGINAL_PATTERN} or{@link #ORIGINAL_PATTERN_COPY}
-   * This checks that when transaction=true is set and throws if it finds any files that don't
-   * follow convention.
-   */
-  private void validateTableStructure(HiveMetaStore.HMSHandler hmsHandler, Table table)
-    throws MetaException {
-    Path tablePath;
-    try {
-      Warehouse wh = hmsHandler.getWh();
-      if (table.getSd().getLocation() == null || table.getSd().getLocation().isEmpty()) {
-        tablePath = wh.getDefaultTablePath(hmsHandler.getMS().getDatabase(table.getDbName()),
-          table.getTableName());
-      } else {
-        tablePath = wh.getDnsPath(new Path(table.getSd().getLocation()));
-      }
-      FileSystem fs = wh.getFs(tablePath);
-      //FileSystem fs = FileSystem.get(getConf());
-      RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(tablePath, true);
-      while (iterator.hasNext()) {
-        LocatedFileStatus fileStatus = iterator.next();
-        if (!fileStatus.isFile()) {
-          continue;
-        }
-        boolean validFile =
-          (ORIGINAL_PATTERN.matcher(fileStatus.getPath().getName()).matches() ||
-            ORIGINAL_PATTERN_COPY.matcher(fileStatus.getPath().getName()).matches()
-          );
-        if (!validFile) {
-          throw new IllegalStateException("Unexpected data file name format.  Cannot convert " +
-            getTableName(table) + " to transactional table.  File: " + fileStatus.getPath());
-        }
-      }
-    } catch (IOException|NoSuchObjectException e) {
-      String msg = "Unable to list files for " + getTableName(table);
-      LOG.error(msg, e);
-      MetaException e1 = new MetaException(msg);
-      e1.initCause(e);
-      throw e1;
-    }
-  }
-  private static String getTableName(Table table) {
-    return table.getDbName() + "." + table.getTableName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java
deleted file mode 100644
index 1dc9588..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddForeignKeyEvent.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * 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 java.util.List;
-
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-
-public class AddForeignKeyEvent extends ListenerEvent {
-  private final List<SQLForeignKey> fks;
-
-  public AddForeignKeyEvent(List<SQLForeignKey> fks, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.fks = fks;
-  }
-
-  public List<SQLForeignKey> getForeignKeyCols() {
-    return fks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
deleted file mode 100644
index 57fe5d1..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddIndexEvent.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.api.Index;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java
deleted file mode 100644
index c01083c..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddNotNullConstraintEvent.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * 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 java.util.List;
-
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
-
-public class AddNotNullConstraintEvent extends ListenerEvent {
-  private final List<SQLNotNullConstraint> nns;
-
-  public AddNotNullConstraintEvent(List<SQLNotNullConstraint> nns, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.nns = nns;
-  }
-
-  public List<SQLNotNullConstraint> getNotNullConstraintCols() {
-    return nns;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
deleted file mode 100644
index 85f5fb8..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-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 org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class AddPartitionEvent extends ListenerEvent {
-
-  private final Table table;
-  private final List<Partition> partitions;
-  private PartitionSpecProxy partitionSpecProxy;
-
-  public AddPartitionEvent(Table table, List<Partition> partitions, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.table = table;
-    this.partitions = partitions;
-    this.partitionSpecProxy = null;
-  }
-
-  public AddPartitionEvent(Table table, Partition partition, boolean status, HMSHandler handler) {
-    this(table, Arrays.asList(partition), status, handler);
-  }
-
-  /**
-   * Alternative constructor to use PartitionSpec APIs.
-   */
-  public AddPartitionEvent(Table table, PartitionSpecProxy partitionSpec, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.table = table;
-    this.partitions = null;
-    this.partitionSpecProxy = partitionSpec;
-  }
-
-  /**
-   * @return The table.
-   */
-  public Table getTable() {
-    return table;
-  }
-
-
-  // Note : List<Partition> getPartitions() removed with HIVE-9609 because it will result in OOM errors with large add_partitions.
-
-  /**
-   * @return Iterator for partitions.
-   */
-  public Iterator<Partition> getPartitionIterator() {
-    if (partitions != null){
-      return partitions.iterator();
-    } else {
-      return partitionSpecProxy == null ? null : partitionSpecProxy.getPartitionIterator();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java
deleted file mode 100644
index cb0f562..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPrimaryKeyEvent.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * 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 java.util.List;
-
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-
-public class AddPrimaryKeyEvent extends ListenerEvent {
-
-  private final List<SQLPrimaryKey> pks;
-
-  public AddPrimaryKeyEvent(List<SQLPrimaryKey> pks, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.pks = pks;
-  }
-
-  public List<SQLPrimaryKey> getPrimaryKeyCols() {
-    return pks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java
deleted file mode 100644
index c02a309..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddUniqueConstraintEvent.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * 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 java.util.List;
-
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
-
-public class AddUniqueConstraintEvent extends ListenerEvent {
-  private final List<SQLUniqueConstraint> uks;
-
-  public AddUniqueConstraintEvent(List<SQLUniqueConstraint> uks, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.uks = uks;
-  }
-
-  public List<SQLUniqueConstraint> getUniqueConstraintCols() {
-    return uks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
deleted file mode 100644
index 99e51f6..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterIndexEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.api.Index;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
deleted file mode 100644
index 4025f9f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class AlterPartitionEvent extends ListenerEvent {
-
-  private final Partition oldPart;
-  private final Partition newPart;
-  private final Table table;
-  private final boolean isTruncateOp;
-
-  public AlterPartitionEvent(Partition oldPart, Partition newPart, Table table, boolean isTruncateOp,
-                             boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.oldPart = oldPart;
-    this.newPart = newPart;
-    this.table = table;
-    this.isTruncateOp = isTruncateOp;
-  }
-
-  /**
-   * @return the old partition
-   */
-  public Partition getOldPartition() {
-    return oldPart;
-  }
-
-  /**
-   *
-   * @return the new partition
-   */
-  public Partition getNewPartition() {
-    return newPart;
-  }
-
-  /**
-   * Get the table this partition is in
-   * @return
-   */
-  public Table getTable() {
-    return table;
-  }
-
-  /**
-   * Get the truncate table flag
-   * @return
-   */
-  public boolean getIsTruncateOp() {
-    return isTruncateOp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
deleted file mode 100644
index eb9d94f..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class AlterTableEvent extends ListenerEvent {
-
-  private final Table newTable;
-  private final Table oldTable;
-  private final boolean isTruncateOp;
-
-  public AlterTableEvent (Table oldTable, Table newTable, boolean isTruncateOp, boolean status, HMSHandler handler) {
-    super (status, handler);
-    this.oldTable = oldTable;
-    this.newTable = newTable;
-    this.isTruncateOp = isTruncateOp;
-  }
-
-  /**
-   * @return the old table
-   */
-  public Table getOldTable() {
-    return oldTable;
-  }
-
-  /**
-   * @return the new table
-   */
-  public Table getNewTable() {
-    return newTable;
-  }
-
-  /**
-   * @return the flag for truncate
-   */
-  public boolean getIsTruncateOp() {
-    return isTruncateOp;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java
deleted file mode 100644
index 9d23dc8..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/ConfigChangeEvent.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class ConfigChangeEvent extends ListenerEvent {
-
-  private final String key;
-  private final String oldValue;
-  private final String newValue;
-
-  public ConfigChangeEvent(HiveMetaStore.HMSHandler handler, String key,
-      String oldValue, String newValue) {
-    super(true, handler);
-    this.key = key;
-    this.oldValue = oldValue;
-    this.newValue = newValue;
-  }
-
-  public String getKey() {
-    return key;
-  }
-
-  public String getOldValue() {
-    return oldValue;
-  }
-
-  public String getNewValue() {
-    return newValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
deleted file mode 100644
index e853aec..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Database;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CreateDatabaseEvent extends ListenerEvent {
-
-  private final Database db;
-
-  public CreateDatabaseEvent (Database db, boolean status, HMSHandler handler) {
-    super (status, handler);
-    this.db = db;
-  }
-
-  /**
-   * @return the db
-   */
-  public Database getDatabase () {
-    return db;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
deleted file mode 100644
index 0c6ab17..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Function;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CreateFunctionEvent extends ListenerEvent {
-
-  private final Function function;
-
-  public CreateFunctionEvent (Function function, boolean status, HMSHandler handler) {
-    super (status, handler);
-    this.function = function;
-  }
-
-  /**
-   * @return the function
-   */
-  public Function getFunction () {
-    return function;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
deleted file mode 100644
index 09a40bb..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CreateTableEvent extends ListenerEvent {
-
-  private final Table table;
-
-  public CreateTableEvent (Table table, boolean status, HMSHandler handler) {
-    super (status, handler);
-    this.table = table;
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable () {
-    return table;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
deleted file mode 100644
index 5396863..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.HMSHandler;
-
-public class DropConstraintEvent extends ListenerEvent {
-
-  private final String dbName;
-  private final String tableName;
-  private final String constraintName;
-  public DropConstraintEvent(String dbName, String tableName, String constraintName,
-      boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.dbName = dbName;
-    this.tableName = tableName;
-    this.constraintName = constraintName;
-  }
-
-  public String getDbName() {
-    return dbName;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public String getConstraintName() {
-    return constraintName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
deleted file mode 100644
index 92e95a6..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Database;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class DropDatabaseEvent extends ListenerEvent {
-
-  private final Database db;
-
-  public DropDatabaseEvent(Database db, boolean status, HMSHandler handler) {
-    super (status, handler);
-    this.db = db;
-  }
-
-  /**
-   * @return the db
-   */
-  public Database getDatabase() {
-    return db;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
deleted file mode 100644
index 52f04ca..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Function;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class DropFunctionEvent extends ListenerEvent {
-
-  private final Function function;
-
-  public DropFunctionEvent(Function function, boolean status, HMSHandler handler) {
-    super(status, handler);
-    this.function = function;
-  }
-
-  /**
-   * @return the function
-   */
-  public Function getFunction() {
-    return function;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
deleted file mode 100644
index 9b75055..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropIndexEvent.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.api.Index;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
deleted file mode 100644
index d8f2ea9..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-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.Collections;
-import java.util.Iterator;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class DropPartitionEvent extends ListenerEvent {
-
-  private final Table table;
-  private final Iterable<Partition> partitions;
-  private final boolean deleteData;
-
-  public DropPartitionEvent (Table table,
-      Partition partition, boolean status, boolean deleteData, HMSHandler handler) {
-    super (status, handler);
-    this.table = table;
-    this.partitions = Collections.singletonList(partition);
-    // In HiveMetaStore, the deleteData flag indicates whether DFS data should be
-    // removed on a drop.
-    this.deleteData = deleteData;
-  }
-
-  /**
-   * @return the partitions
-   */
-  public Iterator<Partition> getPartitionIterator() {
-    return partitions.iterator();
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable() {
-
-    return table;
-  }
-
-  /**
-   * @return the deleteData flag
-   */
-  public boolean getDeleteData() {
-
-    return deleteData;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
deleted file mode 100644
index 2f2ea9d..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * 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.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.Table;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class DropTableEvent extends ListenerEvent {
-
-  private final Table table;
-  private final boolean deleteData;
-
-  public DropTableEvent(Table table, boolean status, boolean deleteData, HMSHandler handler) {
-    super(status, handler);
-    this.table = table;
-    // In HiveMetaStore, the deleteData flag indicates whether DFS data should be
-    // removed on a drop.
-    this.deleteData = deleteData;
-  }
-
-  /**
-   * @return the table
-   */
-  public Table getTable() {
-    return table;
-  }
-
-  /**
-   * @return the deleteData flag
-   */
-  public boolean getDeleteData() {
-    return deleteData;
-  }
-}


[11/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
deleted file mode 100644
index 70d33ff..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ /dev/null
@@ -1,900 +0,0 @@
-/**
- * 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;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
-import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.DropTableEvent;
-import org.apache.hadoop.hive.metastore.messaging.EventMessage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.conf.HiveConf;
-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;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.InvalidInputException;
-import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hive.common.util.HiveStringUtils;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-/**
- * Hive specific implementation of alter
- */
-public class HiveAlterHandler implements AlterHandler {
-
-  protected Configuration hiveConf;
-  private static final Logger LOG = LoggerFactory.getLogger(HiveAlterHandler.class
-      .getName());
-
-  @Override
-  public Configuration getConf() {
-    return hiveConf;
-  }
-
-  @Override
-  @SuppressWarnings("nls")
-  public void setConf(Configuration conf) {
-    hiveConf = conf;
-  }
-
-  @Override
-  public void alterTable(RawStore msdb, Warehouse wh, String dbname,
-    String name, Table newt, EnvironmentContext environmentContext)
-      throws InvalidOperationException, MetaException {
-    alterTable(msdb, wh, dbname, name, newt, environmentContext, null);
-  }
-
-  @Override
-  public void alterTable(RawStore msdb, Warehouse wh, String dbname,
-      String name, Table newt, EnvironmentContext environmentContext,
-      HMSHandler handler) throws InvalidOperationException, MetaException {
-    name = name.toLowerCase();
-    dbname = dbname.toLowerCase();
-
-    final boolean cascade = environmentContext != null
-        && environmentContext.isSetProperties()
-        && StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(
-            StatsSetupConst.CASCADE));
-    if (newt == null) {
-      throw new InvalidOperationException("New table is invalid: " + newt);
-    }
-
-    String newTblName = newt.getTableName().toLowerCase();
-    String newDbName = newt.getDbName().toLowerCase();
-
-    if (!MetaStoreUtils.validateName(newTblName, hiveConf)) {
-      throw new InvalidOperationException(newTblName + " is not a valid object name");
-    }
-    String validate = MetaStoreUtils.validateTblColumns(newt.getSd().getCols());
-    if (validate != null) {
-      throw new InvalidOperationException("Invalid column " + validate);
-    }
-
-    Path srcPath = null;
-    FileSystem srcFs = null;
-    Path destPath = null;
-    FileSystem destFs = null;
-
-    boolean success = false;
-    boolean dataWasMoved = false;
-    Table oldt = null;
-    List<TransactionalMetaStoreEventListener> transactionalListeners = null;
-    if (handler != null) {
-      transactionalListeners = handler.getTransactionalListeners();
-    }
-
-    try {
-      boolean rename = false;
-      boolean isPartitionedTable = false;
-      List<Partition> parts = null;
-
-      // check if table with the new name already exists
-      if (!newTblName.equals(name) || !newDbName.equals(dbname)) {
-        if (msdb.getTable(newDbName, newTblName) != null) {
-          throw new InvalidOperationException("new table " + newDbName
-              + "." + newTblName + " already exists");
-        }
-        rename = true;
-      }
-
-      msdb.openTransaction();
-      // get old table
-      oldt = msdb.getTable(dbname, name);
-      if (oldt == null) {
-        throw new InvalidOperationException("table " + dbname + "." + name + " doesn't exist");
-      }
-
-      if (oldt.getPartitionKeysSize() != 0) {
-        isPartitionedTable = true;
-      }
-
-      // Views derive the column type from the base table definition.  So the view definition
-      // can be altered to change the column types.  The column type compatibility checks should
-      // be done only for non-views.
-      if (HiveConf.getBoolVar(hiveConf,
-            HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES,
-            false) &&
-          !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())) {
-        // Throws InvalidOperationException if the new column types are not
-        // compatible with the current column types.
-        checkColTypeChangeCompatible(oldt.getSd().getCols(), newt.getSd().getCols());
-      }
-
-      //check that partition keys have not changed, except for virtual views
-      //however, allow the partition comments to change
-      boolean partKeysPartiallyEqual = checkPartialPartKeysEqual(oldt.getPartitionKeys(),
-          newt.getPartitionKeys());
-
-      if(!oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())){
-        if (!partKeysPartiallyEqual) {
-          throw new InvalidOperationException("partition keys can not be changed.");
-        }
-      }
-
-      // rename needs change the data location and move the data to the new location corresponding
-      // to the new name if:
-      // 1) the table is not a virtual view, and
-      // 2) the table is not an external table, and
-      // 3) the user didn't change the default location (or new location is empty), and
-      // 4) the table was not initially created with a specified location
-      if (rename
-          && !oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())
-          && (oldt.getSd().getLocation().compareTo(newt.getSd().getLocation()) == 0
-            || StringUtils.isEmpty(newt.getSd().getLocation()))
-          && !MetaStoreUtils.isExternalTable(oldt)) {
-        Database olddb = msdb.getDatabase(dbname);
-        // if a table was created in a user specified location using the DDL like
-        // create table tbl ... location ...., it should be treated like an external table
-        // in the table rename, its data location should not be changed. We can check
-        // if the table directory was created directly under its database directory to tell
-        // if it is such a table
-        srcPath = new Path(oldt.getSd().getLocation());
-        String oldtRelativePath = (new Path(olddb.getLocationUri()).toUri())
-            .relativize(srcPath.toUri()).toString();
-        boolean tableInSpecifiedLoc = !oldtRelativePath.equalsIgnoreCase(name)
-            && !oldtRelativePath.equalsIgnoreCase(name + Path.SEPARATOR);
-
-        if (!tableInSpecifiedLoc) {
-          srcFs = wh.getFs(srcPath);
-
-          // get new location
-          Database db = msdb.getDatabase(newDbName);
-          Path databasePath = constructRenamedPath(wh.getDatabasePath(db), srcPath);
-          destPath = new Path(databasePath, newTblName);
-          destFs = wh.getFs(destPath);
-
-          newt.getSd().setLocation(destPath.toString());
-
-          // check that destination does not exist otherwise we will be
-          // overwriting data
-          // check that src and dest are on the same file system
-          if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
-            throw new InvalidOperationException("table new location " + destPath
-                + " is on a different file system than the old location "
-                + srcPath + ". This operation is not supported");
-          }
-
-          try {
-            if (destFs.exists(destPath)) {
-              throw new InvalidOperationException("New location for this table "
-                  + newDbName + "." + newTblName + " already exists : " + destPath);
-            }
-            // check that src exists and also checks permissions necessary, rename src to dest
-            if (srcFs.exists(srcPath) && wh.renameDir(srcPath, destPath, true)) {
-              dataWasMoved = true;
-            }
-          } catch (IOException | MetaException e) {
-            LOG.error("Alter Table operation for " + dbname + "." + name + " failed.", e);
-            throw new InvalidOperationException("Alter Table operation for " + dbname + "." + name +
-                " failed to move data due to: '" + getSimpleMessage(e)
-                + "' See hive log file for details.");
-          }
-        }
-
-        if (isPartitionedTable) {
-          String oldTblLocPath = srcPath.toUri().getPath();
-          String newTblLocPath = dataWasMoved ? destPath.toUri().getPath() : null;
-
-          // also the location field in partition
-          parts = msdb.getPartitions(dbname, name, -1);
-          Map<Partition, ColumnStatistics> columnStatsNeedUpdated = new HashMap<Partition, ColumnStatistics>();
-          for (Partition part : parts) {
-            String oldPartLoc = part.getSd().getLocation();
-            if (dataWasMoved && oldPartLoc.contains(oldTblLocPath)) {
-              URI oldUri = new Path(oldPartLoc).toUri();
-              String newPath = oldUri.getPath().replace(oldTblLocPath, newTblLocPath);
-              Path newPartLocPath = new Path(oldUri.getScheme(), oldUri.getAuthority(), newPath);
-              part.getSd().setLocation(newPartLocPath.toString());
-            }
-            part.setDbName(newDbName);
-            part.setTableName(newTblName);
-            ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
-                part.getValues(), part.getSd().getCols(), oldt, part);
-            if (colStats != null) {
-              columnStatsNeedUpdated.put(part, colStats);
-            }
-          }
-          msdb.alterTable(dbname, name, newt);
-          // alterPartition is only for changing the partition location in the table rename
-          if (dataWasMoved) {
-            for (Partition part : parts) {
-              msdb.alterPartition(newDbName, newTblName, part.getValues(), part);
-            }
-          }
-
-          for (Entry<Partition, ColumnStatistics> partColStats : columnStatsNeedUpdated.entrySet()) {
-            ColumnStatistics newPartColStats = partColStats.getValue();
-            newPartColStats.getStatsDesc().setDbName(newDbName);
-            newPartColStats.getStatsDesc().setTableName(newTblName);
-            msdb.updatePartitionColumnStatistics(newPartColStats, partColStats.getKey().getValues());
-          }
-        } else {
-          alterTableUpdateTableColumnStats(msdb, oldt, newt);
-        }
-      } else {
-        // operations other than table rename
-        if (MetaStoreUtils.requireCalStats(hiveConf, null, null, newt, environmentContext) &&
-            !isPartitionedTable) {
-          Database db = msdb.getDatabase(newDbName);
-          // Update table stats. For partitioned table, we update stats in alterPartition()
-          MetaStoreUtils.updateTableStatsFast(db, newt, wh, false, true, environmentContext);
-        }
-
-        if (cascade && isPartitionedTable) {
-          //Currently only column related changes can be cascaded in alter table
-          if(!MetaStoreUtils.areSameColumns(oldt.getSd().getCols(), newt.getSd().getCols())) {
-            parts = msdb.getPartitions(dbname, name, -1);
-            for (Partition part : parts) {
-              List<FieldSchema> oldCols = part.getSd().getCols();
-              part.getSd().setCols(newt.getSd().getCols());
-              ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
-                  part.getValues(), oldCols, oldt, part);
-              assert(colStats == null);
-              msdb.alterPartition(dbname, name, part.getValues(), part);
-            }
-            msdb.alterTable(dbname, name, newt);
-          } else {
-            LOG.warn("Alter table does not cascade changes to its partitions.");
-          }
-        } else {
-          alterTableUpdateTableColumnStats(msdb, oldt, newt);
-        }
-      }
-
-      if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-        if (oldt.getDbName().equalsIgnoreCase(newt.getDbName())) {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventMessage.EventType.ALTER_TABLE,
-                  new AlterTableEvent(oldt, newt, false, true, handler),
-                  environmentContext);
-        } else {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventMessage.EventType.DROP_TABLE,
-                  new DropTableEvent(oldt, true, false, handler),
-                  environmentContext);
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventMessage.EventType.CREATE_TABLE,
-                  new CreateTableEvent(newt, true, handler),
-                  environmentContext);
-          if (isPartitionedTable) {
-            parts = msdb.getPartitions(newt.getDbName(), newt.getTableName(), -1);
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventMessage.EventType.ADD_PARTITION,
-                    new AddPartitionEvent(newt, parts, true, handler),
-                    environmentContext);
-          }
-        }
-      }
-      // commit the changes
-      success = msdb.commitTransaction();
-    } catch (InvalidObjectException e) {
-      LOG.debug("Failed to get object from Metastore ", e);
-      throw new InvalidOperationException(
-          "Unable to change partition or table."
-              + " Check metastore logs for detailed stack." + e.getMessage());
-    } catch (InvalidInputException e) {
-        LOG.debug("Accessing Metastore failed due to invalid input ", e);
-        throw new InvalidOperationException(
-            "Unable to change partition or table."
-                + " Check metastore logs for detailed stack." + e.getMessage());
-    } catch (NoSuchObjectException e) {
-      LOG.debug("Object not found in metastore ", e);
-      throw new InvalidOperationException(
-          "Unable to change partition or table. Database " + dbname + " does not exist"
-              + " Check metastore logs for detailed stack." + e.getMessage());
-    } finally {
-      if (!success) {
-        LOG.error("Failed to alter table " + dbname + "." + name);
-        msdb.rollbackTransaction();
-        if (dataWasMoved) {
-          try {
-            if (destFs.exists(destPath)) {
-              if (!destFs.rename(destPath, srcPath)) {
-                LOG.error("Failed to restore data from " + destPath + " to " + srcPath
-                    + " in alter table failure. Manual restore is needed.");
-              }
-            }
-          } catch (IOException e) {
-            LOG.error("Failed to restore data from " + destPath + " to " + srcPath
-                +  " in alter table failure. Manual restore is needed.");
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * MetaException that encapsulates error message from RemoteException from hadoop RPC which wrap
-   * the stack trace into e.getMessage() which makes logs/stack traces confusing.
-   * @param ex
-   * @return
-   */
-  String getSimpleMessage(Exception ex) {
-    if(ex instanceof MetaException) {
-      String msg = ex.getMessage();
-      if(msg == null || !msg.contains("\n")) {
-        return msg;
-      }
-      return msg.substring(0, msg.indexOf('\n'));
-    }
-    return ex.getMessage();
-  }
-
-  @Override
-  public Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<String> part_vals, final Partition new_part,
-    EnvironmentContext environmentContext)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    return alterPartition(msdb, wh, dbname, name, part_vals, new_part, environmentContext, null);
-  }
-
-  @Override
-  public Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<String> part_vals, final Partition new_part,
-    EnvironmentContext environmentContext, HMSHandler handler)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    boolean success = false;
-    Partition oldPart = null;
-    List<TransactionalMetaStoreEventListener> transactionalListeners = null;
-    if (handler != null) {
-      transactionalListeners = handler.getTransactionalListeners();
-    }
-
-    // Set DDL time to now if not specified
-    if (new_part.getParameters() == null ||
-        new_part.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
-        Integer.parseInt(new_part.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
-      new_part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
-          .currentTimeMillis() / 1000));
-    }
-
-    Table tbl = msdb.getTable(dbname, name);
-    if (tbl == null) {
-      throw new InvalidObjectException(
-          "Unable to alter partition because table or database does not exist.");
-    }
-
-    //alter partition
-    if (part_vals == null || part_vals.size() == 0) {
-      try {
-        msdb.openTransaction();
-        oldPart = msdb.getPartition(dbname, name, new_part.getValues());
-        if (MetaStoreUtils.requireCalStats(hiveConf, oldPart, new_part, tbl, environmentContext)) {
-          // if stats are same, no need to update
-          if (MetaStoreUtils.isFastStatsSame(oldPart, new_part)) {
-            MetaStoreUtils.updateBasicState(environmentContext, new_part.getParameters());
-          } else {
-            MetaStoreUtils.updatePartitionStatsFast(new_part, wh, false, true, environmentContext);
-          }
-        }
-
-        // PartitionView does not have SD. We do not need update its column stats
-        if (oldPart.getSd() != null) {
-          updateOrGetPartitionColumnStats(msdb, dbname, name, new_part.getValues(),
-              oldPart.getSd().getCols(), tbl, new_part);
-        }
-        msdb.alterPartition(dbname, name, new_part.getValues(), new_part);
-        if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                EventMessage.EventType.ALTER_PARTITION,
-                                                new AlterPartitionEvent(oldPart, new_part, tbl, false, true, handler),
-                                                environmentContext);
-
-
-        }
-        success = msdb.commitTransaction();
-      } catch (InvalidObjectException e) {
-        throw new InvalidOperationException("alter is not possible");
-      } catch (NoSuchObjectException e){
-        //old partition does not exist
-        throw new InvalidOperationException("alter is not possible");
-      } finally {
-        if(!success) {
-          msdb.rollbackTransaction();
-        }
-      }
-      return oldPart;
-    }
-
-    //rename partition
-    String oldPartLoc = null;
-    String newPartLoc = null;
-    Path srcPath = null;
-    Path destPath = null;
-    FileSystem srcFs = null;
-    FileSystem destFs = null;
-    boolean dataWasMoved = false;
-    try {
-      msdb.openTransaction();
-      try {
-        oldPart = msdb.getPartition(dbname, name, part_vals);
-      } catch (NoSuchObjectException e) {
-        // this means there is no existing partition
-        throw new InvalidObjectException(
-            "Unable to rename partition because old partition does not exist");
-      }
-
-      Partition check_part;
-      try {
-        check_part = msdb.getPartition(dbname, name, new_part.getValues());
-      } catch(NoSuchObjectException e) {
-        // this means there is no existing partition
-        check_part = null;
-      }
-
-      if (check_part != null) {
-        throw new AlreadyExistsException("Partition already exists:" + dbname + "." + name + "." +
-            new_part.getValues());
-      }
-
-      // when renaming a partition, we should update
-      // 1) partition SD Location
-      // 2) partition column stats if there are any because of part_name field in HMS table PART_COL_STATS
-      // 3) rename the partition directory if it is not an external table
-      if (!tbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
-        try {
-          // if tbl location is available use it
-          // else derive the tbl location from database location
-          destPath = wh.getPartitionPath(msdb.getDatabase(dbname), tbl, new_part.getValues());
-          destPath = constructRenamedPath(destPath, new Path(new_part.getSd().getLocation()));
-        } catch (NoSuchObjectException e) {
-          LOG.debug("Didn't find object in metastore ", e);
-          throw new InvalidOperationException(
-            "Unable to change partition or table. Database " + dbname + " does not exist"
-              + " Check metastore logs for detailed stack." + e.getMessage());
-        }
-
-        if (destPath != null) {
-          newPartLoc = destPath.toString();
-          oldPartLoc = oldPart.getSd().getLocation();
-          LOG.info("srcPath:" + oldPartLoc);
-          LOG.info("descPath:" + newPartLoc);
-          srcPath = new Path(oldPartLoc);
-          srcFs = wh.getFs(srcPath);
-          destFs = wh.getFs(destPath);
-          // check that src and dest are on the same file system
-          if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
-            throw new InvalidOperationException("New table location " + destPath
-              + " is on a different file system than the old location "
-              + srcPath + ". This operation is not supported.");
-          }
-
-          try {
-            if (srcFs.exists(srcPath)) {
-              if (newPartLoc.compareTo(oldPartLoc) != 0 && destFs.exists(destPath)) {
-                throw new InvalidOperationException("New location for this table "
-                  + tbl.getDbName() + "." + tbl.getTableName()
-                  + " already exists : " + destPath);
-              }
-              //if destPath's parent path doesn't exist, we should mkdir it
-              Path destParentPath = destPath.getParent();
-              if (!wh.mkdirs(destParentPath)) {
-                  throw new MetaException("Unable to create path " + destParentPath);
-              }
-
-              //rename the data directory
-              wh.renameDir(srcPath, destPath, true);
-              LOG.info("Partition directory rename from " + srcPath + " to " + destPath + " done.");
-              dataWasMoved = true;
-            }
-          } catch (IOException e) {
-            LOG.error("Cannot rename partition directory from " + srcPath + " to " + destPath, e);
-            throw new InvalidOperationException("Unable to access src or dest location for partition "
-                + tbl.getDbName() + "." + tbl.getTableName() + " " + new_part.getValues());
-          } catch (MetaException me) {
-            LOG.error("Cannot rename partition directory from " + srcPath + " to " + destPath, me);
-            throw me;
-          }
-
-          new_part.getSd().setLocation(newPartLoc);
-        }
-      } else {
-        new_part.getSd().setLocation(oldPart.getSd().getLocation());
-      }
-
-      if (MetaStoreUtils.requireCalStats(hiveConf, oldPart, new_part, tbl, environmentContext)) {
-        MetaStoreUtils.updatePartitionStatsFast(new_part, wh, false, true, environmentContext);
-      }
-
-      String newPartName = Warehouse.makePartName(tbl.getPartitionKeys(), new_part.getValues());
-      ColumnStatistics cs = updateOrGetPartitionColumnStats(msdb, dbname, name, oldPart.getValues(),
-          oldPart.getSd().getCols(), tbl, new_part);
-      msdb.alterPartition(dbname, name, part_vals, new_part);
-      if (cs != null) {
-        cs.getStatsDesc().setPartName(newPartName);
-        try {
-          msdb.updatePartitionColumnStatistics(cs, new_part.getValues());
-        } catch (InvalidInputException iie) {
-          throw new InvalidOperationException("Unable to update partition stats in table rename." + iie);
-        } catch (NoSuchObjectException nsoe) {
-          // It is ok, ignore
-        }
-      }
-
-      if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-        MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                              EventMessage.EventType.ALTER_PARTITION,
-                                              new AlterPartitionEvent(oldPart, new_part, tbl, false, true, handler),
-                                              environmentContext);
-      }
-
-      success = msdb.commitTransaction();
-    } finally {
-      if (!success) {
-        LOG.error("Failed to rename a partition. Rollback transaction");
-        msdb.rollbackTransaction();
-        if (dataWasMoved) {
-          LOG.error("Revert the data move in renaming a partition.");
-          try {
-            if (destFs.exists(destPath)) {
-              wh.renameDir(destPath, srcPath, false);
-            }
-          } catch (MetaException me) {
-            LOG.error("Failed to restore partition data from " + destPath + " to " + srcPath
-                +  " in alter partition failure. Manual restore is needed.");
-          } catch (IOException ioe) {
-            LOG.error("Failed to restore partition data from " + destPath + " to " + srcPath
-                +  " in alter partition failure. Manual restore is needed.");
-          }
-        }
-      }
-    }
-    return oldPart;
-  }
-
-  @Override
-  public List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<Partition> new_parts,
-    EnvironmentContext environmentContext)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    return alterPartitions(msdb, wh, dbname, name, new_parts, environmentContext, null);
-  }
-
-  @Override
-  public List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<Partition> new_parts, EnvironmentContext environmentContext,
-    HMSHandler handler)
-      throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    List<Partition> oldParts = new ArrayList<Partition>();
-    List<List<String>> partValsList = new ArrayList<List<String>>();
-    List<TransactionalMetaStoreEventListener> transactionalListeners = null;
-    if (handler != null) {
-      transactionalListeners = handler.getTransactionalListeners();
-    }
-
-    Table tbl = msdb.getTable(dbname, name);
-    if (tbl == null) {
-      throw new InvalidObjectException(
-          "Unable to alter partitions because table or database does not exist.");
-    }
-
-    boolean success = false;
-    try {
-      msdb.openTransaction();
-      for (Partition tmpPart: new_parts) {
-        // Set DDL time to now if not specified
-        if (tmpPart.getParameters() == null ||
-            tmpPart.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
-            Integer.parseInt(tmpPart.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
-          tmpPart.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
-              .currentTimeMillis() / 1000));
-        }
-
-        Partition oldTmpPart = msdb.getPartition(dbname, name, tmpPart.getValues());
-        oldParts.add(oldTmpPart);
-        partValsList.add(tmpPart.getValues());
-
-        if (MetaStoreUtils.requireCalStats(hiveConf, oldTmpPart, tmpPart, tbl, environmentContext)) {
-          // Check if stats are same, no need to update
-          if (MetaStoreUtils.isFastStatsSame(oldTmpPart, tmpPart)) {
-            MetaStoreUtils.updateBasicState(environmentContext, tmpPart.getParameters());
-          } else {
-            MetaStoreUtils.updatePartitionStatsFast(tmpPart, wh, false, true, environmentContext);
-          }
-        }
-
-        // PartitionView does not have SD and we do not need to update its column stats
-        if (oldTmpPart.getSd() != null) {
-          updateOrGetPartitionColumnStats(msdb, dbname, name, oldTmpPart.getValues(),
-              oldTmpPart.getSd().getCols(), tbl, tmpPart);
-        }
-      }
-
-      msdb.alterPartitions(dbname, name, partValsList, new_parts);
-      Iterator<Partition> oldPartsIt = oldParts.iterator();
-      for (Partition newPart : new_parts) {
-        Partition oldPart;
-        if (oldPartsIt.hasNext()) {
-          oldPart = oldPartsIt.next();
-        } else {
-          throw new InvalidOperationException("Missing old partition corresponding to new partition " +
-              "when invoking MetaStoreEventListener for alterPartitions event.");
-        }
-
-        if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                                                EventMessage.EventType.ALTER_PARTITION,
-                                                new AlterPartitionEvent(oldPart, newPart, tbl, false, true, handler));
-        }
-      }
-
-      success = msdb.commitTransaction();
-    } catch (InvalidObjectException | NoSuchObjectException e) {
-      throw new InvalidOperationException("Alter partition operation failed: " + e);
-    } finally {
-      if(!success) {
-        msdb.rollbackTransaction();
-      }
-    }
-
-    return oldParts;
-  }
-
-  private boolean checkPartialPartKeysEqual(List<FieldSchema> oldPartKeys,
-      List<FieldSchema> newPartKeys) {
-    //return true if both are null, or false if one is null and the other isn't
-    if (newPartKeys == null || oldPartKeys == null) {
-      return oldPartKeys == newPartKeys;
-    }
-    if (oldPartKeys.size() != newPartKeys.size()) {
-      return false;
-    }
-    Iterator<FieldSchema> oldPartKeysIter = oldPartKeys.iterator();
-    Iterator<FieldSchema> newPartKeysIter = newPartKeys.iterator();
-    FieldSchema oldFs;
-    FieldSchema newFs;
-    while (oldPartKeysIter.hasNext()) {
-      oldFs = oldPartKeysIter.next();
-      newFs = newPartKeysIter.next();
-      // Alter table can change the type of partition key now.
-      // So check the column name only.
-      if (!oldFs.getName().equals(newFs.getName())) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  /**
-   * Uses the scheme and authority of the object's current location and the path constructed
-   * using the object's new name to construct a path for the object's new location.
-   */
-  private Path constructRenamedPath(Path defaultNewPath, Path currentPath) {
-    URI currentUri = currentPath.toUri();
-
-    return new Path(currentUri.getScheme(), currentUri.getAuthority(),
-        defaultNewPath.toUri().getPath());
-  }
-
-  @VisibleForTesting
-  void alterTableUpdateTableColumnStats(RawStore msdb, Table oldTable, Table newTable)
-      throws MetaException, InvalidObjectException {
-    String dbName = oldTable.getDbName().toLowerCase();
-    String tableName = HiveStringUtils.normalizeIdentifier(oldTable.getTableName());
-    String newDbName = newTable.getDbName().toLowerCase();
-    String newTableName = HiveStringUtils.normalizeIdentifier(newTable.getTableName());
-
-    try {
-      List<FieldSchema> oldCols = oldTable.getSd().getCols();
-      List<FieldSchema> newCols = newTable.getSd().getCols();
-      List<ColumnStatisticsObj> newStatsObjs = new ArrayList<ColumnStatisticsObj>();
-      ColumnStatistics colStats = null;
-      boolean updateColumnStats = true;
-
-      // Nothing to update if everything is the same
-        if (newDbName.equals(dbName) &&
-            newTableName.equals(tableName) &&
-            MetaStoreUtils.columnsIncludedByNameType(oldCols, newCols)) {
-          updateColumnStats = false;
-        }
-
-        if (updateColumnStats) {
-          List<String> oldColNames = new ArrayList<String>(oldCols.size());
-          for (FieldSchema oldCol : oldCols) {
-            oldColNames.add(oldCol.getName());
-          }
-
-          // Collect column stats which need to be rewritten and remove old stats
-          colStats = msdb.getTableColumnStatistics(dbName, tableName, oldColNames);
-          if (colStats == null) {
-            updateColumnStats = false;
-          } else {
-            List<ColumnStatisticsObj> statsObjs = colStats.getStatsObj();
-            if (statsObjs != null) {
-              List<String> deletedCols = new ArrayList<String>();
-              for (ColumnStatisticsObj statsObj : statsObjs) {
-                boolean found = false;
-                for (FieldSchema newCol : newCols) {
-                  if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
-                      && statsObj.getColType().equalsIgnoreCase(newCol.getType())) {
-                    found = true;
-                    break;
-                  }
-                }
-
-                if (found) {
-                  if (!newDbName.equals(dbName) || !newTableName.equals(tableName)) {
-                    msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
-                    newStatsObjs.add(statsObj);
-                    deletedCols.add(statsObj.getColName());
-                  }
-                } else {
-                  msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
-                  deletedCols.add(statsObj.getColName());
-                }
-              }
-              StatsSetupConst.removeColumnStatsState(newTable.getParameters(), deletedCols);
-            }
-          }
-        }
-
-        // Change to new table and append stats for the new table
-        msdb.alterTable(dbName, tableName, newTable);
-        if (updateColumnStats && !newStatsObjs.isEmpty()) {
-          ColumnStatisticsDesc statsDesc = colStats.getStatsDesc();
-          statsDesc.setDbName(newDbName);
-          statsDesc.setTableName(newTableName);
-          colStats.setStatsObj(newStatsObjs);
-          msdb.updateTableColumnStatistics(colStats);
-        }
-    } catch (NoSuchObjectException nsoe) {
-      LOG.debug("Could not find db entry." + nsoe);
-    } catch (InvalidInputException e) {
-      //should not happen since the input were verified before passed in
-      throw new InvalidObjectException("Invalid inputs to update table column stats: " + e);
-    }
-  }
-
-  private ColumnStatistics updateOrGetPartitionColumnStats(
-      RawStore msdb, String dbname, String tblname, List<String> partVals,
-      List<FieldSchema> oldCols, Table table, Partition part)
-          throws MetaException, InvalidObjectException {
-    ColumnStatistics newPartsColStats = null;
-    try {
-      List<FieldSchema> newCols = part.getSd() == null ?
-          new ArrayList<FieldSchema>() : part.getSd().getCols();
-      String oldPartName = Warehouse.makePartName(table.getPartitionKeys(), partVals);
-      String newPartName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
-      boolean rename = !part.getDbName().equals(dbname) || !part.getTableName().equals(tblname)
-          || !oldPartName.equals(newPartName);
-
-      // do not need to update column stats if alter partition is not for rename or changing existing columns
-      if (!rename && MetaStoreUtils.columnsIncludedByNameType(oldCols, newCols)) {
-        return newPartsColStats;
-      }
-      List<String> oldColNames = new ArrayList<String>(oldCols.size());
-      for (FieldSchema oldCol : oldCols) {
-        oldColNames.add(oldCol.getName());
-      }
-      List<String> oldPartNames = Lists.newArrayList(oldPartName);
-      List<ColumnStatistics> partsColStats = msdb.getPartitionColumnStatistics(dbname, tblname,
-          oldPartNames, oldColNames);
-      assert (partsColStats.size() <= 1);
-      for (ColumnStatistics partColStats : partsColStats) { //actually only at most one loop
-        List<ColumnStatisticsObj> newStatsObjs = new ArrayList<ColumnStatisticsObj>();
-        List<ColumnStatisticsObj> statsObjs = partColStats.getStatsObj();
-        List<String> deletedCols = new ArrayList<String>();
-        for (ColumnStatisticsObj statsObj : statsObjs) {
-          boolean found =false;
-          for (FieldSchema newCol : newCols) {
-            if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
-                && statsObj.getColType().equalsIgnoreCase(newCol.getType())) {
-              found = true;
-              break;
-            }
-          }
-          if (found) {
-            if (rename) {
-              msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
-                  partVals, statsObj.getColName());
-              newStatsObjs.add(statsObj);
-            }
-          } else {
-            msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
-                partVals, statsObj.getColName());
-            deletedCols.add(statsObj.getColName());
-          }
-        }
-        StatsSetupConst.removeColumnStatsState(part.getParameters(), deletedCols);
-        if (!newStatsObjs.isEmpty()) {
-          partColStats.setStatsObj(newStatsObjs);
-          newPartsColStats = partColStats;
-        }
-      }
-    } catch (NoSuchObjectException nsoe) {
-      // ignore this exception, actually this exception won't be thrown from getPartitionColumnStatistics
-    } catch (InvalidInputException iie) {
-      throw new InvalidObjectException("Invalid input to delete partition column stats." + iie);
-    }
-
-    return newPartsColStats;
-  }
-
-  private void checkColTypeChangeCompatible(List<FieldSchema> oldCols, List<FieldSchema> newCols)
-      throws InvalidOperationException {
-    List<String> incompatibleCols = new ArrayList<>();
-    int maxCols = Math.min(oldCols.size(), newCols.size());
-    for (int i = 0; i < maxCols; i++) {
-      if (!ColumnType.areColTypesCompatible(
-          ColumnType.getTypeName(oldCols.get(i).getType()),
-          ColumnType.getTypeName(newCols.get(i).getType()))) {
-        incompatibleCols.add(newCols.get(i).getName());
-      }
-    }
-    if (!incompatibleCols.isEmpty()) {
-      throw new InvalidOperationException(
-          "The following columns have types incompatible with the existing " +
-              "columns in their respective positions :\n" +
-              org.apache.commons.lang.StringUtils.join(incompatibleCols, ',')
-      );
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 7337c1c..8a55305 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -130,7 +130,10 @@ import org.apache.hadoop.hive.metastore.repl.DumpDirCleanerTask;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
 import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
 import org.apache.hadoop.hive.metastore.security.TUGIContainingTransport;
+import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService;
+import org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService;
+import org.apache.hadoop.hive.metastore.txn.AcidWriteSetService;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.serde2.Deserializer;
@@ -215,17 +218,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
   }
 
-  /**
-   * An ugly interface because everything about this file is ugly. RawStore is threadlocal so this
-   * thread-local disease propagates everywhere, and FileMetadataManager cannot just get a RawStore
-   * or handlers to use; it will need to have this method to make thread-local handlers and a
-   * thread-local RawStore.
-   */
-  public interface ThreadLocalRawStore {
-    RawStore getMS() throws MetaException;
-  }
-
-  public static class HMSHandler extends FacebookBase implements IHMSHandler, ThreadLocalRawStore {
+  public static class HMSHandler extends FacebookBase implements IHMSHandler {
     public static final Logger LOG = HiveMetaStore.LOG;
     private final HiveConf hiveConf; // stores datastore (jpox) properties,
                                      // right now they come from jpox.properties
@@ -406,10 +399,20 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return threadLocalIpAddress.get();
     }
 
+    /**
+     * Use {@link #getThreadId()} instead.
+     * @return thread id
+     */
+    @Deprecated
     public static Integer get() {
       return threadLocalId.get();
     }
 
+    @Override
+    public int getThreadId() {
+      return threadLocalId.get();
+    }
+
     public HMSHandler(String name) throws MetaException {
       this(name, new HiveConf(HMSHandler.class), true);
     }
@@ -457,7 +460,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    List<TransactionalMetaStoreEventListener> getTransactionalListeners() {
+    @Override
+    public List<TransactionalMetaStoreEventListener> getTransactionalListeners() {
       return transactionalListeners;
     }
 
@@ -537,7 +541,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         cleaner.schedule(new DumpDirCleanerTask(hiveConf), cleanFreq, cleanFreq);
       }
       expressionProxy = PartFilterExprUtil.createExpressionProxy(hiveConf);
-      fileMetadataManager = new FileMetadataManager((ThreadLocalRawStore)this, hiveConf);
+      fileMetadataManager = new FileMetadataManager(this.getMS(), hiveConf);
     }
 
     private static String addPrefix(String s) {
@@ -580,6 +584,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return modifiedConf;
     }
 
+    @Override
     public Warehouse getWh() {
       return wh;
     }
@@ -1027,14 +1032,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return db;
     }
 
-    /**
-     * Equivalent to get_database, but does not write to audit logs, or fire pre-event listners.
-     * Meant to be used for internal hive classes that don't use the thrift interface.
-     * @param name
-     * @return
-     * @throws NoSuchObjectException
-     * @throws MetaException
-     */
+    @Override
     public Database get_database_core(final String name) throws NoSuchObjectException,
         MetaException {
       Database db = null;
@@ -2400,16 +2398,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return t;
     }
 
-    /**
-     * Equivalent of get_table, but does not log audits and fire pre-event listener.
-     * Meant to be used for calls made by other hive classes, that are not using the
-     * thrift interface.
-     * @param dbname
-     * @param name
-     * @return Table object
-     * @throws MetaException
-     * @throws NoSuchObjectException
-     */
+    @Override
     public Table get_table_core(final String dbname, final String name) throws MetaException,
         NoSuchObjectException {
       Table t = null;
@@ -8059,33 +8048,29 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     if(!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_INITIATOR_ON)) {
       return;
     }
-    startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService"));
-    startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidCompactionHistoryService"));
-    startHouseKeeperService(conf, Class.forName("org.apache.hadoop.hive.ql.txn.AcidWriteSetService"));
 
     ThreadPool.initialize(conf);
-    RunnableConfigurable rc = new AcidOpenTxnsCounterService();
-    rc.setConf(conf);
-    ThreadPool.getPool().scheduleAtFixedRate(rc, 100, MetastoreConf.getTimeVar(conf,
-        MetastoreConf.ConfVars.COUNT_OPEN_TXNS_INTERVAL, TimeUnit.MILLISECONDS),
-        TimeUnit.MILLISECONDS);
-
+    startOneHouseKeeperService(new AcidHouseKeeperService(), conf,
+        MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TIMEDOUT_TXN_REAPER_INTERVAL,
+            TimeUnit.MILLISECONDS));
+    startOneHouseKeeperService(new AcidOpenTxnsCounterService(), conf,
+        MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.COUNT_OPEN_TXNS_INTERVAL,
+            TimeUnit.MILLISECONDS));
+    startOneHouseKeeperService(new AcidCompactionHistoryService(), conf,
+        MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_REAPER_INTERVAL,
+            TimeUnit.MILLISECONDS));
+    startOneHouseKeeperService(new AcidWriteSetService(), conf,
+        MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.WRITE_SET_REAPER_INTERVAL,
+            TimeUnit.MILLISECONDS));
   }
-  private static void startHouseKeeperService(HiveConf conf, Class<?> c) throws Exception {
-    //todo: when metastore adds orderly-shutdown logic, houseKeeper.stop()
-    //should be called form it
-    HouseKeeperService houseKeeper = (HouseKeeperService)c.newInstance();
-    try {
-      houseKeeper.start(conf);
-    }
-    catch (Exception ex) {
-      LOG.error("Failed to start {}" , houseKeeper.getClass() +
-        ".  The system will not handle {} " , houseKeeper.getServiceDescription(),
-        ".  Root Cause: ", ex);
-    }
+
+  private static void startOneHouseKeeperService(RunnableConfigurable rc, Configuration conf,
+                                                 long interval) {
+    rc.setConf(conf);
+    ThreadPool.getPool().scheduleAtFixedRate(rc, 0, interval, TimeUnit.MILLISECONDS);
   }
 
-  public static Map<FileMetadataExprType, FileMetadataHandler> createHandlerMap() {
+  static Map<FileMetadataExprType, FileMetadataHandler> createHandlerMap() {
     Map<FileMetadataExprType, FileMetadataHandler> fmHandlers = new HashMap<>();
     for (FileMetadataExprType v : FileMetadataExprType.values()) {
       switch (v) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/HouseKeeperService.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HouseKeeperService.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HouseKeeperService.java
deleted file mode 100644
index 539ace0..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HouseKeeperService.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.conf.HiveConf;
-
-/**
- * Runs arbitrary background logic inside the metastore service.  
- */
-@InterfaceAudience.LimitedPrivate({"Hive"})
-@InterfaceStability.Evolving
-public interface HouseKeeperService {
-  public void start(HiveConf hiveConf) throws Exception;
-  /**
-   * Should perform orderly shutdown
-   */
-  public void stop();
-  /**
-   * Returns short description of services this module provides.
-   */
-  public String getServiceDescription();
-
-  /**
-   * This is incremented each time the service is performed.  Can be useful to
-   * check if serivce is still alive.
-   */
-  public int getIsAliveCounter();
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
deleted file mode 100644
index 5a72082..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
-import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
-import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
-import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
-import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
-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;
-import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
-import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
-import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.DropConstraintEvent;
-import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
-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.InsertEvent;
-import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
-
-/**
- * This abstract class needs to be extended to  provide implementation of actions that needs
- * to be performed when a particular event occurs on a metastore. These methods
- * are called whenever an event occurs on metastore. Status of the event whether
- * it was successful or not is contained in container event object.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class MetaStoreEventListener implements Configurable {
-
-  private Configuration conf;
-
-  public MetaStoreEventListener(Configuration config){
-    this.conf = config;
-  }
-
-  /**
-   * @param tableEvent table event.
-   * @throws MetaException
-   */
-  public void onConfigChange(ConfigChangeEvent tableEvent) throws MetaException {
-  }
-
-  /**
-   * @param tableEvent table event.
-   * @throws MetaException
-   */
-  public void onCreateTable (CreateTableEvent tableEvent) throws MetaException {
-  }
-
-  /**
-   * @param tableEvent table event.
-   * @throws MetaException
-   */
-  public void onDropTable (DropTableEvent tableEvent)  throws MetaException {
-  }
-
-  /**
-   * @param tableEvent alter table event
-   * @throws MetaException
-   */
-  public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
-  }
-
-  /**
-   * @param partitionEvent add partition event
-   * @throws MetaException
-   */
-  public void onAddPartition (AddPartitionEvent partitionEvent) throws MetaException {
-  }
-
-  /**
-   * @param partitionEvent partition event
-   * @throws MetaException
-   */
-  public void onDropPartition (DropPartitionEvent partitionEvent)  throws MetaException {
-  }
-
-  /**
-   * @param partitionEvent partition event
-   * @throws MetaException
-   */
-  public void onAlterPartition (AlterPartitionEvent partitionEvent)  throws MetaException {
-  }
-
-  /**
-   * @param dbEvent database event
-   * @throws MetaException
-   */
-  public void onCreateDatabase (CreateDatabaseEvent dbEvent) throws MetaException {
-  }
-
-  /**
-   * @param dbEvent database event
-   * @throws MetaException
-   */
-  public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
-  }
-
-  /**
-   * @param partSetDoneEvent
-   * @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 {
-  }
-
-  /**
-   * @param fnEvent function event
-   * @throws MetaException
-   */
-  public void onCreateFunction (CreateFunctionEvent fnEvent) throws MetaException {
-  }
-
-  /**
-   * @param fnEvent function event
-   * @throws MetaException
-   */
-  public void onDropFunction (DropFunctionEvent fnEvent) throws MetaException {
-  }
-
-  /**
-   * This will be called when an insert is executed that does not cause a partition to be added.
-   * If an insert causes a partition to be added it will cause {@link #onAddPartition} to be
-   * called instead.
-   * @param insertEvent
-   * @throws MetaException
-   */
-  public void onInsert(InsertEvent insertEvent) throws MetaException {
-
-  }
-
-  /**
-   * @param addPrimaryKeyEvent add primary key event
-   * @throws MetaException
-   */
-  public void onAddPrimaryKey(AddPrimaryKeyEvent addPrimaryKeyEvent) throws MetaException {
-  }
-
-  /**
-   * @param addForeignKeyEvent add foreign key event
-   * @throws MetaException
-   */
-  public void onAddForeignKey(AddForeignKeyEvent addForeignKeyEvent) throws MetaException {
-  }
-
-  /**
-   * @param addUniqueConstraintEvent add unique constraint event
-   * @throws MetaException
-   */
-  public void onAddUniqueConstraint(AddUniqueConstraintEvent addUniqueConstraintEvent) throws MetaException {
-  }
-
-  /**
-   * @param addNotNullConstraintEvent add not null constraint event
-   * @throws MetaException
-   */
-  public void onAddNotNullConstraint(AddNotNullConstraintEvent addNotNullConstraintEvent) throws MetaException {
-  }
-
-  /**
-   * @param dropConstraintEvent drop constraint event
-   * @throws MetaException
-   */
-  public void onDropConstraint(DropConstraintEvent dropConstraintEvent) throws MetaException {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return this.conf;
-  }
-
-  @Override
-  public void setConf(Configuration config) {
-    this.conf = config;
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
deleted file mode 100644
index 79de79d..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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;
-
-/**
- * Keeps a list of reserved keys used by Hive listeners when updating the ListenerEvent
- * parameters.
- */
-public class MetaStoreEventListenerConstants {
-  /*
-   * DbNotificationListener keys reserved for updating ListenerEvent parameters.
-   *
-   * DB_NOTIFICATION_EVENT_ID_KEY_NAME This key will have the event identifier that DbNotificationListener
-   *                                   processed during an event. This event identifier might be shared
-   *                                   across other MetaStoreEventListener implementations.
-   */
-  public static final String DB_NOTIFICATION_EVENT_ID_KEY_NAME = "DB_NOTIFICATION_EVENT_ID_KEY_NAME";
-
-  /*
-   * HiveMetaStore keys reserved for updating ListenerEvent parameters.
-   *
-   * HIVE_METASTORE_TRANSACTION_ACTIVE This key is used to check if a listener event is run inside a current
-   *                                   transaction. A boolean value is used for active (true) or no active (false).
-   */
-  public static final String HIVE_METASTORE_TRANSACTION_ACTIVE = "HIVE_METASTORE_TRANSACTION_ACTIVE";
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java
deleted file mode 100644
index 2ce621e..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitContext.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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;
-
-/**
- * Base class which provides context to implementations of MetaStoreInitListener
- */
-
-public class MetaStoreInitContext {
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java
deleted file mode 100644
index c0f62bc..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreInitListener.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-
-/**
- * This abstract class needs to be extended to  provide implementation of actions
- * that needs to be performed when HMSHandler is initialized
- */
-
-public abstract class MetaStoreInitListener implements Configurable {
-
-  private Configuration conf;
-
-  public MetaStoreInitListener(Configuration config){
-    this.conf = config;
-  }
-
-  public abstract void onInit(MetaStoreInitContext context) throws MetaException;
-
-  @Override
-  public Configuration getConf() {
-    return this.conf;
-  }
-
-  @Override
-  public void setConf(Configuration config) {
-    this.conf = config;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
deleted file mode 100644
index 331621e..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/**
- * 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;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
-import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
-import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
-import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
-import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
-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.CreateDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
-import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
-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.InsertEvent;
-import org.apache.hadoop.hive.metastore.events.ListenerEvent;
-
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants.HIVE_METASTORE_TRANSACTION_ACTIVE;
-import static org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
-
-/**
- * This class is used to notify a list of listeners about specific MetaStore events.
- */
-@Private
-public class MetaStoreListenerNotifier {
-
-  private interface EventNotifier {
-    void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException;
-  }
-
-  private static Map<EventType, EventNotifier> notificationEvents = Maps.newHashMap(
-      ImmutableMap.<EventType, EventNotifier>builder()
-          .put(EventType.CREATE_DATABASE, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onCreateDatabase((CreateDatabaseEvent)event);
-            }
-          })
-          .put(EventType.DROP_DATABASE, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onDropDatabase((DropDatabaseEvent)event);
-            }
-          })
-          .put(EventType.CREATE_TABLE, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onCreateTable((CreateTableEvent)event);
-            }
-          })
-          .put(EventType.DROP_TABLE, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onDropTable((DropTableEvent)event);
-            }
-          })
-          .put(EventType.ADD_PARTITION, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAddPartition((AddPartitionEvent)event);
-            }
-          })
-          .put(EventType.DROP_PARTITION, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onDropPartition((DropPartitionEvent)event);
-            }
-          })
-          .put(EventType.ALTER_TABLE, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAlterTable((AlterTableEvent)event);
-            }
-          })
-          .put(EventType.ALTER_PARTITION, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAlterPartition((AlterPartitionEvent)event);
-            }
-          })
-          .put(EventType.INSERT, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onInsert((InsertEvent)event);
-            }
-          })
-          .put(EventType.CREATE_FUNCTION, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onCreateFunction((CreateFunctionEvent)event);
-            }
-          })
-          .put(EventType.DROP_FUNCTION, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onDropFunction((DropFunctionEvent)event);
-            }
-          })
-          .put(EventType.CREATE_INDEX, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAddIndex((AddIndexEvent)event);
-            }
-          })
-          .put(EventType.DROP_INDEX, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onDropIndex((DropIndexEvent)event);
-            }
-          })
-          .put(EventType.ALTER_INDEX, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAlterIndex((AlterIndexEvent)event);
-            }
-          })
-          .put(EventType.ADD_PRIMARYKEY, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAddPrimaryKey((AddPrimaryKeyEvent)event);
-            }
-          })
-          .put(EventType.ADD_FOREIGNKEY, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAddForeignKey((AddForeignKeyEvent)event);
-            }
-          })
-          .put(EventType.ADD_UNIQUECONSTRAINT, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAddUniqueConstraint((AddUniqueConstraintEvent)event);
-            }
-          })
-          .put(EventType.ADD_NOTNULLCONSTRAINT, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onAddNotNullConstraint((AddNotNullConstraintEvent)event);
-            }
-          })
-          .build()
-  );
-
-  /**
-   * Notify a list of listeners about a specific metastore event. Each listener notified might update
-   * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
-   * be returned to the caller.
-   *
-   * @param listeners List of MetaStoreEventListener listeners.
-   * @param eventType Type of the notification event.
-   * @param event The ListenerEvent with information about the event.
-   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
-   *         map if no parameters were updated or if no listeners were notified.
-   * @throws MetaException If an error occurred while calling the listeners.
-   */
-  public static Map<String, String> notifyEvent(List<? extends MetaStoreEventListener> listeners,
-                                                EventType eventType,
-                                                ListenerEvent event) throws MetaException {
-
-    Preconditions.checkNotNull(listeners, "Listeners must not be null.");
-    Preconditions.checkNotNull(event, "The event must not be null.");
-
-    for (MetaStoreEventListener listener : listeners) {
-        notificationEvents.get(eventType).notify(listener, event);
-    }
-
-    // Each listener called above might set a different parameter on the event.
-    // This write permission is allowed on the listener side to avoid breaking compatibility if we change the API
-    // method calls.
-    return event.getParameters();
-  }
-
-  /**
-   * Notify a list of listeners about a specific metastore event. Each listener notified might update
-   * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
-   * be returned to the caller.
-   *
-   * @param listeners List of MetaStoreEventListener listeners.
-   * @param eventType Type of the notification event.
-   * @param event The ListenerEvent with information about the event.
-   * @param environmentContext An EnvironmentContext object with parameters sent by the HMS client.
-   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
-   *         map if no parameters were updated or if no listeners were notified.
-   * @throws MetaException If an error occurred while calling the listeners.
-   */
-  public static Map<String, String> notifyEvent(List<? extends MetaStoreEventListener> listeners,
-                                                EventType eventType,
-                                                ListenerEvent event,
-                                                EnvironmentContext environmentContext) throws MetaException {
-
-    Preconditions.checkNotNull(event, "The event must not be null.");
-
-    event.setEnvironmentContext(environmentContext);
-    return notifyEvent(listeners, eventType, event);
-  }
-
-  /**
-   * Notify a list of listeners about a specific metastore event. Each listener notified might update
-   * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
-   * be returned to the caller.
-   *
-   * Sometimes these events are run inside a DB transaction and might cause issues with the listeners,
-   * for instance, Sentry blocks the HMS until an event is seen committed on the DB. To notify the listener about this,
-   * a new parameter to verify if a transaction is active is added to the ListenerEvent, and is up to the listener
-   * to skip this notification if so.
-   *
-   * @param listeners List of MetaStoreEventListener listeners.
-   * @param eventType Type of the notification event.
-   * @param event The ListenerEvent with information about the event.
-   * @param environmentContext An EnvironmentContext object with parameters sent by the HMS client.
-   * @param parameters A list of key/value pairs with the new parameters to add.
-   * @param ms The RawStore object from where to check if a transaction is active.
-   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
-   *         map if no parameters were updated or if no listeners were notified.
-   * @throws MetaException If an error occurred while calling the listeners.
-   */
-  public static Map<String, String> notifyEvent(List<? extends MetaStoreEventListener> listeners,
-                                                EventType eventType,
-                                                ListenerEvent event,
-                                                EnvironmentContext environmentContext,
-                                                Map<String, String> parameters,
-                                                final RawStore ms) throws MetaException {
-
-    Preconditions.checkNotNull(event, "The event must not be null.");
-
-    event.putParameters(parameters);
-
-    if (ms != null) {
-      event.putParameter(HIVE_METASTORE_TRANSACTION_ACTIVE, Boolean.toString(ms.isActiveTransaction()));
-    }
-
-    return notifyEvent(listeners, eventType, event, environmentContext);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java
deleted file mode 100644
index bca6a31..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStorePreEventListener.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.events.PreEventContext;
-
-/**
- * This abstract class needs to be extended to  provide implementation of actions that needs
- * to be performed before a particular event occurs on a metastore. These methods
- * are called before an event occurs on metastore.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class MetaStorePreEventListener implements Configurable {
-
-  private Configuration conf;
-
-  public MetaStorePreEventListener(Configuration config){
-    this.conf = config;
-  }
-
-  public abstract void onEvent(PreEventContext context)
-      throws MetaException, NoSuchObjectException, InvalidOperationException;
-
-  @Override
-  public Configuration getConf() {
-    return this.conf;
-  }
-
-  @Override
-  public void setConf(Configuration config) {
-    this.conf = config;
-  }
-}


[03/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java
new file mode 100644
index 0000000..78259ba
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterIndexMessage.java
@@ -0,0 +1,33 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public abstract class AlterIndexMessage extends EventMessage {
+
+  public abstract Index getIndexObjBefore() throws Exception;
+
+  public abstract Index getIndexObjAfter() throws Exception;
+
+  protected AlterIndexMessage() {
+    super(EventType.ALTER_INDEX);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
new file mode 100644
index 0000000..aaa7ef5
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterPartitionMessage.java
@@ -0,0 +1,69 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.Map;
+
+public abstract class AlterPartitionMessage extends EventMessage {
+
+  protected AlterPartitionMessage() {
+    super(EventType.ALTER_PARTITION);
+  }
+
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  public abstract boolean getIsTruncateOp();
+
+  public abstract Map<String,String> getKeyValues();
+
+  public abstract Table getTableObj() throws Exception;
+
+  public abstract Partition getPtnObjBefore() throws Exception;
+
+  public abstract Partition getPtnObjAfter() throws Exception;
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null) throw new IllegalStateException("Table name unset.");
+    if (getKeyValues() == null) throw new IllegalStateException("Partition values unset");
+    try {
+      if (getTableObj() == null){
+        throw new IllegalStateException("Table object not set.");
+      }
+      if (getPtnObjAfter() == null){
+        throw new IllegalStateException("Partition object(after) not set.");
+      }
+      if (getPtnObjBefore() == null){
+        throw new IllegalStateException("Partition object(before) not set.");
+      }
+    } catch (Exception e) {
+      if (! (e instanceof IllegalStateException)){
+        throw new IllegalStateException("Event not set up correctly",e);
+      } else {
+        throw (IllegalStateException) e;
+      }
+    }
+    return super.checkValid();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
new file mode 100644
index 0000000..30e2862
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AlterTableMessage.java
@@ -0,0 +1,58 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+
+public abstract class AlterTableMessage extends EventMessage {
+
+  protected AlterTableMessage() {
+    super(EventType.ALTER_TABLE);
+  }
+
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  public abstract boolean getIsTruncateOp();
+
+  public abstract Table getTableObjBefore() throws Exception;
+
+  public abstract Table getTableObjAfter() throws Exception;
+
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null) throw new IllegalStateException("Table name unset.");
+    try {
+      if (getTableObjAfter() == null){
+        throw new IllegalStateException("Table object(after) not set.");
+      }
+      if (getTableObjBefore() == null){
+        throw new IllegalStateException("Table object(before) not set.");
+      }
+    } catch (Exception e) {
+      if (! (e instanceof IllegalStateException)){
+        throw new IllegalStateException("Event not set up correctly",e);
+      } else {
+        throw (IllegalStateException) e;
+      }
+    }
+    return super.checkValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java
new file mode 100644
index 0000000..328c118
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateDatabaseMessage.java
@@ -0,0 +1,28 @@
+/*
+ * 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.messaging;
+
+public abstract class CreateDatabaseMessage extends EventMessage {
+
+  protected CreateDatabaseMessage() {
+    super(EventType.CREATE_DATABASE);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java
new file mode 100644
index 0000000..5478143
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateFunctionMessage.java
@@ -0,0 +1,46 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Function;
+
+public abstract class CreateFunctionMessage extends EventMessage {
+
+  protected CreateFunctionMessage() {
+    super(EventType.CREATE_FUNCTION);
+  }
+
+  public abstract Function getFunctionObj() throws Exception;
+
+  @Override
+  public EventMessage checkValid() {
+    try {
+      if (getFunctionObj() == null)
+        throw new IllegalStateException("Function object unset.");
+    } catch (Exception e) {
+      if (! (e instanceof IllegalStateException)){
+        throw new IllegalStateException("Event not set up correctly", e);
+      } else {
+        throw (IllegalStateException) e;
+      }
+    }
+    return super.checkValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java
new file mode 100644
index 0000000..f107100
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateIndexMessage.java
@@ -0,0 +1,46 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+
+public abstract class CreateIndexMessage extends EventMessage {
+
+  protected CreateIndexMessage() {
+    super(EventType.CREATE_INDEX);
+  }
+
+  public abstract Index getIndexObj() throws Exception;
+
+  @Override
+  public EventMessage checkValid() {
+    try {
+      if (getIndexObj() == null)
+        throw new IllegalStateException("Function object unset.");
+    } catch (Exception e) {
+      if (! (e instanceof IllegalStateException)){
+        throw new IllegalStateException("Event not set up correctly", e);
+      } else {
+        throw (IllegalStateException) e;
+      }
+    }
+    return super.checkValid();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java
new file mode 100644
index 0000000..49732ff
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateTableMessage.java
@@ -0,0 +1,53 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+
+public abstract class CreateTableMessage extends EventMessage {
+
+  protected CreateTableMessage() {
+    super(EventType.CREATE_TABLE);
+  }
+
+  /**
+   * Getter for the name of table created
+   * @return Table-name (String).
+   */
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  public abstract Table getTableObj() throws Exception;
+
+  /**
+   * Get list of files created as a result of this DML operation
+   *
+   * @return The iterable of files
+   */
+  public abstract Iterable<String> getFiles();
+
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null)
+      throw new IllegalStateException("Table name unset.");
+    return super.checkValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java
new file mode 100644
index 0000000..95c9f9f
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropConstraintMessage.java
@@ -0,0 +1,29 @@
+/*
+ * 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.messaging;
+
+public abstract class DropConstraintMessage extends EventMessage {
+  protected DropConstraintMessage() {
+    super(EventType.DROP_CONSTRAINT);
+  }
+
+  public abstract String getTable();
+
+  public abstract String getConstraint();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java
new file mode 100644
index 0000000..a450d47
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropDatabaseMessage.java
@@ -0,0 +1,27 @@
+/*
+ * 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.messaging;
+
+public abstract class DropDatabaseMessage extends EventMessage {
+
+  protected DropDatabaseMessage() {
+    super(EventType.DROP_DATABASE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java
new file mode 100644
index 0000000..c418feb
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropFunctionMessage.java
@@ -0,0 +1,38 @@
+/*
+ * 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.messaging;
+
+public abstract class DropFunctionMessage extends EventMessage {
+
+  public abstract String getFunctionName();
+
+  protected DropFunctionMessage() {
+    super(EventType.DROP_FUNCTION);
+  }
+
+  @Override
+  public EventMessage checkValid() {
+    if (getFunctionName() == null){
+      throw new IllegalStateException("Function name unset.");
+    }
+    return super.checkValid();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java
new file mode 100644
index 0000000..210b592
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropIndexMessage.java
@@ -0,0 +1,46 @@
+/*
+ * 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.messaging;
+
+public abstract class DropIndexMessage extends EventMessage {
+
+  public abstract String getIndexName();
+  public abstract String getOrigTableName();
+  public abstract String getIndexTableName();
+
+  protected DropIndexMessage() {
+    super(EventType.DROP_INDEX);
+  }
+
+  @Override
+  public EventMessage checkValid() {
+    if (getIndexName() == null){
+      throw new IllegalStateException("Index name unset.");
+    }
+    if (getOrigTableName() == null){
+      throw new IllegalStateException("Index original table name unset.");
+    }
+    // NOTE: we do not do a not-null check on getIndexTableName,
+    // since, per the index design wiki, it can actually be null.
+
+    return super.checkValid();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java
new file mode 100644
index 0000000..fa7c9db
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropPartitionMessage.java
@@ -0,0 +1,49 @@
+/*
+ * 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.messaging;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+
+public abstract class DropPartitionMessage extends EventMessage {
+
+  protected DropPartitionMessage() {
+    super(EventType.DROP_PARTITION);
+  }
+
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  public abstract Table getTableObj() throws Exception;
+
+  public abstract List<Map<String, String>> getPartitions ();
+
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null)
+      throw new IllegalStateException("Table name unset.");
+    if (getPartitions() == null)
+      throw new IllegalStateException("Partition-list unset.");
+    return super.checkValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java
new file mode 100644
index 0000000..b8e0e78
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropTableMessage.java
@@ -0,0 +1,46 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+
+public abstract class DropTableMessage extends EventMessage {
+
+  protected DropTableMessage() {
+    super(EventType.DROP_TABLE);
+  }
+
+  /**
+   * Getter for the name of the table being dropped.
+   * @return Table-name (String).
+   */
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  public abstract Table getTableObj() throws Exception;
+
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null)
+      throw new IllegalStateException("Table name unset.");
+    return super.checkValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
new file mode 100644
index 0000000..7b22fac
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
@@ -0,0 +1,111 @@
+/*
+ * 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.messaging;
+
+/**
+ * Class representing messages emitted when Metastore operations are done.
+ * (E.g. Creation and deletion of databases, tables and partitions.)
+ */
+public abstract class EventMessage {
+
+  /**
+   * Enumeration of all supported types of Metastore operations.
+   */
+  public enum EventType {
+
+    CREATE_DATABASE(MessageFactory.CREATE_DATABASE_EVENT),
+    DROP_DATABASE(MessageFactory.DROP_DATABASE_EVENT),
+    CREATE_TABLE(MessageFactory.CREATE_TABLE_EVENT),
+    DROP_TABLE(MessageFactory.DROP_TABLE_EVENT),
+    ADD_PARTITION(MessageFactory.ADD_PARTITION_EVENT),
+    DROP_PARTITION(MessageFactory.DROP_PARTITION_EVENT),
+    ALTER_TABLE(MessageFactory.ALTER_TABLE_EVENT),
+    ALTER_PARTITION(MessageFactory.ALTER_PARTITION_EVENT),
+    INSERT(MessageFactory.INSERT_EVENT),
+    CREATE_FUNCTION(MessageFactory.CREATE_FUNCTION_EVENT),
+    DROP_FUNCTION(MessageFactory.DROP_FUNCTION_EVENT),
+    CREATE_INDEX(MessageFactory.CREATE_INDEX_EVENT),
+    DROP_INDEX(MessageFactory.DROP_INDEX_EVENT),
+    ALTER_INDEX(MessageFactory.ALTER_INDEX_EVENT),
+    ADD_PRIMARYKEY(MessageFactory.ADD_PRIMARYKEY_EVENT),
+    ADD_FOREIGNKEY(MessageFactory.ADD_FOREIGNKEY_EVENT),
+    ADD_UNIQUECONSTRAINT(MessageFactory.ADD_UNIQUECONSTRAINT_EVENT),
+    ADD_NOTNULLCONSTRAINT(MessageFactory.ADD_NOTNULLCONSTRAINT_EVENT),
+    DROP_CONSTRAINT(MessageFactory.DROP_CONSTRAINT_EVENT);
+
+    private String typeString;
+
+    EventType(String typeString) {
+      this.typeString = typeString;
+    }
+
+    @Override
+    public String toString() { return typeString; }
+  }
+
+  protected EventType eventType;
+
+  protected EventMessage(EventType eventType) {
+    this.eventType = eventType;
+  }
+
+  public EventType getEventType() {
+    return eventType;
+  }
+
+  /**
+   * Getter for HCatalog Server's URL.
+   * (This is where the event originates from.)
+   * @return HCatalog Server's URL (String).
+   */
+  public abstract String getServer();
+
+  /**
+   * Getter for the Kerberos principal of the HCatalog service.
+   * @return HCatalog Service Principal (String).
+   */
+  public abstract String getServicePrincipal();
+
+  /**
+   * Getter for the name of the Database on which the Metastore operation is done.
+   * @return Database-name (String).
+   */
+  public abstract String getDB();
+
+  /**
+   * Getter for the timestamp associated with the operation.
+   * @return Timestamp (Long - seconds since epoch).
+   */
+  public abstract Long   getTimestamp();
+
+  /**
+   * Class invariant. Checked after construction or deserialization.
+   */
+  public EventMessage checkValid() {
+    if (getServer() == null || getServicePrincipal() == null)
+      throw new IllegalStateException("Server-URL/Service-Principal shouldn't be null.");
+    if (getEventType() == null)
+      throw new IllegalStateException("Event-type unset.");
+    if (getDB() == null)
+      throw new IllegalArgumentException("DB-name unset.");
+
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java
new file mode 100644
index 0000000..c470097
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/InsertMessage.java
@@ -0,0 +1,75 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+/**
+ * HCat message sent when an insert is done to a table or partition.
+ */
+public abstract class InsertMessage extends EventMessage {
+
+  protected InsertMessage() {
+    super(EventType.INSERT);
+  }
+
+  /**
+   * Getter for the name of the table being insert into.
+   * @return Table-name (String).
+   */
+  public abstract String getTable();
+
+  public abstract String getTableType();
+
+  /**
+   * Getter for the replace flag being insert into/overwrite
+   * @return Replace flag to represent INSERT INTO or INSERT OVERWRITE (Boolean).
+   */
+  public abstract boolean isReplace();
+
+  /**
+   * Get list of file name and checksum created as a result of this DML operation
+   *
+   * @return The iterable of files
+   */
+  public abstract Iterable<String> getFiles();
+
+  /**
+   * Get the table object associated with the insert
+   *
+   * @return The Json format of Table object
+   */
+  public abstract Table getTableObj() throws Exception;
+
+  /**
+   * Get the partition object associated with the insert
+   *
+   * @return The Json format of Partition object if the table is partitioned else return null.
+   */
+  public abstract Partition getPtnObj() throws Exception;
+
+  @Override
+  public EventMessage checkValid() {
+    if (getTable() == null)
+      throw new IllegalStateException("Table name unset.");
+    return super.checkValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
new file mode 100644
index 0000000..810dc64
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
@@ -0,0 +1,179 @@
+/*
+ * 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.messaging;
+
+/**
+ * Interface for converting HCat events from String-form back to EventMessage instances.
+ */
+public abstract class MessageDeserializer {
+
+  /**
+   * Method to construct EventMessage from string.
+   */
+  public EventMessage getEventMessage(String eventTypeString, String messageBody) {
+
+    switch (EventMessage.EventType.valueOf(eventTypeString)) {
+    case CREATE_DATABASE:
+      return getCreateDatabaseMessage(messageBody);
+    case DROP_DATABASE:
+      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);
+    case CREATE_FUNCTION:
+      return getCreateFunctionMessage(messageBody);
+    case DROP_FUNCTION:
+      return getDropFunctionMessage(messageBody);
+    case CREATE_INDEX:
+      return getCreateIndexMessage(messageBody);
+    case DROP_INDEX:
+      return getDropIndexMessage(messageBody);
+    case ALTER_INDEX:
+      return getAlterIndexMessage(messageBody);
+    case INSERT:
+      return getInsertMessage(messageBody);
+    case ADD_PRIMARYKEY:
+      return getAddPrimaryKeyMessage(messageBody);
+    case ADD_FOREIGNKEY:
+      return getAddForeignKeyMessage(messageBody);
+    case ADD_UNIQUECONSTRAINT:
+      return getAddUniqueConstraintMessage(messageBody);
+    case ADD_NOTNULLCONSTRAINT:
+      return getAddNotNullConstraintMessage(messageBody);
+    case DROP_CONSTRAINT:
+      return getDropConstraintMessage(messageBody);
+    default:
+      throw new IllegalArgumentException("Unsupported event-type: " + eventTypeString);
+    }
+  }
+
+  /**
+   * Method to de-serialize CreateDatabaseMessage instance.
+   */
+  public abstract CreateDatabaseMessage getCreateDatabaseMessage(String messageBody);
+
+  /**
+   * Method to de-serialize DropDatabaseMessage instance.
+   */
+  public abstract DropDatabaseMessage getDropDatabaseMessage(String messageBody);
+
+  /**
+   * Method to de-serialize CreateTableMessage instance.
+   */
+  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);
+
+  /**
+   * Method to de-serialize AddPartitionMessage instance.
+   */
+  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);
+
+  /**
+   * Method to de-serialize CreateFunctionMessage instance.
+   */
+  public abstract CreateFunctionMessage getCreateFunctionMessage(String messageBody);
+
+  /**
+   * Method to de-serialize DropFunctionMessage instance.
+   */
+  public abstract DropFunctionMessage getDropFunctionMessage(String messageBody);
+
+  /**
+   * Method to de-serialize CreateIndexMessage instance.
+   */
+  public abstract CreateIndexMessage getCreateIndexMessage(String messageBody);
+
+  /**
+   * Method to de-serialize DropIndexMessage instance.
+   */
+  public abstract DropIndexMessage getDropIndexMessage(String messageBody);
+
+  /**
+   * Method to de-serialize AlterIndexMessage instance.
+   */
+  public abstract AlterIndexMessage getAlterIndexMessage(String messageBody);
+
+  /**
+   * Method to deserialize InsertMessage
+   * @param messageBody the message in serialized form
+   * @return message in object form
+   */
+  public abstract InsertMessage getInsertMessage(String messageBody);
+
+  /**
+   * Method to de-serialize AddPrimaryKeyMessage instance.
+   */
+  public abstract AddPrimaryKeyMessage getAddPrimaryKeyMessage(String messageBody);
+
+  /**
+   * Method to de-serialize AddForeignKeyMessage instance.
+   */
+  public abstract AddForeignKeyMessage getAddForeignKeyMessage(String messageBody);
+
+  /**
+   * Method to de-serialize AddUniqueConstraintMessage instance.
+   */
+  public abstract AddUniqueConstraintMessage getAddUniqueConstraintMessage(String messageBody);
+
+  /**
+   * Method to de-serialize AddNotNullConstraintMessage instance.
+   */
+  public abstract AddNotNullConstraintMessage getAddNotNullConstraintMessage(String messageBody);
+
+  /**
+   * Method to de-serialize DropConstraintMessage instance.
+   */
+  public abstract DropConstraintMessage getDropConstraintMessage(String messageBody);
+
+  // Protection against construction.
+  protected MessageDeserializer() {}
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
new file mode 100644
index 0000000..46fd336
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -0,0 +1,289 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Abstract Factory for the construction of HCatalog message instances.
+ */
+public abstract class MessageFactory {
+
+  // Common name constants for event messages
+  public static final String ADD_PARTITION_EVENT = "ADD_PARTITION";
+  public static final String ALTER_PARTITION_EVENT = "ALTER_PARTITION";
+  public static final String DROP_PARTITION_EVENT = "DROP_PARTITION";
+  public static final String CREATE_TABLE_EVENT = "CREATE_TABLE";
+  public static final String ALTER_TABLE_EVENT = "ALTER_TABLE";
+  public static final String DROP_TABLE_EVENT = "DROP_TABLE";
+  public static final String CREATE_DATABASE_EVENT = "CREATE_DATABASE";
+  public static final String DROP_DATABASE_EVENT = "DROP_DATABASE";
+  public static final String INSERT_EVENT = "INSERT";
+  public static final String CREATE_FUNCTION_EVENT = "CREATE_FUNCTION";
+  public static final String DROP_FUNCTION_EVENT = "DROP_FUNCTION";
+  public static final String CREATE_INDEX_EVENT = "CREATE_INDEX";
+  public static final String DROP_INDEX_EVENT = "DROP_INDEX";
+  public static final String ALTER_INDEX_EVENT = "ALTER_INDEX";
+  public static final String ADD_PRIMARYKEY_EVENT = "ADD_PRIMARYKEY";
+  public static final String ADD_FOREIGNKEY_EVENT = "ADD_FOREIGNKEY";
+  public static final String ADD_UNIQUECONSTRAINT_EVENT = "ADD_UNIQUECONSTRAINT";
+  public static final String ADD_NOTNULLCONSTRAINT_EVENT = "ADD_NOTNULLCONSTRAINT";
+  public static final String DROP_CONSTRAINT_EVENT = "DROP_CONSTRAINT";
+
+  private static MessageFactory instance = null;
+
+  protected static final Configuration conf = MetastoreConf.newMetastoreConf();
+  /*
+  // TODO MS-SPLIT I'm 99% certain we don't need this, as MetastoreConf.newMetastoreConf already
+  adds this resource.
+  static {
+    conf.addResource("hive-site.xml");
+  }
+  */
+
+  protected static final String MS_SERVER_URL = MetastoreConf.getVar(conf, ConfVars.THRIFT_URIS, "");
+  protected static final String MS_SERVICE_PRINCIPAL =
+      MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL, "");
+
+  /**
+   * Getter for MessageFactory instance.
+   */
+  public static MessageFactory getInstance() {
+    if (instance == null) {
+      instance =
+          getInstance(MetastoreConf.getVar(conf, ConfVars.EVENT_MESSAGE_FACTORY));
+    }
+    return instance;
+  }
+
+  private static MessageFactory getInstance(String className) {
+    try {
+      return JavaUtils.newInstance(JavaUtils.getClass(className, MessageFactory.class));
+    }
+    catch (MetaException e) {
+      throw new IllegalStateException("Could not construct MessageFactory implementation: ", e);
+    }
+  }
+
+  /**
+   * Getter for MessageDeserializer, corresponding to the specified format and version.
+   * @param format Serialization format for notifications.
+   * @param version Version of serialization format (currently ignored.)
+   * @return MessageDeserializer.
+   */
+  public static MessageDeserializer getDeserializer(String format,
+                            String version) {
+    return getInstance(MetastoreConf.getVar(conf, ConfVars.EVENT_MESSAGE_FACTORY)).getDeserializer();
+    // Note : The reason this method exists outside the no-arg getDeserializer method is in
+    // case there is a user-implemented MessageFactory that's used, and some the messages
+    // are in an older format and the rest in another. Then, what MessageFactory is default
+    // is irrelevant, we should always use the one that was used to create it to deserialize.
+    //
+    // There exist only 2 implementations of this - json and jms
+    //
+    // Additional note : rather than as a config parameter, does it make sense to have
+    // this use jdbc-like semantics that each MessageFactory made available register
+    // itself for discoverability? Might be worth pursuing.
+  }
+
+  public abstract MessageDeserializer getDeserializer();
+
+  /**
+   * Getter for message-format.
+   */
+  public abstract String getMessageFormat();
+
+  /**
+   * Factory method for CreateDatabaseMessage.
+   * @param db The Database being added.
+   * @return CreateDatabaseMessage instance.
+   */
+  public abstract CreateDatabaseMessage buildCreateDatabaseMessage(Database db);
+
+  /**
+   * Factory method for DropDatabaseMessage.
+   * @param db The Database being dropped.
+   * @return DropDatabaseMessage instance.
+   */
+  public abstract DropDatabaseMessage buildDropDatabaseMessage(Database db);
+
+  /**
+   * Factory method for CreateTableMessage.
+   * @param table The Table being created.
+   * @param files Iterator of files
+   * @return CreateTableMessage instance.
+   */
+  public abstract CreateTableMessage buildCreateTableMessage(Table table, Iterator<String> files);
+
+  /**
+   * 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
+   * @param isTruncateOp Flag to denote truncate table
+   * @return
+   */
+  public abstract AlterTableMessage buildAlterTableMessage(Table before, Table after, boolean isTruncateOp);
+
+  /**
+   * Factory method for DropTableMessage.
+   * @param table The Table being dropped.
+   * @return DropTableMessage instance.
+   */
+  public abstract DropTableMessage buildDropTableMessage(Table table);
+
+    /**
+     * Factory method for AddPartitionMessage.
+     * @param table The Table to which the partitions are added.
+     * @param partitions The iterator to set of Partitions being added.
+     * @param partitionFiles The iterator of partition files
+     * @return AddPartitionMessage instance.
+     */
+  public abstract AddPartitionMessage buildAddPartitionMessage(Table table, Iterator<Partition> partitions,
+      Iterator<PartitionFiles> partitionFiles);
+
+  /**
+   * Factory method for building AlterPartitionMessage
+   * @param table The table in which the partition is being altered
+   * @param before The partition before it was altered
+   * @param after The partition after it was altered
+   * @param isTruncateOp Flag to denote truncate partition
+   * @return a new AlterPartitionMessage
+   */
+  public abstract AlterPartitionMessage buildAlterPartitionMessage(Table table, Partition before,
+                                                                   Partition after, boolean isTruncateOp);
+
+  /**
+   * Factory method for DropPartitionMessage.
+   * @param table The Table from which the partition is dropped.
+   * @param partitions The set of partitions being dropped.
+   * @return DropPartitionMessage instance.
+   */
+  public abstract DropPartitionMessage buildDropPartitionMessage(Table table, Iterator<Partition> partitions);
+
+  /**
+   * Factory method for CreateFunctionMessage.
+   * @param fn The Function being added.
+   * @return CreateFunctionMessage instance.
+   */
+  public abstract CreateFunctionMessage buildCreateFunctionMessage(Function fn);
+
+  /**
+   * Factory method for DropFunctionMessage.
+   * @param fn The Function being dropped.
+   * @return DropFunctionMessage instance.
+   */
+  public abstract DropFunctionMessage buildDropFunctionMessage(Function fn);
+
+  /**
+   * Factory method for CreateIndexMessage.
+   * @param idx The Index being added.
+   * @return CreateIndexMessage instance.
+   */
+  public abstract CreateIndexMessage buildCreateIndexMessage(Index idx);
+
+  /**
+   * Factory method for DropIndexMessage.
+   * @param idx The Index being dropped.
+   * @return DropIndexMessage instance.
+   */
+  public abstract DropIndexMessage buildDropIndexMessage(Index idx);
+
+  /**
+   * Factory method for AlterIndexMessage.
+   * @param before The index before the alter
+   * @param after The index after the alter
+   * @return AlterIndexMessage
+   */
+  public abstract AlterIndexMessage buildAlterIndexMessage(Index before, Index after);
+
+  /**
+   * Factory method for building insert message
+   *
+   * @param tableObj Table object where the insert occurred in
+   * @param ptnObj Partition object where the insert occurred in, may be null if
+   *          the insert was done into a non-partitioned table
+   * @param replace Flag to represent if INSERT OVERWRITE or INSERT INTO
+   * @param files Iterator of file created
+   * @return instance of InsertMessage
+   */
+  public abstract InsertMessage buildInsertMessage(Table tableObj, Partition ptnObj,
+                                                   boolean replace, Iterator<String> files);
+
+  /***
+   * Factory method for building add primary key message
+   *
+   * @param pks list of primary keys
+   * @return instance of AddPrimaryKeyMessage
+   */
+  public abstract AddPrimaryKeyMessage buildAddPrimaryKeyMessage(List<SQLPrimaryKey> pks);
+
+  /***
+   * Factory method for building add foreign key message
+   *
+   * @param fks list of foreign keys
+   * @return instance of AddForeignKeyMessage
+   */
+  public abstract AddForeignKeyMessage buildAddForeignKeyMessage(List<SQLForeignKey> fks);
+
+  /***
+   * Factory method for building add unique constraint message
+   *
+   * @param uks list of unique constraints
+   * @return instance of SQLUniqueConstraint
+   */
+  public abstract AddUniqueConstraintMessage buildAddUniqueConstraintMessage(List<SQLUniqueConstraint> uks);
+
+  /***
+   * Factory method for building add not null constraint message
+   *
+   * @param nns list of not null constraints
+   * @return instance of SQLNotNullConstraint
+   */
+  public abstract AddNotNullConstraintMessage buildAddNotNullConstraintMessage(List<SQLNotNullConstraint> nns);
+
+  /***
+   * Factory method for building drop constraint message
+   * @param dbName
+   * @param tableName
+   * @param constraintName
+   * @return
+   */
+  public abstract DropConstraintMessage buildDropConstraintMessage(String dbName, String tableName,
+      String constraintName);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java
new file mode 100644
index 0000000..308eaf1
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/PartitionFiles.java
@@ -0,0 +1,53 @@
+/*
+ * 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.messaging;
+
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class PartitionFiles {
+
+  @JsonProperty
+  private String partitionName;
+  @JsonProperty
+  private List<String> files;
+
+  public PartitionFiles(String partitionName, Iterator<String> files) {
+    this.partitionName = partitionName;
+    this.files = Lists.newArrayList(files);
+  }
+
+  public PartitionFiles() {
+  }
+
+  public String getPartitionName() {
+    return partitionName;
+  }
+
+  public void setPartitionName(String partitionName) {
+    this.partitionName = partitionName;
+  }
+
+  public Iterable<String> getFiles() {
+    return files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java
new file mode 100644
index 0000000..9c912bf
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddForeignKeyMessage.java
@@ -0,0 +1,101 @@
+/*
+ * 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.messaging.json;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON implementation of AddForeignKeyMessage
+ */
+public class JSONAddForeignKeyMessage extends AddForeignKeyMessage {
+
+  @JsonProperty
+  String server, servicePrincipal;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<String> foreignKeyListJson;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAddForeignKeyMessage() {
+  }
+
+  public JSONAddForeignKeyMessage(String server, String servicePrincipal, List<SQLForeignKey> fks,
+      Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.timestamp = timestamp;
+    this.foreignKeyListJson = new ArrayList<>();
+    try {
+      for (SQLForeignKey pk : fks) {
+        foreignKeyListJson.add(JSONMessageFactory.createForeignKeyObjJson(pk));
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys() throws Exception {
+    List<SQLForeignKey> fks = new ArrayList<>();
+    for (String pkJson : foreignKeyListJson) {
+      fks.add((SQLForeignKey)JSONMessageFactory.getTObj(pkJson, SQLForeignKey.class));
+    }
+    return fks;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java
new file mode 100644
index 0000000..79312e4
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddNotNullConstraintMessage.java
@@ -0,0 +1,96 @@
+/*
+ * 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.messaging.json;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class JSONAddNotNullConstraintMessage extends AddNotNullConstraintMessage {
+  @JsonProperty
+  String server, servicePrincipal;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<String> notNullConstraintListJson;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAddNotNullConstraintMessage() {
+  }
+
+  public JSONAddNotNullConstraintMessage(String server, String servicePrincipal, List<SQLNotNullConstraint> nns,
+      Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.timestamp = timestamp;
+    this.notNullConstraintListJson = new ArrayList<>();
+    try {
+      for (SQLNotNullConstraint nn : nns) {
+        notNullConstraintListJson.add(JSONMessageFactory.createNotNullConstraintObjJson(nn));
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints() throws Exception {
+    List<SQLNotNullConstraint> nns = new ArrayList<>();
+    for (String nnJson : notNullConstraintListJson) {
+      nns.add((SQLNotNullConstraint)JSONMessageFactory.getTObj(nnJson, SQLNotNullConstraint.class));
+    }
+    return nns;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java
new file mode 100644
index 0000000..c232cce
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPartitionMessage.java
@@ -0,0 +1,171 @@
+/*
+ * 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.messaging.json;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * JSON implementation of AddPartitionMessage.
+ */
+public class JSONAddPartitionMessage extends AddPartitionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<Map<String, String>> partitions;
+
+  @JsonProperty
+  List<String> partitionListJson;
+
+  @JsonProperty
+  List<PartitionFiles> partitionFiles;
+
+  /**
+   * Default Constructor. Required for Jackson.
+   */
+  public JSONAddPartitionMessage() {
+  }
+
+  /**
+   * Note that we get an Iterator rather than an Iterable here: so we can only walk thru the list once
+   */
+  public JSONAddPartitionMessage(String server, String servicePrincipal, Table tableObj,
+      Iterator<Partition> partitionsIterator, Iterator<PartitionFiles> partitionFileIter,
+      Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = tableObj.getDbName();
+    this.table = tableObj.getTableName();
+    this.tableType = tableObj.getTableType();
+    this.timestamp = timestamp;
+    partitions = new ArrayList<>();
+    partitionListJson = new ArrayList<>();
+    Partition partitionObj;
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
+      while (partitionsIterator.hasNext()) {
+        partitionObj = partitionsIterator.next();
+        partitions.add(JSONMessageFactory.getPartitionKeyValues(tableObj, partitionObj));
+        partitionListJson.add(JSONMessageFactory.createPartitionObjJson(partitionObj));
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+    this.partitionFiles = Lists.newArrayList(partitionFileIter);
+    checkValid();
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return db;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public String getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public List<Map<String, String>> getPartitions() {
+    return partitions;
+  }
+
+  @Override
+  public Iterable<Partition> getPartitionObjs() throws Exception {
+    // glorified cast from Iterable<TBase> to Iterable<Partition>
+    return Iterables.transform(
+        JSONMessageFactory.getTObjs(partitionListJson,Partition.class),
+        new Function<Object, Partition>() {
+      @Nullable
+      @Override
+      public Partition apply(@Nullable Object input) {
+        return (Partition) input;
+      }
+    });
+  }
+
+  public String getTableObjJson() {
+    return tableObjJson;
+  }
+
+  public List<String> getPartitionListJson() {
+    return partitionListJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+
+  @Override
+  public Iterable<PartitionFiles> getPartitionFilesIter() {
+    return partitionFiles;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java
new file mode 100644
index 0000000..f4f4d5d
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddPrimaryKeyMessage.java
@@ -0,0 +1,101 @@
+/*
+ * 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.messaging.json;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.messaging.AddPrimaryKeyMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON implementation of AddPrimaryKeyMessage
+ */
+public class JSONAddPrimaryKeyMessage extends AddPrimaryKeyMessage {
+
+  @JsonProperty
+  String server, servicePrincipal;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<String> primaryKeyListJson;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAddPrimaryKeyMessage() {
+  }
+
+  public JSONAddPrimaryKeyMessage(String server, String servicePrincipal, List<SQLPrimaryKey> pks,
+      Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.timestamp = timestamp;
+    this.primaryKeyListJson = new ArrayList<>();
+    try {
+      for (SQLPrimaryKey pk : pks) {
+        primaryKeyListJson.add(JSONMessageFactory.createPrimaryKeyObjJson(pk));
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys() throws Exception {
+    List<SQLPrimaryKey> pks = new ArrayList<>();
+    for (String pkJson : primaryKeyListJson) {
+      pks.add((SQLPrimaryKey)JSONMessageFactory.getTObj(pkJson, SQLPrimaryKey.class));
+    }
+    return pks;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java
new file mode 100644
index 0000000..5a3cf96
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAddUniqueConstraintMessage.java
@@ -0,0 +1,98 @@
+/*
+ * 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.messaging.json;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class JSONAddUniqueConstraintMessage extends AddUniqueConstraintMessage {
+  @JsonProperty
+  String server, servicePrincipal;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<String> uniqueConstraintListJson;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAddUniqueConstraintMessage() {
+  }
+
+  public JSONAddUniqueConstraintMessage(String server, String servicePrincipal, List<SQLUniqueConstraint> uks,
+      Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.timestamp = timestamp;
+    this.uniqueConstraintListJson = new ArrayList<>();
+    try {
+      for (SQLUniqueConstraint uk : uks) {
+        uniqueConstraintListJson.add(JSONMessageFactory.createUniqueConstraintObjJson(uk));
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints() throws Exception {
+    List<SQLUniqueConstraint> uks = new ArrayList<>();
+    for (String pkJson : uniqueConstraintListJson) {
+      uks.add((SQLUniqueConstraint)JSONMessageFactory.getTObj(pkJson, SQLUniqueConstraint.class));
+    }
+    return uks;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java
new file mode 100644
index 0000000..d1657b4
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterIndexMessage.java
@@ -0,0 +1,98 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.messaging.AlterIndexMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of AlterIndexMessage.
+ */
+public class JSONAlterIndexMessage extends AlterIndexMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, beforeIndexObjJson, afterIndexObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONAlterIndexMessage() {}
+
+  public JSONAlterIndexMessage(String server, String servicePrincipal, Index before, Index after,
+                               Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = after.getDbName();
+    this.timestamp = timestamp;
+    try {
+      this.beforeIndexObjJson = JSONMessageFactory.createIndexObjJson(before);
+      this.afterIndexObjJson = JSONMessageFactory.createIndexObjJson(after);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Index object", ex);
+    }
+
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getBeforeIndexObjJson() {
+    return beforeIndexObjJson;
+  }
+
+  public String getAfterIndexObjJson() {
+    return afterIndexObjJson;
+  }
+
+  @Override
+  public Index getIndexObjBefore() throws Exception {
+    return (Index)  JSONMessageFactory.getTObj(beforeIndexObjJson, Index.class);
+  }
+
+  @Override
+  public Index getIndexObjAfter() throws Exception {
+    return (Index)  JSONMessageFactory.getTObj(afterIndexObjJson, Index.class);
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
new file mode 100644
index 0000000..68cbd95
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterPartitionMessage.java
@@ -0,0 +1,148 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.util.Map;
+
+/**
+ * JSON alter table message
+ */
+public class JSONAlterPartitionMessage extends AlterPartitionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjJson;
+
+  @JsonProperty
+  String isTruncateOp;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  Map<String, String> keyValues;
+
+  @JsonProperty
+  String partitionObjBeforeJson, partitionObjAfterJson;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAlterPartitionMessage() {
+  }
+
+  public JSONAlterPartitionMessage(String server, String servicePrincipal, Table tableObj,
+      Partition partitionObjBefore, Partition partitionObjAfter, boolean isTruncateOp, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = tableObj.getDbName();
+    this.table = tableObj.getTableName();
+    this.tableType = tableObj.getTableType();
+    this.isTruncateOp = Boolean.toString(isTruncateOp);
+    this.timestamp = timestamp;
+    this.keyValues = JSONMessageFactory.getPartitionKeyValues(tableObj, partitionObjBefore);
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
+      this.partitionObjBeforeJson = JSONMessageFactory.createPartitionObjJson(partitionObjBefore);
+      this.partitionObjAfterJson = JSONMessageFactory.createPartitionObjJson(partitionObjAfter);
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+    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 getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public boolean getIsTruncateOp() { return Boolean.parseBoolean(isTruncateOp); }
+
+  @Override
+  public Map<String, String> getKeyValues() {
+    return keyValues;
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
+  }
+
+  @Override
+  public Partition getPtnObjBefore() throws Exception {
+    return (Partition) JSONMessageFactory.getTObj(partitionObjBeforeJson, Partition.class);
+  }
+
+  @Override
+  public Partition getPtnObjAfter() throws Exception {
+    return (Partition) JSONMessageFactory.getTObj(partitionObjAfterJson, Partition.class);
+  }
+
+  public String getTableObjJson() {
+    return tableObjJson;
+  }
+
+  public String getPartitionObjBeforeJson() {
+    return partitionObjBeforeJson;
+  }
+
+  public String getPartitionObjAfterJson() {
+    return partitionObjAfterJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+}


[02/12] hive git commit: HIVE-17812 Move remaining classes that HiveMetaStore depends on. This closes #261. (Alan Gates, reviewed by Vihang Karajgaonkar)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
new file mode 100644
index 0000000..ce8c0fa
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAlterTableMessage.java
@@ -0,0 +1,123 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON alter table message
+ */
+public class JSONAlterTableMessage extends AlterTableMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjBeforeJson, tableObjAfterJson;
+
+  @JsonProperty
+  String isTruncateOp;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAlterTableMessage() {
+  }
+
+  public JSONAlterTableMessage(String server, String servicePrincipal, Table tableObjBefore, Table tableObjAfter,
+      boolean isTruncateOp, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = tableObjBefore.getDbName();
+    this.table = tableObjBefore.getTableName();
+    this.tableType = tableObjBefore.getTableType();
+    this.isTruncateOp = Boolean.toString(isTruncateOp);
+    this.timestamp = timestamp;
+    try {
+      this.tableObjBeforeJson = JSONMessageFactory.createTableObjJson(tableObjBefore);
+      this.tableObjAfterJson = JSONMessageFactory.createTableObjJson(tableObjAfter);
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+    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 getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public boolean getIsTruncateOp() { return Boolean.parseBoolean(isTruncateOp); }
+
+  @Override
+  public Table getTableObjBefore() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjBeforeJson,Table.class);
+  }
+
+  @Override
+  public Table getTableObjAfter() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjAfterJson,Table.class);
+  }
+
+  public String getTableObjBeforeJson() {
+    return tableObjBeforeJson;
+  }
+
+  public String getTableObjAfterJson() {
+    return tableObjAfterJson ;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java
new file mode 100644
index 0000000..f442e99
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateDatabaseMessage.java
@@ -0,0 +1,71 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateDatabaseMessage.
+ */
+public class JSONCreateDatabaseMessage extends CreateDatabaseMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONCreateDatabaseMessage() {}
+
+  public JSONCreateDatabaseMessage(String server, String servicePrincipal, String db, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java
new file mode 100644
index 0000000..81f5d25
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateFunctionMessage.java
@@ -0,0 +1,86 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateFunctionMessage.
+ */
+public class JSONCreateFunctionMessage extends CreateFunctionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, functionObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONCreateFunctionMessage() {}
+
+  public JSONCreateFunctionMessage(String server, String servicePrincipal, Function fn, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = fn.getDbName();
+    this.timestamp = timestamp;
+    try {
+      this.functionObjJson = JSONMessageFactory.createFunctionObjJson(fn);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Function object", ex);
+    }
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getFunctionObjJson() {
+    return functionObjJson;
+  }
+
+  @Override
+  public Function getFunctionObj() throws Exception {
+    return (Function) JSONMessageFactory.getTObj(functionObjJson,Function.class);
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java
new file mode 100644
index 0000000..a40e524
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateIndexMessage.java
@@ -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.hadoop.hive.metastore.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.messaging.CreateIndexMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateIndexMessage.
+ */
+public class JSONCreateIndexMessage extends CreateIndexMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, indexObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONCreateIndexMessage() {}
+
+  public JSONCreateIndexMessage(String server, String servicePrincipal, Index index, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = index.getDbName();
+    try {
+      this.indexObjJson = JSONMessageFactory.createIndexObjJson(index);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Index object", ex);
+    }
+
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getIndexObjJson() {
+    return indexObjJson;
+  }
+
+  @Override
+  public Index getIndexObj() throws Exception {
+    return (Index)  JSONMessageFactory.getTObj(indexObjJson, Index.class);
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java
new file mode 100644
index 0000000..d894af9
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateTableMessage.java
@@ -0,0 +1,130 @@
+/*
+ * 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.messaging.json;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import com.google.common.collect.Lists;
+
+/**
+ * JSON implementation of CreateTableMessage.
+ */
+public class JSONCreateTableMessage extends CreateTableMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjJson;
+  @JsonProperty
+  Long timestamp;
+  @JsonProperty
+  List<String> files;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONCreateTableMessage() {
+  }
+
+  public JSONCreateTableMessage(String server, String servicePrincipal, String db, String table,
+      String tableType, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    this.table = table;
+    this.tableType = tableType;
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  public JSONCreateTableMessage(String server, String servicePrincipal, String db, String table,
+      Long timestamp) {
+    this(server, servicePrincipal, db, table, null, timestamp);
+  }
+
+  public JSONCreateTableMessage(String server, String servicePrincipal, Table tableObj,
+      Iterator<String> fileIter, Long timestamp) {
+    this(server, servicePrincipal, tableObj.getDbName(), tableObj.getTableName(),
+        tableObj.getTableType(), timestamp);
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+    this.files = Lists.newArrayList(fileIter);
+  }
+
+  @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 getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
+  }
+
+  public String getTableObjJson() {
+    return tableObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+
+  @Override
+  public Iterable<String> getFiles() {
+    return files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java
new file mode 100644
index 0000000..9e5d582
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropConstraintMessage.java
@@ -0,0 +1,90 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON implementation of DropConstraintMessage
+ */
+public class JSONDropConstraintMessage extends DropConstraintMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, dbName, tableName, constraintName;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONDropConstraintMessage() {
+  }
+
+  public JSONDropConstraintMessage(String server, String servicePrincipal, String dbName,
+      String tableName, String constraintName, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.timestamp = timestamp;
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.constraintName = constraintName;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return dbName;
+  }
+
+  @Override
+  public String getTable() {
+    return tableName;
+  }
+
+  @Override
+  public String getConstraint() {
+    return constraintName;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java
new file mode 100644
index 0000000..cd8bcdc
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropDatabaseMessage.java
@@ -0,0 +1,71 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON implementation of DropDatabaseMessage.
+ */
+public class JSONDropDatabaseMessage extends DropDatabaseMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONDropDatabaseMessage() {}
+
+  public JSONDropDatabaseMessage(String server, String servicePrincipal, String db, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    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 toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java
new file mode 100644
index 0000000..ca32d16
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropFunctionMessage.java
@@ -0,0 +1,78 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateDatabaseMessage.
+ */
+public class JSONDropFunctionMessage extends DropFunctionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, functionName;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONDropFunctionMessage() {}
+
+  public JSONDropFunctionMessage(String server, String servicePrincipal, Function fn, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = fn.getDbName();
+    this.functionName = fn.getFunctionName();
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+
+  @Override
+  public String getFunctionName() {
+    return functionName;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java
new file mode 100644
index 0000000..fb719c2
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropIndexMessage.java
@@ -0,0 +1,90 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.messaging.DropIndexMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of DropIndexMessage.
+ */
+public class JSONDropIndexMessage extends DropIndexMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, indexName, origTableName, indexTableName;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONDropIndexMessage() {}
+
+  public JSONDropIndexMessage(String server, String servicePrincipal, Index index, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = index.getDbName();
+    this.indexName = index.getIndexName();
+    this.origTableName = index.getOrigTableName();
+    this.indexTableName = index.getIndexTableName();
+
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  @Override
+  public String getIndexName() {
+    return indexName;
+  }
+
+  @Override
+  public String getOrigTableName() {
+    return origTableName;
+  }
+
+  @Override
+  public String getIndexTableName() {
+    return indexTableName;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java
new file mode 100644
index 0000000..4689711
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropPartitionMessage.java
@@ -0,0 +1,130 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * JSON implementation of DropPartitionMessage.
+ */
+public class JSONDropPartitionMessage extends DropPartitionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  List<Map<String, String>> partitions;
+
+  /**
+   * Default Constructor. Required for Jackson.
+   */
+  public JSONDropPartitionMessage() {
+  }
+
+  public JSONDropPartitionMessage(String server, String servicePrincipal, String db, String table,
+      List<Map<String, String>> partitions, Long timestamp) {
+    this(server, servicePrincipal, db, table,  null, partitions, timestamp);
+  }
+
+  public JSONDropPartitionMessage(String server, String servicePrincipal, String db, String table,
+      String tableType, List<Map<String, String>> partitions, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    this.table = table;
+    this.tableType = tableType;
+    this.partitions = partitions;
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  public JSONDropPartitionMessage(String server, String servicePrincipal, Table tableObj,
+      List<Map<String, String>> partitionKeyValues, long timestamp) {
+    this(server, servicePrincipal, tableObj.getDbName(), tableObj.getTableName(),
+        tableObj.getTableType(), partitionKeyValues, timestamp);
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return db;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public String getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public List<Map<String, String>> getPartitions() {
+    return partitions;
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjJson, Table.class);
+  }
+
+  public String getTableObjJson() {
+    return tableObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java
new file mode 100644
index 0000000..591c9dd
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropTableMessage.java
@@ -0,0 +1,116 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.DropTableMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON implementation of DropTableMessage.
+ */
+public class JSONDropTableMessage extends DropTableMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONDropTableMessage() {
+  }
+
+  public JSONDropTableMessage(String server, String servicePrincipal, String db, String table,
+      Long timestamp) {
+    this(server, servicePrincipal, db, table, null, timestamp);
+  }
+
+  public JSONDropTableMessage(String server, String servicePrincipal, String db, String table,
+      String tableType, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = db;
+    this.table = table;
+    this.tableType = tableType;
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  public JSONDropTableMessage(String server, String servicePrincipal, Table tableObj,
+      Long timestamp) {
+    this(server, servicePrincipal, tableObj.getDbName(), tableObj.getTableName(),
+        tableObj.getTableType(), timestamp);
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+    checkValid();
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public String getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
+  }
+
+  @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 toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
new file mode 100644
index 0000000..39372bd
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
@@ -0,0 +1,144 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import com.google.common.collect.Lists;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * JSON implementation of InsertMessage
+ */
+public class JSONInsertMessage extends InsertMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, table, tableType, tableObjJson, ptnObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  @JsonProperty
+  String replace;
+
+  @JsonProperty
+  List<String> files;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONInsertMessage() {
+  }
+
+  public JSONInsertMessage(String server, String servicePrincipal, Table tableObj, Partition ptnObj,
+                           boolean replace, Iterator<String> fileIter, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+
+    if (null == tableObj) {
+      throw new IllegalArgumentException("Table not valid.");
+    }
+
+    this.db = tableObj.getDbName();
+    this.table = tableObj.getTableName();
+    this.tableType = tableObj.getTableType();
+
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(tableObj);
+      if (null != ptnObj) {
+        this.ptnObjJson = JSONMessageFactory.createPartitionObjJson(ptnObj);
+      } else {
+        this.ptnObjJson = null;
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize: ", e);
+    }
+
+    this.timestamp = timestamp;
+    this.replace = Boolean.toString(replace);
+    this.files = Lists.newArrayList(fileIter);
+
+    checkValid();
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public String getTableType() {
+    if (tableType != null) return tableType; else return "";
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public Iterable<String> getFiles() {
+    return files;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return db;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public boolean isReplace() { return Boolean.parseBoolean(replace); }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (Table) JSONMessageFactory.getTObj(tableObjJson,Table.class);
+  }
+
+  @Override
+  public Partition getPtnObj() throws Exception {
+    return ((null == ptnObjJson) ? null : (Partition) JSONMessageFactory.getTObj(ptnObjJson, Partition.class));
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
new file mode 100644
index 0000000..15fa4aa
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
@@ -0,0 +1,243 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddPrimaryKeyMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
+import org.apache.hadoop.hive.metastore.messaging.AlterIndexMessage;
+import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateIndexMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropIndexMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
+import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+
+/**
+ * MessageDeserializer implementation, for deserializing from JSON strings.
+ */
+public class JSONMessageDeserializer extends MessageDeserializer {
+
+  static ObjectMapper mapper = new ObjectMapper(); // Thread-safe.
+
+  static {
+    mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, false);
+    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, false);
+    mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, false);
+  }
+
+  @Override
+  public CreateDatabaseMessage getCreateDatabaseMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONCreateDatabaseMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONCreateDatabaseMessage.", exception);
+    }
+  }
+
+  @Override
+  public DropDatabaseMessage getDropDatabaseMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONDropDatabaseMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONDropDatabaseMessage.", exception);
+    }
+  }
+
+  @Override
+  public CreateTableMessage getCreateTableMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONCreateTableMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONCreateTableMessage.", exception);
+    }
+  }
+
+  @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);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONDropTableMessage.", exception);
+    }
+  }
+
+  @Override
+  public AddPartitionMessage getAddPartitionMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAddPartitionMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct AddPartitionMessage.", exception);
+    }
+  }
+
+  @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 {
+      return mapper.readValue(messageBody, JSONDropPartitionMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct DropPartitionMessage.", exception);
+    }
+  }
+
+  @Override
+  public CreateFunctionMessage getCreateFunctionMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONCreateFunctionMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONCreateFunctionMessage.", exception);
+    }
+  }
+
+  @Override
+  public DropFunctionMessage getDropFunctionMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONDropFunctionMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONDropDatabaseMessage.", exception);
+    }
+  }
+
+  @Override
+  public CreateIndexMessage getCreateIndexMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONCreateIndexMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONCreateIndexMessage.", exception);
+    }
+  }
+
+  @Override
+  public DropIndexMessage getDropIndexMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONDropIndexMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONDropIndexMessage.", exception);
+    }
+  }
+
+  @Override
+  public AlterIndexMessage getAlterIndexMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAlterIndexMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONAlterIndexMessage.", exception);
+    }
+  }
+
+  @Override
+  public InsertMessage getInsertMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONInsertMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct InsertMessage", e);
+    }
+  }
+
+  @Override
+  public AddPrimaryKeyMessage getAddPrimaryKeyMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAddPrimaryKeyMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AddPrimaryKeyMessage", e);
+    }
+  }
+
+  @Override
+  public AddForeignKeyMessage getAddForeignKeyMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAddForeignKeyMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AddForeignKeyMessage", e);
+    }
+  }
+
+  @Override
+  public AddUniqueConstraintMessage getAddUniqueConstraintMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAddUniqueConstraintMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AddUniqueConstraintMessage", e);
+    }
+  }
+
+  @Override
+  public AddNotNullConstraintMessage getAddNotNullConstraintMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAddNotNullConstraintMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AddNotNullConstraintMessage", e);
+    }
+  }
+
+  @Override
+  public DropConstraintMessage getDropConstraintMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONDropConstraintMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct DropConstraintMessage", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
new file mode 100644
index 0000000..916a8e8
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -0,0 +1,354 @@
+/*
+ * 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.messaging.json;
+
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddPrimaryKeyMessage;
+import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
+import org.apache.hadoop.hive.metastore.messaging.AlterIndexMessage;
+import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateIndexMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropIndexMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
+import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
+import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
+import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ObjectNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+
+/**
+ * The JSON implementation of the MessageFactory. Constructs JSON implementations of each
+ * message-type.
+ */
+public class JSONMessageFactory extends MessageFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JSONMessageFactory.class.getName());
+
+  private static JSONMessageDeserializer deserializer = new JSONMessageDeserializer();
+
+  @Override
+  public MessageDeserializer getDeserializer() {
+    return deserializer;
+  }
+
+  @Override
+  public String getMessageFormat() {
+    return "json-0.2";
+  }
+
+  @Override
+  public CreateDatabaseMessage buildCreateDatabaseMessage(Database db) {
+    return new JSONCreateDatabaseMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, db.getName(), now());
+  }
+
+  @Override
+  public DropDatabaseMessage buildDropDatabaseMessage(Database db) {
+    return new JSONDropDatabaseMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, db.getName(), now());
+  }
+
+  @Override
+  public CreateTableMessage buildCreateTableMessage(Table table, Iterator<String> fileIter) {
+    return new JSONCreateTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table, fileIter, now());
+  }
+
+  @Override
+  public AlterTableMessage buildAlterTableMessage(Table before, Table after, boolean isTruncateOp) {
+    return new JSONAlterTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, before, after, isTruncateOp, now());
+  }
+
+  @Override
+  public DropTableMessage buildDropTableMessage(Table table) {
+    return new JSONDropTableMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table, now());
+  }
+
+  @Override
+  public AddPartitionMessage buildAddPartitionMessage(Table table,
+      Iterator<Partition> partitionsIterator, Iterator<PartitionFiles> partitionFileIter) {
+    return new JSONAddPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table,
+        partitionsIterator, partitionFileIter, now());
+  }
+
+  @Override
+  public AlterPartitionMessage buildAlterPartitionMessage(Table table, Partition before,
+      Partition after, boolean isTruncateOp) {
+    return new JSONAlterPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table, before, after, isTruncateOp,
+        now());
+  }
+
+  @Override
+  public DropPartitionMessage buildDropPartitionMessage(Table table,
+      Iterator<Partition> partitionsIterator) {
+    return new JSONDropPartitionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, table,
+        getPartitionKeyValues(table, partitionsIterator), now());
+  }
+
+  @Override
+  public CreateFunctionMessage buildCreateFunctionMessage(Function fn) {
+    return new JSONCreateFunctionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, fn, now());
+  }
+
+  @Override
+  public DropFunctionMessage buildDropFunctionMessage(Function fn) {
+    return new JSONDropFunctionMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, fn, now());
+  }
+
+  @Override
+  public CreateIndexMessage buildCreateIndexMessage(Index idx) {
+    return new JSONCreateIndexMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, idx, now());
+  }
+
+  @Override
+  public DropIndexMessage buildDropIndexMessage(Index idx) {
+    return new JSONDropIndexMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, idx, now());
+  }
+
+  @Override
+  public AlterIndexMessage buildAlterIndexMessage(Index before, Index after) {
+    return new JSONAlterIndexMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, before, after, now());
+  }
+
+  @Override
+  public InsertMessage buildInsertMessage(Table tableObj, Partition partObj,
+                                          boolean replace, Iterator<String> fileIter) {
+    return new JSONInsertMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, tableObj, partObj, replace, fileIter, now());
+  }
+
+  @Override
+  public AddPrimaryKeyMessage buildAddPrimaryKeyMessage(List<SQLPrimaryKey> pks) {
+    return new JSONAddPrimaryKeyMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, pks, now());
+  }
+
+  @Override
+  public AddForeignKeyMessage buildAddForeignKeyMessage(List<SQLForeignKey> fks) {
+    return new JSONAddForeignKeyMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, fks, now());
+  }
+
+  @Override
+  public AddUniqueConstraintMessage buildAddUniqueConstraintMessage(List<SQLUniqueConstraint> uks) {
+    return new JSONAddUniqueConstraintMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, uks, now());
+  }
+
+  @Override
+  public AddNotNullConstraintMessage buildAddNotNullConstraintMessage(List<SQLNotNullConstraint> nns) {
+    return new JSONAddNotNullConstraintMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, nns, now());
+  }
+
+  @Override
+  public DropConstraintMessage buildDropConstraintMessage(String dbName, String tableName,
+      String constraintName) {
+    return new JSONDropConstraintMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, dbName, tableName,
+        constraintName, now());
+  }
+
+  private long now() {
+    return System.currentTimeMillis() / 1000;
+  }
+
+  static Map<String, String> getPartitionKeyValues(Table table, Partition partition) {
+    Map<String, String> partitionKeys = new LinkedHashMap<>();
+    for (int i = 0; i < table.getPartitionKeysSize(); ++i)
+      partitionKeys.put(table.getPartitionKeys().get(i).getName(), partition.getValues().get(i));
+    return partitionKeys;
+  }
+
+  static List<Map<String, String>> getPartitionKeyValues(final Table table,
+      Iterator<Partition> iterator) {
+    return Lists.newArrayList(Iterators.transform(iterator,
+        new com.google.common.base.Function<Partition, Map<String, String>>() {
+          @Override
+          public Map<String, String> apply(@Nullable Partition partition) {
+            return getPartitionKeyValues(table, partition);
+          }
+        }));
+  }
+
+  static String createPrimaryKeyObjJson(SQLPrimaryKey primaryKeyObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(primaryKeyObj, "UTF-8");
+  }
+
+  static String createForeignKeyObjJson(SQLForeignKey foreignKeyObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(foreignKeyObj, "UTF-8");
+  }
+
+  static String createUniqueConstraintObjJson(SQLUniqueConstraint uniqueConstraintObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(uniqueConstraintObj, "UTF-8");
+  }
+
+  static String createNotNullConstraintObjJson(SQLNotNullConstraint notNullConstaintObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(notNullConstaintObj, "UTF-8");
+  }
+
+  static String createTableObjJson(Table tableObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(tableObj, "UTF-8");
+  }
+
+  static String createPartitionObjJson(Partition partitionObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(partitionObj, "UTF-8");
+  }
+
+  static String createFunctionObjJson(Function functionObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(functionObj, "UTF-8");
+  }
+
+  static String createIndexObjJson(Index indexObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(indexObj, "UTF-8");
+  }
+
+  public static ObjectNode getJsonTree(NotificationEvent event) throws Exception {
+    return getJsonTree(event.getMessage());
+  }
+
+  public static ObjectNode getJsonTree(String eventMessage) throws Exception {
+    JsonParser jsonParser = (new JsonFactory()).createJsonParser(eventMessage);
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(jsonParser, ObjectNode.class);
+  }
+
+  public static Table getTableObj(ObjectNode jsonTree) throws Exception {
+    TDeserializer deSerializer = new TDeserializer(new TJSONProtocol.Factory());
+    Table tableObj = new Table();
+    String tableJson = jsonTree.get("tableObjJson").asText();
+    deSerializer.deserialize(tableObj, tableJson, "UTF-8");
+    return tableObj;
+  }
+
+  /*
+   * TODO: Some thoughts here : We have a current todo to move some of these methods over to
+   * MessageFactory instead of being here, so we can override them, but before we move them over,
+   * we should keep the following in mind:
+   *
+   * a) We should return Iterables, not Lists. That makes sure that we can be memory-safe when
+   * implementing it rather than forcing ourselves down a path wherein returning List is part of
+   * our interface, and then people use .size() or somesuch which makes us need to materialize
+   * the entire list and not change. Also, returning Iterables allows us to do things like
+   * Iterables.transform for some of these.
+   * b) We should not have "magic" names like "tableObjJson", because that breaks expectation of a
+   * couple of things - firstly, that of serialization format, although that is fine for this
+   * JSONMessageFactory, and secondly, that makes us just have a number of mappings, one for each
+   * obj type, and sometimes, as the case is with alter, have multiples. Also, any event-specific
+   * item belongs in that event message / event itself, as opposed to in the factory. It's okay to
+   * have utility accessor methods here that are used by each of the messages to provide accessors.
+   * I'm adding a couple of those here.
+   *
+   */
+
+  public static TBase getTObj(String tSerialized, Class<? extends TBase> objClass) throws Exception{
+    TDeserializer thriftDeSerializer = new TDeserializer(new TJSONProtocol.Factory());
+    TBase obj = objClass.newInstance();
+    thriftDeSerializer.deserialize(obj, tSerialized, "UTF-8");
+    return obj;
+  }
+
+  public static Iterable<? extends TBase> getTObjs(
+      Iterable<String> objRefStrs, final Class<? extends TBase> objClass) throws Exception {
+
+    try {
+      return Iterables.transform(objRefStrs, new com.google.common.base.Function<String,TBase>(){
+        @Override
+        public TBase apply(@Nullable String objStr){
+          try {
+            return getTObj(objStr, objClass);
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+    } catch (RuntimeException re){
+      // We have to add this bit of exception handling here, because Function.apply does not allow us to throw
+      // the actual exception that might be a checked exception, so we wind up needing to throw a RuntimeException
+      // with the previously thrown exception as its cause. However, since RuntimeException.getCause() returns
+      // a throwable instead of an Exception, we have to account for the possibility that the underlying code
+      // might have thrown a Throwable that we wrapped instead, in which case, continuing to throw the
+      // RuntimeException is the best thing we can do.
+      Throwable t = re.getCause();
+      if (t instanceof Exception){
+        throw (Exception) t;
+      } else {
+        throw re;
+      }
+    }
+  }
+
+  // If we do not need this format of accessor using ObjectNode, this is a candidate for removal as well
+  public static Iterable<? extends TBase> getTObjs(
+      ObjectNode jsonTree, String objRefListName, final Class<? extends TBase> objClass) throws Exception {
+    Iterable<JsonNode> jsonArrayIterator = jsonTree.get(objRefListName);
+    com.google.common.base.Function<JsonNode,String> textExtractor =
+        new com.google.common.base.Function<JsonNode, String>() {
+      @Nullable
+      @Override
+      public String apply(@Nullable JsonNode input) {
+        return input.asText();
+      }
+    };
+    return getTObjs(Iterables.transform(jsonArrayIterator, textExtractor), objClass);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java
new file mode 100644
index 0000000..97bff0e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidCompactionHistoryService.java
@@ -0,0 +1,62 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.txn;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.RunnableConfigurable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Purges obsolete items from compaction history data
+ */
+public class AcidCompactionHistoryService implements RunnableConfigurable {
+  private static final Logger LOG = LoggerFactory.getLogger(AcidCompactionHistoryService.class);
+
+  private Configuration conf;
+  private TxnStore txnHandler;
+
+  @Override
+  public void setConf(Configuration configuration) {
+    this.conf = configuration;
+    txnHandler = TxnUtils.getTxnStore(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void run() {
+    TxnStore.MutexAPI.LockHandle handle = null;
+    try {
+      handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.CompactionHistory.name());
+      long startTime = System.currentTimeMillis();
+      txnHandler.purgeCompactionHistory();
+      LOG.debug("History reaper reaper ran for " + (System.currentTimeMillis() - startTime)/1000 +
+          "seconds.");
+    } catch(Throwable t) {
+      LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
+    } finally {
+      if(handle != null) {
+        handle.releaseLocks();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java
new file mode 100644
index 0000000..7450a2f
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java
@@ -0,0 +1,63 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.txn;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.RunnableConfigurable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Performs background tasks for Transaction management in Hive.
+ * Runs inside Hive Metastore Service.
+ */
+public class AcidHouseKeeperService implements RunnableConfigurable {
+  private static final Logger LOG = LoggerFactory.getLogger(AcidHouseKeeperService.class);
+
+  private Configuration conf;
+  private TxnStore txnHandler;
+
+  @Override
+  public void setConf(Configuration configuration) {
+    this.conf = configuration;
+    txnHandler = TxnUtils.getTxnStore(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void run() {
+    TxnStore.MutexAPI.LockHandle handle = null;
+    try {
+      handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.HouseKeeper.name());
+      long startTime = System.currentTimeMillis();
+      txnHandler.performTimeOuts();
+      LOG.debug("timeout reaper ran for " + (System.currentTimeMillis() - startTime)/1000 +
+          "seconds.");
+    } catch(Throwable t) {
+      LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
+    } finally {
+      if(handle != null) {
+        handle.releaseLocks();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidWriteSetService.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidWriteSetService.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidWriteSetService.java
new file mode 100644
index 0000000..413fe96
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidWriteSetService.java
@@ -0,0 +1,61 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.txn;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.RunnableConfigurable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Periodically cleans WriteSet tracking information used in Transaction management
+ */
+public class AcidWriteSetService implements RunnableConfigurable {
+  private static final Logger LOG = LoggerFactory.getLogger(AcidWriteSetService.class);
+
+  private Configuration conf;
+  private TxnStore txnHandler;
+
+  @Override
+  public void setConf(Configuration configuration) {
+    this.conf = configuration;
+    txnHandler = TxnUtils.getTxnStore(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void run() {
+    TxnStore.MutexAPI.LockHandle handle = null;
+    try {
+      handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.WriteSetCleaner.name());
+      long startTime = System.currentTimeMillis();
+      txnHandler.performWriteSetGC();
+      LOG.debug("cleaner ran for " + (System.currentTimeMillis() - startTime)/1000 + "seconds.");
+    } catch(Throwable t) {
+      LOG.error("Serious error in {}", Thread.currentThread().getName(), ": {}" + t.getMessage(), t);
+    } finally {
+      if(handle != null) {
+        handle.releaseLocks();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index d09c958..756cb4c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -172,6 +172,12 @@ public final class TxnDbUtil {
         LOG.error("Error rolling back: " + re.getMessage());
       }
 
+      // Another thread might have already created these tables.
+      if (e.getMessage() != null && e.getMessage().contains("already exists")) {
+        LOG.info("Txn tables already exist, returning");
+        return;
+      }
+
       // This might be a deadlock, if so, let's retry
       if (e instanceof SQLTransactionRollbackException && deadlockCnt++ < 5) {
         LOG.warn("Caught deadlock, retrying db creation");

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
index da0ee80..2dac899 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
@@ -363,4 +363,20 @@ public class FileUtils {
     }
     return name.toString();
   }
+
+  /**
+   * Determine if two objects reference the same file system.
+   * @param fs1 first file system
+   * @param fs2 second file system
+   * @return return true if both file system arguments point to same file system
+   */
+  public static boolean equalsFileSystem(FileSystem fs1, FileSystem fs2) {
+    //When file system cache is disabled, you get different FileSystem objects
+    // for same file system, so '==' can't be used in such cases
+    //FileSystem api doesn't have a .equals() function implemented, so using
+    //the uri for comparison. FileSystem already uses uri+Configuration for
+    //equality in its CACHE .
+    //Once equality has been added in HDFS-9159, we should make use of it
+    return fs1.getUri().equals(fs2.getUri());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
index c10e36f..ecbddc3 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/HdfsUtils.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -44,6 +45,10 @@ import java.util.Map;
 public class HdfsUtils {
   private static final Logger LOG = LoggerFactory.getLogger(HdfsUtils.class);
   private static final String DISTCP_OPTIONS_PREFIX = "distcp.options.";
+  // TODO: this relies on HDFS not changing the format; we assume if we could get inode ID, this
+  //       is still going to work. Otherwise, file IDs can be turned off. Later, we should use
+  //       as public utility method in HDFS to obtain the inode-based path.
+  private static final String HDFS_ID_PATH_PREFIX = "/.reserved/.inodes/";
 
   /**
    * Check the permissions on a file.
@@ -197,4 +202,20 @@ public class HdfsUtils {
     return params;
   }
 
+  public static Path getFileIdPath(
+      FileSystem fileSystem, Path path, long fileId) {
+    return (fileSystem instanceof DistributedFileSystem)
+        ? new Path(HDFS_ID_PATH_PREFIX + fileId) : path;
+  }
+
+  public static long getFileId(FileSystem fs, String path) throws IOException {
+    return ensureDfs(fs).getClient().getFileInfo(path).getFileId();
+  }
+
+  private static DistributedFileSystem ensureDfs(FileSystem fs) {
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new UnsupportedOperationException("Only supported for DFS; got " + fs.getClass());
+    }
+    return (DistributedFileSystem)fs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c5a9673a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java
index 593dee3..b08d9fd 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java
@@ -88,6 +88,20 @@ public class JavaUtils {
   }
 
   /**
+   * Create an object of the given class using a no-args constructor
+   * @param theClass class to return new object of
+   * @param <T> the type of the class to be returned
+   * @return an object of the requested type
+   */
+  public static <T> T newInstance(Class<T> theClass) {
+    try {
+      return theClass.newInstance();
+    } catch (InstantiationException|IllegalAccessException e) {
+      throw new RuntimeException("Unable to instantiate " + theClass.getName(), e);
+    }
+  }
+
+  /**
    * @return name of current host
    */
   public static String hostname() {