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

svn commit: r1401784 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ metastore/src/java/org/apache/hadoop/hive/metastore/ metastore/src/java/org/apache/hadoop/hive/metastore/events/ metastore/src/test/org/apache/hadoop/hive/metasto...

Author: kevinwilfong
Date: Wed Oct 24 17:05:06 2012
New Revision: 1401784

URL: http://svn.apache.org/viewvc?rev=1401784&view=rev
Log:
HIVE-3152. Disallow certain character patterns in partition names. (Ivan Gorbachev via kevinwilfong)

Added:
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionNameWhitelistPreEventListener.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistPreEventHook.java
    hive/trunk/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q
    hive/trunk/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
    hive/trunk/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q
    hive/trunk/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q
    hive/trunk/ql/src/test/results/clientnegative/add_partition_with_whitelist.q.out
    hive/trunk/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out
    hive/trunk/ql/src/test/results/clientpositive/add_partition_no_whitelist.q.out
    hive/trunk/ql/src/test/results/clientpositive/add_partition_with_whitelist.q.out
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Wed Oct 24 17:05:06 2012
@@ -320,6 +320,9 @@ public class HiveConf extends Configurat
     METASTORE_EVENT_CLEAN_FREQ("hive.metastore.event.clean.freq",0L),
     METASTORE_EVENT_EXPIRY_DURATION("hive.metastore.event.expiry.duration",0L),
     METASTORE_EXECUTE_SET_UGI("hive.metastore.execute.setugi", false),
+    METASTORE_PARTITION_NAME_WHITELIST_PATTERN(
+        "hive.metastore.partition.name.whitelist.pattern", ""),
+
 
     // Default parameters for creating tables
     NEWTABLEDEFAULTPARA("hive.table.parameters.default", ""),

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Wed Oct 24 17:05:06 2012
@@ -268,6 +268,12 @@
 </property>
 
 <property>
+  <name>hive.metastore.partition.name.whitelist.pattern</name>
+  <value></value>
+  <description>Partition names will be checked against this regex pattern and rejected if not matched.  To use, enable hive.metastore.pre.event.listeners=org.apache.hadoop.hive.metastore.PartitionNameWhitelistPreEventListener  Listener will not register if this property value is empty.</description>
+</property>
+
+<property>
   <name>hive.metastore.end.function.listeners</name>
   <value></value>
   <description>list of comma separated listeners for the end of metastore functions.</description>

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Wed Oct 24 17:05:06 2012
@@ -101,6 +101,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
 import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege;
@@ -175,27 +176,27 @@ public class HiveMetaStore extends Thrif
 
     private Warehouse wh; // hdfs warehouse
     private final ThreadLocal<RawStore> threadLocalMS =
-      new ThreadLocal<RawStore>() {
-        @Override
-        protected synchronized RawStore initialValue() {
-          return null;
-        }
-      };
+        new ThreadLocal<RawStore>() {
+          @Override
+          protected synchronized RawStore initialValue() {
+            return null;
+          }
+        };
 
     // Thread local configuration is needed as many threads could make changes
     // to the conf using the connection hook
     private final ThreadLocal<Configuration> threadLocalConf =
-      new ThreadLocal<Configuration>() {
-        @Override
-        protected synchronized Configuration initialValue() {
-          return null;
-        }
-      };
+        new ThreadLocal<Configuration>() {
+          @Override
+          protected synchronized Configuration initialValue() {
+            return null;
+          }
+        };
 
     public static final String AUDIT_FORMAT =
-      "ugi=%s\t" + // ugi
-      "ip=%s\t" + // remote IP
-      "cmd=%s\t"; // command
+        "ugi=%s\t" + // ugi
+            "ip=%s\t" + // remote IP
+            "cmd=%s\t"; // command
     public static final Log auditLog = LogFactory.getLog(
         HiveMetaStore.class.getName() + ".audit");
     private static final ThreadLocal<Formatter> auditFormatter =
@@ -231,7 +232,7 @@ public class HiveMetaStore extends Thrif
       }
 
       auditLog.info(fmt.format(AUDIT_FORMAT, ugi.getUserName(),
-        address, cmd).toString());
+          address, cmd).toString());
     }
 
     // The next serial number to be assigned
@@ -245,7 +246,7 @@ public class HiveMetaStore extends Thrif
     };
 
     // This will only be set if the metastore is being accessed from a metastore Thrift server,
-    // not if it is from the CLI.  Also, only if the TTransport being used to connect is an
+    // not if it is from the CLI. Also, only if the TTransport being used to connect is an
     // instance of TSocket.
     private static ThreadLocal<String> threadLocalIpAddress = new ThreadLocal<String>() {
       @Override
@@ -321,7 +322,8 @@ public class HiveMetaStore extends Thrif
         }
       }
 
-      preListeners = MetaStoreUtils.getMetaStoreListeners(MetaStorePreEventListener.class, hiveConf,
+      preListeners = MetaStoreUtils.getMetaStoreListeners(MetaStorePreEventListener.class,
+          hiveConf,
           hiveConf.getVar(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS));
       listeners = MetaStoreUtils.getMetaStoreListeners(MetaStoreEventListener.class, hiveConf,
           hiveConf.getVar(HiveConf.ConfVars.METASTORE_EVENT_LISTENERS));
@@ -531,16 +533,10 @@ public class HiveMetaStore extends Thrif
       Path dbPath = new Path(db.getLocationUri());
       boolean success = false;
       boolean madeDir = false;
+
       try {
-        try {
-          for (MetaStorePreEventListener listener : preListeners) {
-            listener.onEvent(new PreCreateDatabaseEvent(db, this));
-          }
-        } catch (NoSuchObjectException e) {
-          throw new MetaException(e.getMessage());
-        } catch (InvalidOperationException e) {
-          throw new MetaException(e.getMessage());
-        }
+
+        firePreEvent(new PreCreateDatabaseEvent(db, this));
 
         if (!wh.isDir(dbPath)) {
           if (!wh.mkdirs(dbPath)) {
@@ -629,9 +625,7 @@ public class HiveMetaStore extends Thrif
         ms.openTransaction();
         db = ms.getDatabase(name);
 
-        for (MetaStorePreEventListener listener : preListeners) {
-          listener.onEvent(new PreDropDatabaseEvent(db, this));
-        }
+        firePreEvent(new PreDropDatabaseEvent(db, this));
 
         List<String> allTables = get_all_tables(db.getName());
         if (!cascade && !allTables.isEmpty()) {
@@ -653,7 +647,7 @@ public class HiveMetaStore extends Thrif
         int startIndex = 0;
         int endIndex = -1;
         // retrieve the tables from the metastore in batches to alleviate memory constraints
-        while(endIndex < allTables.size() - 1) {
+        while (endIndex < allTables.size() - 1) {
           startIndex = endIndex + 1;
           endIndex = endIndex + tableBatchSize;
           if (endIndex >= allTables.size()) {
@@ -677,7 +671,7 @@ public class HiveMetaStore extends Thrif
                 tablePath = wh.getDnsPath(new Path(table.getSd().getLocation()));
                 if (!wh.isWritable(tablePath.getParent())) {
                   throw new MetaException("Database metadata not deleted since table: " +
-                      table.getTableName() + " has a parent location " +  tablePath.getParent() +
+                      table.getTableName() + " has a parent location " + tablePath.getParent() +
                       " which is not writable by " + hiveConf.getUser());
                 }
 
@@ -726,9 +720,10 @@ public class HiveMetaStore extends Thrif
     }
 
     /**
-     * Returns a BEST GUESS as to whether or not other is a subdirectory of parent.  It does not
+     * Returns a BEST GUESS as to whether or not other is a subdirectory of parent. It does not
      * take into account any intricacies of the underlying file system, which is assumed to be
-     * HDFS.  This should not return any false positives, but may return false negatives.
+     * HDFS. This should not return any false positives, but may return false negatives.
+     *
      * @param parent
      * @param other
      * @return
@@ -880,7 +875,7 @@ public class HiveMetaStore extends Thrif
     private void create_table_core(final RawStore ms, final Table tbl,
         final EnvironmentContext envContext)
         throws AlreadyExistsException, MetaException,
-               InvalidObjectException, NoSuchObjectException {
+        InvalidObjectException, NoSuchObjectException {
 
       if (!MetaStoreUtils.validateName(tbl.getTableName())
           || !MetaStoreUtils.validateColNames(tbl.getSd().getCols())
@@ -891,7 +886,7 @@ public class HiveMetaStore extends Thrif
                   null : tbl.getSd().getSkewedInfo().getSkewedColNames())
           || !MetaStoreUtils.validateSkewedColNamesSubsetCol(
               (null == tbl.getSd().getSkewedInfo()) ?
-              null : tbl.getSd().getSkewedInfo().getSkewedColNames(),
+                  null : tbl.getSd().getSkewedInfo().getSkewedColNames(),
               tbl.getSd().getCols())) {
         throw new InvalidObjectException(tbl.getTableName()
             + " is not a valid object name");
@@ -900,13 +895,7 @@ public class HiveMetaStore extends Thrif
       Path tblPath = null;
       boolean success = false, madeDir = false;
       try {
-        try {
-          for (MetaStorePreEventListener listener : preListeners) {
-            listener.onEvent(new PreCreateTableEvent(tbl, this));
-          }
-        } catch (InvalidOperationException e) {
-          throw new MetaException(e.getMessage());
-        }
+        firePreEvent(new PreCreateTableEvent(tbl, this));
 
         ms.openTransaction();
 
@@ -1026,13 +1015,7 @@ public class HiveMetaStore extends Thrif
           throw new MetaException("Table metadata is corrupted");
         }
 
-        try {
-          for(MetaStorePreEventListener listener : preListeners){
-            listener.onEvent(new PreDropTableEvent(tbl, this));
-          }
-        } catch (InvalidOperationException e) {
-          throw new MetaException(e.getMessage());
-        }
+        firePreEvent(new PreDropTableEvent(tbl, this));
 
         isIndexTable = isIndexTable(tbl);
         if (isIndexTable) {
@@ -1123,7 +1106,7 @@ public class HiveMetaStore extends Thrif
     }
 
     /**
-     * Retrieves the partitions specified by partitionKeys.  If checkLocation, for locations of
+     * Retrieves the partitions specified by partitionKeys. If checkLocation, for locations of
      * partitions which may not be subdirectories of tablePath checks to make the locations are
      * writable.
      *
@@ -1143,7 +1126,7 @@ public class HiveMetaStore extends Thrif
      */
     private List<Path> dropPartitionsAndGetLocations(RawStore ms, String dbName,
         String tableName, Path tablePath, List<FieldSchema> partitionKeys, boolean checkLocation)
-            throws MetaException, IOException {
+        throws MetaException, IOException {
 
       int partitionBatchSize = HiveConf.getIntVar(hiveConf,
           ConfVars.METASTORE_BATCH_RETRIEVE_MAX);
@@ -1155,7 +1138,7 @@ public class HiveMetaStore extends Thrif
 
       // call dropPartition on each of the table's partitions to follow the
       // procedure for cleanly dropping partitions.
-      while(true) {
+      while (true) {
         List<Partition> partsToDelete = ms.getPartitions(dbName, tableName, partitionBatchSize);
         if (partsToDelete == null || partsToDelete.isEmpty()) {
           break;
@@ -1166,13 +1149,13 @@ public class HiveMetaStore extends Thrif
 
             Path partPath = wh.getDnsPath(new Path(part.getSd().getLocation()));
             if (tableDnsPath == null ||
-                (partPath != null && !isSubdirectory(tableDnsPath, partPath))){
+                (partPath != null && !isSubdirectory(tableDnsPath, partPath))) {
               if (!wh.isWritable(partPath.getParent())) {
-                  throw new MetaException("Table metadata not deleted since the partition " +
-                      Warehouse.makePartName(partitionKeys, part.getValues()) +
-                      " has parent location " + partPath.getParent() + " which is not writable " +
-                      "by " + hiveConf.getUser());
-               }
+                throw new MetaException("Table metadata not deleted since the partition " +
+                    Warehouse.makePartName(partitionKeys, part.getValues()) +
+                    " has parent location " + partPath.getParent() + " which is not writable " +
+                    "by " + hiveConf.getUser());
+              }
               partPaths.add(partPath);
             }
           }
@@ -1294,8 +1277,8 @@ public class HiveMetaStore extends Thrif
     }
 
     private Partition append_partition_common(RawStore ms, String dbName, String tableName,
-        List<String> part_vals) throws InvalidObjectException,
-        AlreadyExistsException, MetaException {
+        List<String> part_vals) throws InvalidObjectException, AlreadyExistsException,
+        MetaException {
 
       Partition part = new Partition();
       boolean success = false, madeDir = false;
@@ -1306,6 +1289,9 @@ public class HiveMetaStore extends Thrif
         part.setTableName(tableName);
         part.setValues(part_vals);
 
+        PreAddPartitionEvent event = new PreAddPartitionEvent(part, this);
+        firePreEvent(event);
+
         Table tbl = ms.getTable(part.getDbName(), part.getTableName());
         if (tbl == null) {
           throw new InvalidObjectException(
@@ -1357,10 +1343,28 @@ public class HiveMetaStore extends Thrif
             wh.deleteDir(partLocation, true);
           }
         }
+
+        for (MetaStoreEventListener listener : listeners) {
+          AddPartitionEvent addPartitionEvent =
+              new AddPartitionEvent(part, success, this);
+          listener.onAddPartition(addPartitionEvent);
+        }
       }
       return part;
     }
 
+    private void firePreEvent(PreEventContext event) throws MetaException {
+      for (MetaStorePreEventListener listener : preListeners) {
+        try {
+          listener.onEvent(event);
+        } catch (NoSuchObjectException e) {
+          throw new MetaException(e.getMessage());
+        } catch (InvalidOperationException e) {
+          throw new MetaException(e.getMessage());
+        }
+      }
+    }
+
     public Partition append_partition(final String dbName, final String tableName,
         final List<String> part_vals) throws InvalidObjectException,
         AlreadyExistsException, MetaException {
@@ -1434,7 +1438,8 @@ public class HiveMetaStore extends Thrif
      *
      * @param ms
      * @param part
-     * @param envContext parameters passed by the client
+     * @param envContext
+     *          parameters passed by the client
      * @return
      * @throws InvalidObjectException
      * @throws AlreadyExistsException
@@ -1447,15 +1452,7 @@ public class HiveMetaStore extends Thrif
       boolean success = false, madeDir = false;
       Path partLocation = null;
       try {
-        try {
-          for(MetaStorePreEventListener listener : preListeners){
-            listener.onEvent(new PreAddPartitionEvent(part, this));
-          }
-        } catch (NoSuchObjectException e) {
-          throw new MetaException(e.getMessage());
-        } catch (InvalidOperationException e) {
-          throw new MetaException(e.getMessage());
-        }
+        firePreEvent(new PreAddPartitionEvent(part, this));
 
         Partition old_part = null;
         try {
@@ -1520,18 +1517,18 @@ public class HiveMetaStore extends Thrif
         }
 
         // Inherit table properties into partition properties.
-        Map<String,String> tblParams = tbl.getParameters();
-        String inheritProps =  hiveConf.getVar(ConfVars.METASTORE_PART_INHERIT_TBL_PROPS).trim();
+        Map<String, String> tblParams = tbl.getParameters();
+        String inheritProps = hiveConf.getVar(ConfVars.METASTORE_PART_INHERIT_TBL_PROPS).trim();
         // Default value is empty string in which case no properties will be inherited.
         // * implies all properties needs to be inherited
         Set<String> inheritKeys = new HashSet<String>(Arrays.asList(inheritProps.split(",")));
-        if(inheritKeys.contains("*")){
-          inheritKeys =  tblParams.keySet();
+        if (inheritKeys.contains("*")) {
+          inheritKeys = tblParams.keySet();
         }
 
         for (String key : inheritKeys) {
           String paramVal = tblParams.get(key);
-          if(null != paramVal){ // add the property only if it exists in table properties
+          if (null != paramVal) { // add the property only if it exists in table properties
             part.putToParameters(key, paramVal);
           }
         }
@@ -1618,13 +1615,7 @@ public class HiveMetaStore extends Thrif
         ms.openTransaction();
         part = ms.getPartition(db_name, tbl_name, part_vals);
 
-        try {
-          for(MetaStorePreEventListener listener : preListeners){
-            listener.onEvent(new PreDropPartitionEvent(part, this));
-          }
-        } catch (InvalidOperationException e) {
-          throw new MetaException(e.getMessage());
-        }
+        firePreEvent(new PreDropPartitionEvent(part, this));
 
         if (part == null) {
           throw new NoSuchObjectException("Partition doesn't exist. "
@@ -1719,7 +1710,7 @@ public class HiveMetaStore extends Thrif
       Partition ret = null;
       try {
         ret = getMS().getPartitionWithAuth(db_name, tbl_name, part_vals,
-                user_name, group_names);
+            user_name, group_names);
       } catch (InvalidObjectException e) {
         throw new NoSuchObjectException(e.getMessage());
       } finally {
@@ -1752,7 +1743,7 @@ public class HiveMetaStore extends Thrif
       List<Partition> ret = null;
       try {
         ret = getMS().getPartitionsWithAuth(dbName, tblName, maxParts,
-                userName, groupNames);
+            userName, groupNames);
       } catch (InvalidObjectException e) {
         throw new NoSuchObjectException(e.getMessage());
       } finally {
@@ -1788,8 +1779,8 @@ public class HiveMetaStore extends Thrif
         final String tableName, final Partition newPartition,
         final EnvironmentContext envContext)
         throws InvalidOperationException, MetaException, TException {
-        rename_partition(dbName, tableName, null,
-            newPartition, envContext);
+      rename_partition(dbName, tableName, null,
+          newPartition, envContext);
     }
 
     @Override
@@ -1807,7 +1798,7 @@ public class HiveMetaStore extends Thrif
         TException {
       startTableFunction("alter_partition", db_name, tbl_name);
 
-      if(LOG.isInfoEnabled()) {
+      if (LOG.isInfoEnabled()) {
         LOG.info("New partition values:" + new_part.getValues());
         if (part_vals != null && part_vals.size() > 0) {
           LOG.info("Old Partition values:" + part_vals);
@@ -1816,14 +1807,7 @@ public class HiveMetaStore extends Thrif
 
       Partition oldPart = null;
       try {
-        try {
-          for (MetaStorePreEventListener listener : preListeners) {
-            listener.onEvent(
-                new PreAlterPartitionEvent(db_name, tbl_name, part_vals, new_part, this));
-          }
-        } catch (NoSuchObjectException e) {
-          throw new MetaException(e.getMessage());
-        }
+        firePreEvent(new PreAlterPartitionEvent(db_name, tbl_name, part_vals, new_part, this));
 
         oldPart = alterHandler.alterPartition(getMS(), wh, db_name, tbl_name, part_vals, new_part);
 
@@ -1852,7 +1836,7 @@ public class HiveMetaStore extends Thrif
       startTableFunction("alter_partitions", db_name, tbl_name);
 
       if (LOG.isInfoEnabled()) {
-        for (Partition tmpPart: new_parts) {
+        for (Partition tmpPart : new_parts) {
           LOG.info("New partition values:" + tmpPart.getValues());
         }
       }
@@ -1860,11 +1844,11 @@ public class HiveMetaStore extends Thrif
       // all prehooks are fired together followed by all post hooks
       List<Partition> oldParts = null;
       try {
-        for (Partition tmpPart: new_parts) {
+        for (Partition tmpPart : new_parts) {
           try {
             for (MetaStorePreEventListener listener : preListeners) {
               listener.onEvent(
-                new PreAlterPartitionEvent(db_name, tbl_name, null, tmpPart, this));
+                  new PreAlterPartitionEvent(db_name, tbl_name, null, tmpPart, this));
             }
           } catch (NoSuchObjectException e) {
             throw new MetaException(e.getMessage());
@@ -1873,10 +1857,10 @@ public class HiveMetaStore extends Thrif
         oldParts = alterHandler.alterPartitions(getMS(), wh, db_name, tbl_name, new_parts);
 
         Iterator<Partition> olditr = oldParts.iterator();
-        for (Partition tmpPart: new_parts) {
+        for (Partition tmpPart : new_parts) {
           Partition oldTmpPart = null;
           if (olditr.hasNext()) {
-            oldTmpPart = (Partition)olditr.next();
+            oldTmpPart = (Partition) olditr.next();
           }
           else {
             throw new InvalidOperationException("failed to alterpartitions");
@@ -1961,13 +1945,8 @@ public class HiveMetaStore extends Thrif
       try {
         Table oldt = get_table(dbname, name);
 
-        try {
-          for (MetaStorePreEventListener listener : preListeners) {
-            listener.onEvent(new PreAlterTableEvent(oldt, newTable, this));
-          }
-        } catch (NoSuchObjectException e) {
-          throw new MetaException(e.getMessage());
-        }
+        firePreEvent(new PreAlterTableEvent(oldt, newTable, this));
+
         alterHandler.alterTable(getMS(), wh, dbname, name, newTable);
         success = true;
 
@@ -2200,7 +2179,7 @@ public class HiveMetaStore extends Thrif
       try {
         RawStore ms = getMS();
         List<String> partVals = getPartValsFromName(ms, db_name, tbl_name, part_name);
-        ret =  append_partition_common(ms, db_name, tbl_name, partVals);
+        ret = append_partition_common(ms, db_name, tbl_name, partVals);
       } finally {
         endFunction("append_partition_by_name", ret != null);
       }
@@ -2232,7 +2211,7 @@ public class HiveMetaStore extends Thrif
       boolean ret = false;
       try {
         ret = drop_partition_by_name_core(getMS(), db_name, tbl_name,
-                part_name, deleteData);
+            part_name, deleteData);
       } catch (IOException e) {
         throw new MetaException(e.getMessage());
       } finally {
@@ -2269,7 +2248,7 @@ public class HiveMetaStore extends Thrif
       List<Partition> ret = null;
       try {
         ret = getMS().listPartitionsPsWithAuth(db_name, tbl_name, part_vals, max_parts,
-                userName, groupNames);
+            userName, groupNames);
       } catch (InvalidObjectException e) {
         throw new MetaException(e.getMessage());
       } finally {
@@ -2393,7 +2372,7 @@ public class HiveMetaStore extends Thrif
       boolean ret = false;
       try {
         ret = drop_index_by_name_core(getMS(), dbName, tblName,
-                indexName, deleteData);
+            indexName, deleteData);
       } catch (IOException e) {
         throw new MetaException(e.getMessage());
       } finally {
@@ -2601,7 +2580,7 @@ public class HiveMetaStore extends Thrif
       PrincipalPrivilegeSet ret = null;
       try {
         ret = getMS().getColumnPrivilegeSet(
-                dbName, tableName, partName, columnName, userName, groupNames);
+            dbName, tableName, partName, columnName, userName, groupNames);
       } catch (MetaException e) {
         throw e;
       } catch (Exception e) {
@@ -2635,7 +2614,7 @@ public class HiveMetaStore extends Thrif
       PrincipalPrivilegeSet ret = null;
       try {
         ret = getMS().getPartitionPrivilegeSet(dbName, tableName, partName,
-                userName, groupNames);
+            userName, groupNames);
       } catch (MetaException e) {
         throw e;
       } catch (Exception e) {
@@ -2652,7 +2631,7 @@ public class HiveMetaStore extends Thrif
       PrincipalPrivilegeSet ret = null;
       try {
         ret = getMS().getTablePrivilegeSet(dbName, tableName, userName,
-                groupNames);
+            groupNames);
       } catch (MetaException e) {
         throw e;
       } catch (Exception e) {
@@ -3111,10 +3090,7 @@ public class HiveMetaStore extends Thrif
       try {
         startPartitionFunction("markPartitionForEvent", db_name, tbl_name, partName);
         try {
-          for(MetaStorePreEventListener listener : preListeners){
-            listener.onEvent(
-                new PreLoadPartitionDoneEvent(this, db_name, tbl_name, partName));
-          }
+          firePreEvent(new PreLoadPartitionDoneEvent(db_name, tbl_name, partName, this));
           tbl = getMS().markPartitionForEvent(db_name, tbl_name, partName, evtType);
         } catch (Exception original) {
           LOG.error(original);
@@ -3140,7 +3116,7 @@ public class HiveMetaStore extends Thrif
           throw new UnknownTableException("Table: " + tbl_name + " not found.");
         } else {
           for (MetaStoreEventListener listener : listeners) {
-            listener.onLoadPartitionDone(new LoadPartitionDoneEvent(true, this, tbl, partName));
+            listener.onLoadPartitionDone(new LoadPartitionDoneEvent(true, tbl, partName, this));
           }
         }
       } finally {
@@ -3317,16 +3293,16 @@ public class HiveMetaStore extends Thrif
         conf.set((String) item.getKey(), (String) item.getValue());
       }
 
-      //Add shutdown hook.
+      // Add shutdown hook.
       Runtime.getRuntime().addShutdownHook(new Thread() {
-          @Override
-          public void run() {
-              String shutdownMsg = "Shutting down hive metastore.";
-              HMSHandler.LOG.info(shutdownMsg);
-              if (isCliVerbose) {
-                  System.err.println(shutdownMsg);
-                }
+        @Override
+        public void run() {
+          String shutdownMsg = "Shutting down hive metastore.";
+          HMSHandler.LOG.info(shutdownMsg);
+          if (isCliVerbose) {
+            System.err.println(shutdownMsg);
           }
+        }
       });
 
       startMetaStore(cli.port, ShimLoader.getHadoopThriftAuthBridge(), conf);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java Wed Oct 24 17:05:06 2012
@@ -50,13 +50,15 @@ public abstract class MetaStoreEventList
    * @param tableEvent table event.
    * @throws MetaException
    */
-  public abstract void onCreateTable (CreateTableEvent tableEvent) throws MetaException;
+  public void onCreateTable (CreateTableEvent tableEvent) throws MetaException {
+  }
 
   /**
    * @param tableEvent table event.
    * @throws MetaException
    */
-  public abstract void onDropTable (DropTableEvent tableEvent)  throws MetaException;
+  public void onDropTable (DropTableEvent tableEvent)  throws MetaException {
+  }
 
   /**
    * @param add partition event
@@ -67,40 +69,48 @@ public abstract class MetaStoreEventList
    * @param tableEvent alter table event
    * @throws MetaException
    */
-  public abstract void onAlterTable (AlterTableEvent tableEvent) throws MetaException;
-
+  public void onAlterTable (AlterTableEvent tableEvent) throws MetaException {
+  }
 
-  public abstract void onAddPartition (AddPartitionEvent partitionEvent)  throws MetaException;
+  public void onAddPartition (AddPartitionEvent partitionEvent)
+      throws MetaException {
+  }
 
   /**
    * @param partitionEvent partition event
    * @throws MetaException
    */
-  public abstract void onDropPartition (DropPartitionEvent partitionEvent)  throws MetaException;
+  public void onDropPartition (DropPartitionEvent partitionEvent)  throws MetaException {
+  }
 
   /**
    * @param partitionEvent partition event
    * @throws MetaException
    */
-  public abstract void onAlterPartition (AlterPartitionEvent partitionEvent)  throws MetaException;
+  public void onAlterPartition (AlterPartitionEvent partitionEvent)  throws MetaException {
+  }
 
   /**
    * @param dbEvent database event
    * @throws MetaException
    */
-  public abstract void onCreateDatabase (CreateDatabaseEvent dbEvent) throws MetaException;
+  public void onCreateDatabase (CreateDatabaseEvent dbEvent) throws MetaException {
+  }
 
   /**
    * @param dbEvent database event
    * @throws MetaException
    */
-  public abstract void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException;
+  public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
+  }
 
   /**
    * @param partSetDoneEvent
    * @throws MetaException
    */
-  public abstract void onLoadPartitionDone(LoadPartitionDoneEvent partSetDoneEvent) throws MetaException;
+  public void onLoadPartitionDone(LoadPartitionDoneEvent partSetDoneEvent) throws MetaException {
+
+  }
 
   @Override
   public Configuration getConf() {

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionNameWhitelistPreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionNameWhitelistPreEventListener.java?rev=1401784&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionNameWhitelistPreEventListener.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionNameWhitelistPreEventListener.java Wed Oct 24 17:05:06 2012
@@ -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;
+
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+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.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+
+public class PartitionNameWhitelistPreEventListener extends MetaStorePreEventListener {
+
+  // When enabled, this hook causes an exception to be thrown
+  // if partition fields contain characters which are not
+  // matched by the whitelist
+
+  private static String regex;
+  private static Pattern pattern;
+
+  public PartitionNameWhitelistPreEventListener(Configuration config) {
+    super(config);
+
+    regex = config.get(HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN.varname);
+    pattern = Pattern.compile(regex);
+  }
+
+  @Override
+  public void onEvent(PreEventContext event) throws MetaException, NoSuchObjectException,
+      InvalidOperationException {
+
+    switch (event.getEventType()) {
+    case ADD_PARTITION:
+      checkWhitelist(((PreAddPartitionEvent) event).getPartition().getValues());
+      break;
+    }
+
+  }
+
+  private static void checkWhitelist(List<String> partVals) throws MetaException {
+    for (String partVal : partVals) {
+      if (!pattern.matcher(partVal).matches()) {
+        throw new MetaException("Partition value '" + partVal + "' contains a character "
+            + "not matched by whitelist pattern '" + regex + "'.  " + "(configure with "
+            + HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN.varname + ")");
+      }
+    }
+  }
+
+}

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AddPartitionEvent.java Wed Oct 24 17:05:06 2012
@@ -26,7 +26,6 @@ public class AddPartitionEvent extends L
   private final Partition partition;
 
   public AddPartitionEvent (Partition partition, boolean status, HMSHandler handler) {
-
     super (status, handler);
     this.partition = partition;
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterPartitionEvent.java Wed Oct 24 17:05:06 2012
@@ -26,9 +26,9 @@ public class AlterPartitionEvent extends
   private final Partition oldPart;
   private final Partition newPart;
 
-  public AlterPartitionEvent (Partition oldPart, Partition newPart,
+  public AlterPartitionEvent(Partition oldPart, Partition newPart,
       boolean status, HMSHandler handler) {
-    super (status, handler);
+    super(status, handler);
     this.oldPart = oldPart;
     this.newPart = newPart;
   }
@@ -47,4 +47,4 @@ public class AlterPartitionEvent extends
   public Partition getNewPartition() {
     return newPart;
   }
-}
\ No newline at end of file
+}

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/AlterTableEvent.java Wed Oct 24 17:05:06 2012
@@ -27,7 +27,6 @@ public class AlterTableEvent extends Lis
   private final Table newTable;
   private final Table oldTable;
   public AlterTableEvent (Table oldTable, Table newTable, boolean status, HMSHandler handler) {
-
     super (status, handler);
     this.oldTable = oldTable;
     this.newTable = newTable;

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateDatabaseEvent.java Wed Oct 24 17:05:06 2012
@@ -26,7 +26,6 @@ public class CreateDatabaseEvent extends
   private final Database db;
 
   public CreateDatabaseEvent (Database db, boolean status, HMSHandler handler) {
-
     super (status, handler);
     this.db = db;
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateTableEvent.java Wed Oct 24 17:05:06 2012
@@ -26,7 +26,6 @@ public class CreateTableEvent extends Li
   private final Table table;
 
   public CreateTableEvent (Table table, boolean status, HMSHandler handler) {
-
     super (status, handler);
     this.table = table;
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropDatabaseEvent.java Wed Oct 24 17:05:06 2012
@@ -26,7 +26,6 @@ public class DropDatabaseEvent extends L
   private final Database db;
 
   public DropDatabaseEvent(Database db, boolean status, HMSHandler handler) {
-
     super (status, handler);
     this.db = db;
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropPartitionEvent.java Wed Oct 24 17:05:06 2012
@@ -26,7 +26,6 @@ public class DropPartitionEvent extends 
   private final Partition partition;
 
   public DropPartitionEvent (Partition partition, boolean status, HMSHandler handler) {
-
     super (status, handler);
     this.partition = partition;
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropTableEvent.java Wed Oct 24 17:05:06 2012
@@ -21,20 +21,19 @@ package org.apache.hadoop.hive.metastore
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.Table;
 
-public class DropTableEvent extends ListenerEvent{
+public class DropTableEvent extends ListenerEvent {
 
   private final Table table;
 
-  public DropTableEvent (Table table, boolean status, HMSHandler handler) {
-
-    super (status, handler);
+  public DropTableEvent(Table table, boolean status, HMSHandler handler) {
+    super(status, handler);
     this.table = table;
   }
 
   /**
    * @return the table
    */
-  public Table getTable () {
+  public Table getTable() {
     return table;
   }
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java Wed Oct 24 17:05:06 2012
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.metastore.events;
 
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 
 /**
@@ -44,13 +43,6 @@ public abstract class ListenerEvent {
   }
 
   /**
-   * @return the handler
-   */
-  public HMSHandler getHandler() {
-    return handler;
-  }
-
-  /**
    * @return the status of event.
    */
   public boolean getStatus() {
@@ -68,4 +60,10 @@ public abstract class ListenerEvent {
     return environmentContext;
   }
 
+  /**
+   * @return the handler
+   */
+  public HMSHandler getHandler() {
+    return handler;
+  }
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/LoadPartitionDoneEvent.java Wed Oct 24 17:05:06 2012
@@ -29,8 +29,8 @@ public class LoadPartitionDoneEvent exte
 
   private final Map<String,String> partSpec;
 
-  public LoadPartitionDoneEvent(boolean status, HMSHandler handler, Table table,
-      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;

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreCreateTableEvent.java Wed Oct 24 17:05:06 2012
@@ -25,15 +25,15 @@ public class PreCreateTableEvent extends
 
   private final Table table;
 
-  public PreCreateTableEvent (Table table, HMSHandler handler) {
-    super (PreEventType.CREATE_TABLE, handler);
+  public PreCreateTableEvent(Table table, HMSHandler handler) {
+    super(PreEventType.CREATE_TABLE, handler);
     this.table = table;
   }
 
   /**
    * @return the table
    */
-  public Table getTable () {
+  public Table getTable() {
     return table;
   }
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropDatabaseEvent.java Wed Oct 24 17:05:06 2012
@@ -26,7 +26,7 @@ public class PreDropDatabaseEvent extend
   private final Database db;
 
   public PreDropDatabaseEvent(Database db, HMSHandler handler) {
-    super (PreEventType.DROP_DATABASE, handler);
+    super(PreEventType.DROP_DATABASE, handler);
     this.db = db;
   }
 

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreDropTableEvent.java Wed Oct 24 17:05:06 2012
@@ -21,18 +21,20 @@ package org.apache.hadoop.hive.metastore
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.Table;
 
-public class PreDropTableEvent extends PreEventContext{
+public class PreDropTableEvent extends PreEventContext {
 
   private final Table table;
 
-  public PreDropTableEvent (Table table, HMSHandler handler) {
-    super (PreEventType.DROP_TABLE, handler);
+  public PreDropTableEvent(Table table, HMSHandler handler) {
+    super(PreEventType.DROP_TABLE, handler);
     this.table = table;
   }
+
   /**
    * @return the table
    */
-  public Table getTable () {
+  public Table getTable() {
     return table;
   }
+
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java Wed Oct 24 17:05:06 2012
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore
 
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 
+
 /**
  * Base class for all the events which are defined for metastore.
  */
@@ -35,13 +36,13 @@ public abstract class PreEventContext {
     ALTER_PARTITION,
     CREATE_DATABASE,
     DROP_DATABASE,
-    LOAD_PARTITION_DONE,
+    LOAD_PARTITION_DONE
   }
 
   private final PreEventType eventType;
   private final HMSHandler handler;
 
-  public PreEventContext(PreEventType eventType, HMSHandler handler) {
+  public PreEventContext(PreEventType eventType, HMSHandler  handler) {
     this.eventType = eventType;
     this.handler = handler;
   }
@@ -59,4 +60,5 @@ public abstract class PreEventContext {
   public HMSHandler getHandler() {
     return handler;
   }
+
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java Wed Oct 24 17:05:06 2012
@@ -28,8 +28,8 @@ public class PreLoadPartitionDoneEvent e
   private final String tableName;
   private final Map<String,String> partSpec;
 
-  public PreLoadPartitionDoneEvent(HMSHandler handler, String dbName, String tableName,
-      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;

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java Wed Oct 24 17:05:06 2012
@@ -90,4 +90,5 @@ public class DummyListener extends MetaS
   public void onLoadPartitionDone(LoadPartitionDoneEvent partEvent) throws MetaException {
     notifyList.add(partEvent);
   }
+
 }

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java Wed Oct 24 17:05:06 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -246,6 +247,22 @@ public class TestMetaStoreEventListener 
         preAlterPartEvent.getTableName(), preAlterPartEvent.getNewPartition().getValues(),
         preAlterPartEvent.getNewPartition());
 
+    List<String> part_vals = new ArrayList<String>();
+    part_vals.add("c=2012");
+    Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
+
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    assertEquals(preNotifyList.size(), listSize);
+
+    AddPartitionEvent appendPartEvent =
+        (AddPartitionEvent)(notifyList.get(listSize-1));
+    validateAddPartition(newPart, appendPartEvent.getPartition());
+
+    PreAddPartitionEvent preAppendPartEvent =
+        (PreAddPartitionEvent)(preNotifyList.get(listSize-1));
+    validateAddPartition(newPart, preAppendPartEvent.getPartition());
+
     driver.run(String.format("alter table %s rename to %s", tblName, renamed));
     listSize++;
     assertEquals(notifyList.size(), listSize);
@@ -336,4 +353,5 @@ public class TestMetaStoreEventListener 
     assert dropDB.getStatus();
     validateDropDb(db, preDropDB.getDatabase());
   }
+
 }

Added: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistPreEventHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistPreEventHook.java?rev=1401784&view=auto
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistPreEventHook.java (added)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistPreEventHook.java Wed Oct 24 17:05:06 2012
@@ -0,0 +1,164 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.junit.Test;
+
+// Validate CharacterWhitelistPreEventHook to ensure it refuses to process
+// a partition add or append request if partition fields contain
+// Unicode characters or commas
+
+public class TestPartitionNameWhitelistPreEventHook {
+
+  // Runs an instance of DisallowUnicodePreEventListener
+  // Returns whether or not it succeeded
+  private boolean runHook(PreEventContext event) {
+
+    Configuration config = new Configuration();
+
+    // match the printable ASCII characters except for commas
+    config.set(HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN.varname
+        , "[\\x20-\\x7E&&[^,]]*");
+
+    PartitionNameWhitelistPreEventListener hook =
+      new PartitionNameWhitelistPreEventListener(config);
+
+    try {
+      hook.onEvent(event);
+    } catch (Exception e) {
+      return false;
+    }
+
+    return true;
+ }
+
+  // Sample data
+  private List<String> getPartValsWithUnicode() {
+
+    List<String> partVals = new ArrayList<String>();
+    partVals.add("klâwen");
+    partVals.add("tägelîch");
+
+    return partVals;
+
+  }
+
+  private List<String> getPartValsWithCommas() {
+
+    List<String> partVals = new ArrayList<String>();
+    partVals.add("a,b");
+    partVals.add("c,d,e,f");
+
+    return partVals;
+
+  }
+
+  private List<String> getPartValsWithValidCharacters() {
+
+    List<String> partVals = new ArrayList<String>();
+    partVals.add("part1");
+    partVals.add("part2");
+
+    return partVals;
+
+  }
+
+  @Test
+  public void testAddPartitionWithCommas() {
+
+    Partition partition = new Partition();
+    partition.setValues(getPartValsWithCommas());
+
+    PreAddPartitionEvent event = new PreAddPartitionEvent(partition, null);
+
+    Assert.assertFalse("Add a partition with commas in name",
+                       runHook(event));
+  }
+
+  @Test
+  public void testAddPartitionWithUnicode() {
+
+    Partition partition = new Partition();
+    partition.setValues(getPartValsWithUnicode());
+
+    PreAddPartitionEvent event = new PreAddPartitionEvent(partition, null);
+
+    Assert.assertFalse("Add a partition with unicode characters in name",
+                       runHook(event));
+  }
+
+  @Test
+  public void testAddPartitionWithValidPartVal() {
+
+    Partition p = new Partition();
+    p.setValues(getPartValsWithValidCharacters());
+
+    PreAddPartitionEvent event = new PreAddPartitionEvent(p, null);
+
+    Assert.assertTrue("Add a partition with unicode characters in name",
+                       runHook(event));
+  }
+
+  @Test
+  public void testAppendPartitionWithUnicode() {
+
+    Partition p = new Partition();
+    p.setValues(getPartValsWithUnicode());
+
+    PreAddPartitionEvent event = new PreAddPartitionEvent(p, null);
+
+    Assert.assertFalse("Append a partition with unicode characters in name",
+                       runHook(event));
+  }
+
+  @Test
+  public void testAppendPartitionWithCommas() {
+
+    Partition p = new Partition();
+    p.setValues(getPartValsWithCommas());
+
+    PreAddPartitionEvent event = new PreAddPartitionEvent(p, null);
+
+    Assert.assertFalse("Append a partition with unicode characters in name",
+                       runHook(event));
+  }
+
+  @Test
+  public void testAppendPartitionWithValidCharacters() {
+
+    Partition p = new Partition();
+    p.setValues(getPartValsWithValidCharacters());
+
+    PreAddPartitionEvent event = new PreAddPartitionEvent(p, null);
+
+    Assert.assertTrue("Append a partition with no unicode characters in name",
+                       runHook(event));
+  }
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java?rev=1401784&r1=1401783&r2=1401784&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/MetaDataExportListener.java Wed Oct 24 17:05:06 2012
@@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configurat
 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.HiveMetaStore.HMSHandler;
 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;
@@ -40,8 +40,9 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
- * This class listens for drop events and, if set, exports the table's metadata as JSON
- * to the trash of the user performing the drop */
+ * This class listens for drop events and, if set, exports the table's metadata as JSON to the trash
+ * of the user performing the drop
+ */
 public class MetaDataExportListener extends MetaStorePreEventListener {
   public static final Log LOG = LogFactory.getLog(MetaDataExportListener.class);
   private static final SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
@@ -56,8 +57,8 @@ public class MetaDataExportListener exte
     FileSystem fs = null;
     Table tbl = tableEvent.getTable();
     String name = tbl.getTableName();
-    org.apache.hadoop.hive.ql.metadata.Table mTbl =
-      new org.apache.hadoop.hive.ql.metadata.Table(tbl);
+    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);
@@ -66,8 +67,8 @@ public class MetaDataExportListener exte
     Date now = new Date();
     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);
+    boolean moveMetadataToTrash = hiveconf
+        .getBoolVar(HiveConf.ConfVars.MOVE_EXPORTED_METADATA_TO_TRASH);
     Path exportPath = null;
     if (exportPathString != null && exportPathString.length() == 0) {
       exportPath = fs.getHomeDirectory();
@@ -75,8 +76,8 @@ public class MetaDataExportListener exte
       exportPath = new Path(exportPathString);
     }
     Path metaPath = new Path(exportPath, name + "." + dateString);
-    LOG.info("Exporting the metadata of table " +
-      tbl.toString() + " to path " + metaPath.toString());
+    LOG.info("Exporting the metadata of table " + tbl.toString() + " to path "
+        + metaPath.toString());
     try {
       fs.mkdirs(metaPath);
     } catch (IOException e) {

Added: hive/trunk/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q Wed Oct 24 17:05:06 2012
@@ -0,0 +1,9 @@
+SET hive.metastore.pre.event.listeners=org.apache.hadoop.hive.metastore.PartitionNameWhitelistPreEventListener;
+SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ;
+-- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas
+
+CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING);
+SHOW PARTITIONS part_whitelist_test;
+
+ALTER TABLE part_whitelist_test ADD PARTITION (ds='1,2,3,4');
+

Added: hive/trunk/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q Wed Oct 24 17:05:06 2012
@@ -0,0 +1,14 @@
+SET hive.metastore.pre.event.listeners=org.apache.hadoop.hive.metastore.PartitionNameWhitelistPreEventListener;
+SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*;
+
+set hive.exec.dynamic.partition=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+create table source_table like srcpart;
+
+create table dest_table like srcpart;
+
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11);
+
+insert overwrite table dest_table partition (ds, hr) select key, value, hr from source_table where ds='2008-04-08';
+

Added: hive/trunk/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q Wed Oct 24 17:05:06 2012
@@ -0,0 +1,7 @@
+SET hive.metastore.pre.event.listeners= ;
+-- Test with PartitionNameWhitelistPreEventListener NOT registered
+
+CREATE TABLE part_nowhitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING);
+SHOW PARTITIONS part_nowhitelist_test;
+
+ALTER TABLE part_nowhitelist_test ADD PARTITION (ds='1,2,3,4');

Added: hive/trunk/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q Wed Oct 24 17:05:06 2012
@@ -0,0 +1,10 @@
+SET hive.metastore.pre.event.listeners=org.apache.hadoop.hive.metastore.PartitionNameWhitelistPreEventListener;
+SET hive.metastore.partition.name.whitelist.pattern=[A-Za-z]*;
+-- This pattern matches only letters.
+
+CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING);
+SHOW PARTITIONS part_whitelist_test;
+
+ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part');
+
+      

Added: hive/trunk/ql/src/test/results/clientnegative/add_partition_with_whitelist.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/add_partition_with_whitelist.q.out?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/add_partition_with_whitelist.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/add_partition_with_whitelist.q.out Wed Oct 24 17:05:06 2012
@@ -0,0 +1,18 @@
+PREHOOK: query: -- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas
+
+CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas
+
+CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part_whitelist_test
+PREHOOK: query: SHOW PARTITIONS part_whitelist_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS part_whitelist_test
+POSTHOOK: type: SHOWPARTITIONS
+PREHOOK: query: ALTER TABLE part_whitelist_test ADD PARTITION (ds='1,2,3,4')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@part_whitelist_test
+FAILED: Error in metadata: MetaException(message:Partition value '1,2,3,4' contains a character not matched by whitelist pattern '[\\x20-\\x7E&&[^,]]*'.  (configure with hive.metastore.partition.name.whitelist.pattern))
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hive/trunk/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out Wed Oct 24 17:05:06 2012
@@ -0,0 +1,15 @@
+PREHOOK: query: create table source_table like srcpart
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table source_table like srcpart
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@source_table
+PREHOOK: query: create table dest_table like srcpart
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table dest_table like srcpart
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@dest_table
+PREHOOK: query: load data local inpath '../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11)
+PREHOOK: type: LOAD
+PREHOOK: Output: default@source_table
+Failed with exception MetaException(message:Partition value '2008-04-08' contains a character not matched by whitelist pattern '[A-Za-z]*'.  (configure with hive.metastore.partition.name.whitelist.pattern))
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask

Added: hive/trunk/ql/src/test/results/clientpositive/add_partition_no_whitelist.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/add_partition_no_whitelist.q.out?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/add_partition_no_whitelist.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/add_partition_no_whitelist.q.out Wed Oct 24 17:05:06 2012
@@ -0,0 +1,20 @@
+PREHOOK: query: -- Test with PartitionNameWhitelistPreEventListener NOT registered
+
+CREATE TABLE part_nowhitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- Test with PartitionNameWhitelistPreEventListener NOT registered
+
+CREATE TABLE part_nowhitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part_nowhitelist_test
+PREHOOK: query: SHOW PARTITIONS part_nowhitelist_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS part_nowhitelist_test
+POSTHOOK: type: SHOWPARTITIONS
+PREHOOK: query: ALTER TABLE part_nowhitelist_test ADD PARTITION (ds='1,2,3,4')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@part_nowhitelist_test
+POSTHOOK: query: ALTER TABLE part_nowhitelist_test ADD PARTITION (ds='1,2,3,4')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@part_nowhitelist_test
+POSTHOOK: Output: default@part_nowhitelist_test@ds=1,2,3,4

Added: hive/trunk/ql/src/test/results/clientpositive/add_partition_with_whitelist.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/add_partition_with_whitelist.q.out?rev=1401784&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/add_partition_with_whitelist.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/add_partition_with_whitelist.q.out Wed Oct 24 17:05:06 2012
@@ -0,0 +1,20 @@
+PREHOOK: query: -- This pattern matches only letters.
+
+CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- This pattern matches only letters.
+
+CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@part_whitelist_test
+PREHOOK: query: SHOW PARTITIONS part_whitelist_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS part_whitelist_test
+POSTHOOK: type: SHOWPARTITIONS
+PREHOOK: query: ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@part_whitelist_test
+POSTHOOK: query: ALTER TABLE part_whitelist_test ADD PARTITION (ds='Part')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@part_whitelist_test
+POSTHOOK: Output: default@part_whitelist_test@ds=Part