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

svn commit: r984204 [1/2] - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/java/org/apache/hadoop/hive/ql/plan/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientneg...

Author: namit
Date: Tue Aug 10 20:41:57 2010
New Revision: 984204

URL: http://svn.apache.org/viewvc?rev=984204&view=rev
Log:
HIVE-1514. API to change fileformat and location of a partition
(He Yongqiang via namit)


Added:
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter_partition_format_loc.q
    hadoop/hive/trunk/ql/src/test/results/clientpositive/alter_partition_format_loc.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part_no_drop.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl_no_drop.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/diff_part_input_formats.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/fileformat_mix.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/partition_wise_fileformat.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/partition_wise_fileformat2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/partition_wise_fileformat3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/protectmode.q.out

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Aug 10 20:41:57 2010
@@ -36,6 +36,9 @@ Trunk -  Unreleased
     HIVE-1413. Ability to take a table offline
     (Siying Dong via namit)
 
+    HIVE-1514. API to change fileformat and location of a partition
+    (He Yongqiang via namit)
+
   IMPROVEMENTS
 
     HIVE-1394. Do not update transient_lastDdlTime if the partition is modified by a housekeeping
@@ -61,7 +64,7 @@ Trunk -  Unreleased
     HIVE-1229. Replace dependencies on HBase deprecated API.
     (Basab Maulik via jvs)
 
-    HIVE-1513. hive starter scripts should load admin/user supplied script for 
+    HIVE-1513. hive starter scripts should load admin/user supplied script for
     configurability
     (Joydeep Sen Sarma via Ning Zhang)
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Aug 10 20:41:57 2010
@@ -71,7 +71,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
-import org.apache.hadoop.hive.ql.plan.AlterPartitionProtectModeDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
@@ -185,12 +184,6 @@ public class DDLTask extends Task<DDLWor
         return addPartition(db, addPartitionDesc);
       }
 
-      AlterPartitionProtectModeDesc alterPartitionProtectModeDesc =
-        work.getAlterPartitionProtectModeDesc();
-      if (alterPartitionProtectModeDesc != null) {
-        return alterPartitionProtectMode(db, alterPartitionProtectModeDesc);
-      }
-
       AlterTableSimpleDesc simpleDesc = work.getAlterTblSimpleDesc();
 
       if(simpleDesc != null) {
@@ -318,53 +311,6 @@ public class DDLTask extends Task<DDLWor
     return 0;
   }
 
-  private int alterPartitionProtectMode(Hive db,
-      AlterPartitionProtectModeDesc alterPartitionProtectModeDesc)
-      throws HiveException {
-
-    Table tbl = db.getTable(alterPartitionProtectModeDesc.getDbName(),
-        alterPartitionProtectModeDesc.getTableName());
-
-    validateAlterTableType(
-        tbl, AlterTableDesc.AlterTableTypes.ALTERPARTITIONPROTECTMODE);
-
-    Partition oldPart = db.getPartition(
-        tbl, alterPartitionProtectModeDesc.getPartSpec(), false);
-    if (oldPart == null) {
-      console.printError("Cannot modify protect mode of not existing partition");
-    }
-
-    ProtectMode mode = oldPart.getProtectMode();
-
-    if (alterPartitionProtectModeDesc.isProtectModeEnable() &&
-        alterPartitionProtectModeDesc.getProtectModeType() ==
-          AlterPartitionProtectModeDesc.ProtectModeType.OFFLINE) {
-      mode.offline = true;
-    } else if (alterPartitionProtectModeDesc.isProtectModeEnable() &&
-        alterPartitionProtectModeDesc.getProtectModeType() ==
-          AlterPartitionProtectModeDesc.ProtectModeType.NO_DROP) {
-      mode.noDrop = true;
-    } else if (!alterPartitionProtectModeDesc.isProtectModeEnable()&&
-        alterPartitionProtectModeDesc.getProtectModeType() ==
-          AlterPartitionProtectModeDesc.ProtectModeType.OFFLINE) {
-      mode.offline = false;
-    } else if (!alterPartitionProtectModeDesc.isProtectModeEnable() &&
-        alterPartitionProtectModeDesc.getProtectModeType() ==
-          AlterPartitionProtectModeDesc.ProtectModeType.NO_DROP) {
-      mode.noDrop = false;
-    }
-
-    oldPart.setProtectMode(mode);
-
-    try{
-      db.alterPartition(alterPartitionProtectModeDesc.getTableName(), oldPart);
-    } catch(InvalidOperationException e){
-      throw new HiveException(e);
-    }
-
-    return 0;
-  }
-
   /**
    * Rewrite the partition's metadata and force the pre/post execute hooks to
    * be fired.
@@ -1638,6 +1584,11 @@ public class DDLTask extends Task<DDLWor
     // alter the table
     Table tbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, alterTbl
         .getOldName());
+    
+    Partition part = null;
+    if(alterTbl.getPartSpec() != null) {
+      part = db.getPartition(tbl, alterTbl.getPartSpec(), false);
+    }
 
     validateAlterTableType(tbl, alterTbl.getOp());
 
@@ -1779,33 +1730,51 @@ public class DDLTask extends Task<DDLWor
       tbl.setFields(Hive.getFieldsFromDeserializer(tbl.getTableName(), tbl
           .getDeserializer()));
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ADDFILEFORMAT) {
-      tbl.getTTable().getSd().setInputFormat(alterTbl.getInputFormat());
-      tbl.getTTable().getSd().setOutputFormat(alterTbl.getOutputFormat());
-      if (alterTbl.getSerdeName() != null) {
-        tbl.setSerializationLib(alterTbl.getSerdeName());
+      if(part != null) {
+        part.getTPartition().getSd().setInputFormat(alterTbl.getInputFormat());
+        part.getTPartition().getSd().setOutputFormat(alterTbl.getOutputFormat());
+        if (alterTbl.getSerdeName() != null) {
+          part.getTPartition().getSd().getSerdeInfo().setSerializationLib(
+              alterTbl.getSerdeName());
+        }
+      } else {
+        tbl.getTTable().getSd().setInputFormat(alterTbl.getInputFormat());
+        tbl.getTTable().getSd().setOutputFormat(alterTbl.getOutputFormat());
+        if (alterTbl.getSerdeName() != null) {
+          tbl.setSerializationLib(alterTbl.getSerdeName());
+        }
       }
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERPROTECTMODE) {
-      ProtectMode mode = tbl.getProtectMode();
+      boolean protectModeEnable = alterTbl.isProtectModeEnable();
+      AlterTableDesc.ProtectModeType protectMode = alterTbl.getProtectModeType();
+
+      ProtectMode mode = null;
+      if(part != null) {
+        mode = part.getProtectMode();
+      } else {
+        mode = tbl.getProtectMode();
+      }
 
-      if (alterTbl.isProtectModeEnable() &&
-          alterTbl.getProtectModeType() ==
-            AlterTableDesc.ProtectModeType.OFFLINE) {
+      if (protectModeEnable
+          && protectMode == AlterTableDesc.ProtectModeType.OFFLINE) {
         mode.offline = true;
-      } else if (alterTbl.isProtectModeEnable() &&
-          alterTbl.getProtectModeType() ==
-            AlterTableDesc.ProtectModeType.NO_DROP) {
+      } else if (protectModeEnable
+          && protectMode == AlterTableDesc.ProtectModeType.NO_DROP) {
         mode.noDrop = true;
-      } else if (!alterTbl.isProtectModeEnable()&&
-          alterTbl.getProtectModeType() ==
-            AlterTableDesc.ProtectModeType.OFFLINE) {
+      } else if (!protectModeEnable
+          && protectMode == AlterTableDesc.ProtectModeType.OFFLINE) {
         mode.offline = false;
-      } else if (!alterTbl.isProtectModeEnable() &&
-          alterTbl.getProtectModeType() ==
-            AlterTableDesc.ProtectModeType.NO_DROP) {
+      } else if (!protectModeEnable
+          && protectMode == AlterTableDesc.ProtectModeType.NO_DROP) {
         mode.noDrop = false;
       }
 
-      tbl.setProtectMode(mode);
+      if (part != null) {
+        part.setProtectMode(mode);
+      } else {
+        tbl.setProtectMode(mode);        
+      }
+
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ADDCLUSTERSORTCOLUMN) {
       // validate sort columns and bucket columns
       List<String> columns = Utilities.getColumnNamesFromFieldSchema(tbl
@@ -1833,32 +1802,63 @@ public class DDLTask extends Task<DDLWor
       tbl.getTTable().getSd().setBucketCols(bucketCols);
       tbl.getTTable().getSd().setNumBuckets(numBuckets);
       tbl.getTTable().getSd().setSortCols(sortCols);
+    } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERLOCATION) {
+      String newLocation = alterTbl.getNewLocation();
+      try {
+        URI locURI = new URI(newLocation);
+        if (!locURI.isAbsolute() || locURI.getScheme() == null
+            || locURI.getScheme().trim().equals("")) {
+          throw new HiveException(
+              newLocation
+                  + " is not absolute or has no scheme information. "
+                  + "Please specify a complete absolute uri with scheme information.");
+        }
+        if (part != null) {
+          part.setLocation(newLocation);
+        } else {
+          tbl.setDataLocation(locURI);
+        }
+      } catch (URISyntaxException e) {
+        throw new HiveException(e);
+      }
     } else {
       console.printError("Unsupported Alter commnad");
       return 1;
     }
 
     // set last modified by properties
+    String user = null;
     try {
-      tbl.setProperty("last_modified_by", conf.getUser());
+      user = conf.getUser();
     } catch (IOException e) {
       console.printError("Unable to get current user: " + e.getMessage(),
           stringifyException(e));
       return 1;
     }
-    tbl.setProperty("last_modified_time", Long.toString(System
-        .currentTimeMillis() / 1000));
 
-    try {
-      tbl.checkValidity();
-    } catch (HiveException e) {
-      console.printError("Invalid table columns : " + e.getMessage(),
-          stringifyException(e));
-      return 1;
+    if(part == null) {
+      tbl.setProperty("last_modified_by", user);
+      tbl.setProperty("last_modified_time", Long.toString(System
+          .currentTimeMillis() / 1000));
+      try {
+        tbl.checkValidity();
+      } catch (HiveException e) {
+        console.printError("Invalid table columns : " + e.getMessage(),
+            stringifyException(e));
+        return 1;
+      }
+    } else {
+      part.getParameters().put("last_modified_by", user);
+      part.getParameters().put("last_modified_time", Long.toString(System
+          .currentTimeMillis() / 1000));
     }
-
+    
     try {
-      db.alterTable(alterTbl.getOldName(), tbl);
+      if (part == null) {
+        db.alterTable(alterTbl.getOldName(), tbl);
+      } else {
+        db.alterPartition(tbl.getTableName(), part);        
+      }
     } catch (InvalidOperationException e) {
       console.printError("Invalid alter operation: " + e.getMessage());
       LOG.info("alter table: " + stringifyException(e));
@@ -1872,8 +1872,13 @@ public class DDLTask extends Task<DDLWor
     // contains the new table. This is needed for rename - both the old and the
     // new table names are
     // passed
-    work.getInputs().add(new ReadEntity(oldTbl));
-    work.getOutputs().add(new WriteEntity(tbl));
+    if(part != null) {
+      work.getInputs().add(new ReadEntity(part));
+      work.getOutputs().add(new WriteEntity(part));
+    } else {
+      work.getInputs().add(new ReadEntity(oldTbl));
+      work.getOutputs().add(new WriteEntity(tbl));
+    }
     return 0;
   }
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Aug 10 20:41:57 2010
@@ -55,7 +55,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
-import org.apache.hadoop.hive.ql.plan.AlterPartitionProtectModeDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
@@ -108,6 +107,24 @@ public class DDLSemanticAnalyzer extends
     return TokenToTypeName.get(token);
   }
 
+  static class TablePartition {
+    String tableName;
+    HashMap<String, String> partSpec = null;
+    
+    public TablePartition(){
+    }
+    
+    public TablePartition (ASTNode tblPart) throws SemanticException {
+      tableName = unescapeIdentifier(tblPart.getChild(0).getText());
+      if (tblPart.getChildCount() > 1) {
+        ASTNode part = (ASTNode) tblPart.getChild(1);
+        if (part.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+         this.partSpec = DDLSemanticAnalyzer.getPartSpec(part);
+        }
+      }
+    }
+  }
+  
   public DDLSemanticAnalyzer(HiveConf conf) throws SemanticException {
     super(conf);
     // Partition can't have this name
@@ -120,7 +137,20 @@ public class DDLSemanticAnalyzer extends
 
   @Override
   public void analyzeInternal(ASTNode ast) throws SemanticException {
-    if (ast.getToken().getType() == HiveParser.TOK_DROPTABLE) {
+    
+    if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_PARTITION) {
+      TablePartition tblPart = new TablePartition((ASTNode)ast.getChild(0));
+      String tableName = tblPart.tableName;
+      HashMap<String, String> partSpec = tblPart.partSpec;
+      ast = (ASTNode)ast.getChild(1);
+      if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_FILEFORMAT) {
+        analyzeAlterTableFileFormat(ast, tableName, partSpec);
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE) {
+        analyzeAlterTableProtectMode(ast, tableName, partSpec);
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_LOCATION) {
+        analyzeAlterTableLocation(ast, tableName, partSpec);
+      } 
+    } else if (ast.getToken().getType() == HiveParser.TOK_DROPTABLE) {
       analyzeDropTable(ast, false);
     } else if (ast.getToken().getType() == HiveParser.TOK_CREATEINDEX) {
       analyzeCreateIndex(ast);
@@ -166,18 +196,12 @@ public class DDLSemanticAnalyzer extends
       analyzeAlterTableAddParts(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_DROPPARTS) {
       analyzeAlterTableDropParts(ast);
-    } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE) {
-      analyzeAlterTableAlterPartsProtectMode(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_PROPERTIES) {
       analyzeAlterTableProps(ast, false);
     } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES) {
       analyzeAlterTableSerdeProps(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_SERIALIZER) {
       analyzeAlterTableSerde(ast);
-    } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_FILEFORMAT) {
-      analyzeAlterTableFileFormat(ast);
-    } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_PROTECTMODE) {
-      analyzeAlterTableProtectMode(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_CLUSTER_SORT) {
       analyzeAlterTableClusterSort(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_ALTERINDEX_REBUILD) {
@@ -397,14 +421,15 @@ public class DDLSemanticAnalyzer extends
         alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableFileFormat(ASTNode ast)
+  private void analyzeAlterTableFileFormat(ASTNode ast, String tableName,
+      HashMap<String, String> partSpec)
       throws SemanticException {
-    String tableName = unescapeIdentifier(ast.getChild(0).getText());
+
     String inputFormat = null;
     String outputFormat = null;
     String storageHandler = null;
     String serde = null;
-    ASTNode child = (ASTNode) ast.getChild(1);
+    ASTNode child = (ASTNode) ast.getChild(0);
 
     switch (child.getToken().getType()) {
     case HiveParser.TOK_TABLEFILEFORMAT:
@@ -442,22 +467,34 @@ public class DDLSemanticAnalyzer extends
       serde = COLUMNAR_SERDE;
       break;
     }
+    
     AlterTableDesc alterTblDesc = new AlterTableDesc(tableName, inputFormat,
-        outputFormat, serde, storageHandler);
+        outputFormat, serde, storageHandler, partSpec);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        alterTblDesc), conf));
+  }
+
+  private void analyzeAlterTableLocation(ASTNode ast, String tableName,
+      HashMap<String, String> partSpec) throws SemanticException {
+
+    String newLocation = unescapeSQLString(ast.getChild(0).getText());
+
+    AlterTableDesc alterTblDesc = new AlterTableDesc (tableName, newLocation, partSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableProtectMode(ASTNode ast)
+  private void analyzeAlterTableProtectMode(ASTNode ast, String tableName,
+      HashMap<String, String> partSpec)
       throws SemanticException {
-    String tableName = unescapeIdentifier(ast.getChild(0).getText());
 
     AlterTableDesc alterTblDesc =
       new AlterTableDesc(AlterTableTypes.ALTERPROTECTMODE);
 
     alterTblDesc.setOldName(tableName);
+    alterTblDesc.setPartSpec(partSpec);
 
-    ASTNode child = (ASTNode) ast.getChild(1);
+    ASTNode child = (ASTNode) ast.getChild(0);
 
     switch (child.getToken().getType()) {
     case HiveParser.TOK_ENABLE:
@@ -589,7 +626,7 @@ public class DDLSemanticAnalyzer extends
     LOG.info("analyzeDescribeTable done");
   }
 
-  private HashMap<String, String> getPartSpec(ASTNode partspec)
+  private static HashMap<String, String> getPartSpec(ASTNode partspec)
       throws SemanticException {
     HashMap<String, String> partSpec = new LinkedHashMap<String, String>();
     for (int i = 0; i < partspec.getChildCount(); ++i) {
@@ -832,60 +869,6 @@ public class DDLSemanticAnalyzer extends
   }
 
   /**
-   * Alter protect mode of a table or partition
-   *
-   * @param ast
-   *          The parsed command tree.
-   * @throws SemanticException
-   *           Parsin failed
-   */
-  private void analyzeAlterTableAlterPartsProtectMode(CommonTree ast)
-      throws SemanticException {
-
-    String tblName = unescapeIdentifier(ast.getChild(0).getText());
-
-    List<Map<String, String>> partSpecs = getPartitionSpecs(ast);
-    Map<String, String> partSpec = partSpecs.get(0);
-    AlterPartitionProtectModeDesc desc = new AlterPartitionProtectModeDesc(
-        MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, partSpec);
-
-    CommonTree child = (CommonTree) ast.getChild(2);
-
-    switch (child.getToken().getType()) {
-    case HiveParser.TOK_ENABLE:
-      desc.setProtectModeEnable(true);
-      break;
-    case HiveParser.TOK_DISABLE:
-      desc.setProtectModeEnable(false);
-      break;
-    default:
-      throw new SemanticException(
-          "Set Protect mode Syntax parsing error.");
-    }
-
-    ASTNode grandChild = (ASTNode) child.getChild(0);
-    switch (grandChild.getToken().getType()) {
-    case HiveParser.TOK_OFFLINE:
-      desc.setProtectModeType(
-          AlterPartitionProtectModeDesc.ProtectModeType.OFFLINE);
-      break;
-    case HiveParser.TOK_NO_DROP:
-      desc.setProtectModeType(
-          AlterPartitionProtectModeDesc.ProtectModeType.NO_DROP);
-      break;
-    case HiveParser.TOK_READONLY:
-      throw new SemanticException(
-          "Potect mode READONLY is not implemented");
-    default:
-      throw new SemanticException(
-          "Only protect mode NO_DROP or OFFLINE supported");
-    }
-
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc
-        ), conf));
-  }
-
-  /**
    * Rewrite the metadata for one or more partitions in a table. Useful when
    * an external process modifies files on HDFS and you want the pre/post
    * hooks to be fired for the specified partition.

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Tue Aug 10 20:41:57 2010
@@ -93,6 +93,7 @@ TOK_DROPINDEX;
 TOK_LIKETABLE;
 TOK_DESCTABLE;
 TOK_DESCFUNCTION;
+TOK_ALTERTABLE_PARTITION;
 TOK_ALTERTABLE_RENAME;
 TOK_ALTERTABLE_ADDCOLS;
 TOK_ALTERTABLE_RENAMECOL;
@@ -105,9 +106,10 @@ TOK_ALTERTABLE_ARCHIVE;
 TOK_ALTERTABLE_UNARCHIVE;
 TOK_ALTERTABLE_SERDEPROPERTIES;
 TOK_ALTERTABLE_SERIALIZER;
+TOK_TABLE_PARTITION;
 TOK_ALTERTABLE_FILEFORMAT;
+TOK_ALTERTABLE_LOCATION;
 TOK_ALTERTABLE_PROPERTIES;
-TOK_ALTERTABLE_PROTECTMODE;
 TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
 TOK_ALTERINDEX_REBUILD;
 TOK_MSCK;
@@ -353,14 +355,12 @@ alterTableStatementSuffix
     | alterStatementSuffixRenameCol
     | alterStatementSuffixDropPartitions
     | alterStatementSuffixAddPartitions
-    | alterStatementSuffixAlterPartitionsProtectMode
     | alterStatementSuffixTouch
     | alterStatementSuffixArchive
     | alterStatementSuffixUnArchive
     | alterStatementSuffixProperties
     | alterStatementSuffixSerdeProperties
-    | alterStatementSuffixFileFormat
-    | alterStatementSuffixProtectMode
+    | alterTblPartitionStatement
     | alterStatementSuffixClusterbySortby
     ;
 
@@ -405,13 +405,6 @@ alterStatementSuffixAddPartitions
     -> ^(TOK_ALTERTABLE_ADDPARTS Identifier ifNotExists? (partitionSpec partitionLocation?)+)
     ;
     
-alterStatementSuffixAlterPartitionsProtectMode
-@init { msgs.push("alter partition protect mode statement"); }
-@after { msgs.pop(); }
-    : Identifier partitionSpec alterProtectMode
-    -> ^(TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE Identifier partitionSpec alterProtectMode)
-    ;
-
 alterStatementSuffixTouch
 @init { msgs.push("touch statement"); }
 @after { msgs.pop(); }
@@ -470,20 +463,49 @@ alterStatementSuffixSerdeProperties
     -> ^(TOK_ALTERTABLE_SERDEPROPERTIES $name tableProperties)
     ;
 
+tablePartitionPrefix
+@init {msgs.push("table partition prefix");}
+@after {msgs.pop();}
+  :name=Identifier partitionSpec? 
+  ->^(TOK_TABLE_PARTITION $name partitionSpec?)
+  ;
+
+alterTblPartitionStatement
+@init {msgs.push("alter table partition statement");}
+@after {msgs.pop();}
+  :  tablePartitionPrefix alterTblPartitionStatementSuffix
+  -> ^(TOK_ALTERTABLE_PARTITION tablePartitionPrefix alterTblPartitionStatementSuffix)
+  ;
+
+alterTblPartitionStatementSuffix
+@init {msgs.push("alter table partition statement suffix");}
+@after {msgs.pop();}
+  : alterStatementSuffixFileFormat
+  | alterStatementSuffixLocation
+  | alterStatementSuffixProtectMode
+  ;
+
 alterStatementSuffixFileFormat
 @init {msgs.push("alter fileformat statement"); }
-@after {msgs.pop(); }
-	:name=Identifier KW_SET KW_FILEFORMAT fileFormat
-	-> ^(TOK_ALTERTABLE_FILEFORMAT $name fileFormat)
+@after {msgs.pop();}
+	: KW_SET KW_FILEFORMAT fileFormat
+	-> ^(TOK_ALTERTABLE_FILEFORMAT fileFormat)
 	;
 
+alterStatementSuffixLocation
+@init {msgs.push("alter location");}
+@after {msgs.pop();}
+  : KW_SET KW_LOCATION newLoc=StringLiteral
+  -> ^(TOK_ALTERTABLE_LOCATION $newLoc)
+  ;
+
 alterStatementSuffixProtectMode
-@init {msgs.push("alter protectmode statement"); }
-@after {msgs.pop(); }
-	:name=Identifier alterProtectMode
-	-> ^(TOK_ALTERTABLE_PROTECTMODE $name alterProtectMode)
-	;
-	
+@init { msgs.push("alter partition protect mode statement"); }
+@after { msgs.pop(); }
+    : alterProtectMode
+    -> ^(TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE alterProtectMode)
+    ;
+
 alterProtectMode
 @init { msgs.push("protect mode specification enable"); }
 @after { msgs.pop(); }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Tue Aug 10 20:41:57 2010
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.session
 public final class SemanticAnalyzerFactory {
 
   static HashMap<Integer, String> commandType = new HashMap<Integer, String>();
+  static HashMap<Integer, String[]> tablePartitionCommandType = new HashMap<Integer, String[]>(); 
 
   static {
     commandType.put(HiveParser.TOK_EXPLAIN, "EXPLAIN");
@@ -65,16 +66,23 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_ALTERVIEW_PROPERTIES, "ALTERVIEW_PROPERTIES");
     commandType.put(HiveParser.TOK_QUERY, "QUERY");
   }
+  
+  static {
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE, 
+        new String[] { "ALTERTABLE_PROTECTMODE", "ALTERPARTITION_PROTECTMODE" });
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_FILEFORMAT,
+        new String[] { "ALTERTABLE_FILEFORMAT", "ALTERPARTITION_FILEFORMAT" });
+    tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_LOCATION,
+        new String[] { "ALTERTABLE_LOCATION", "ALTERPARTITION_LOCATION" });
+  }
+  
 
   public static BaseSemanticAnalyzer get(HiveConf conf, ASTNode tree)
       throws SemanticException {
     if (tree.getToken() == null) {
       throw new RuntimeException("Empty Syntax Tree");
     } else {
-      if (SessionState.get() != null) {
-        SessionState.get().setCommandType(
-            commandType.get(tree.getToken().getType()));
-      }
+      setSessionCommandType(commandType.get(tree.getToken().getType()));
 
       switch (tree.getToken().getType()) {
       case HiveParser.TOK_EXPLAIN:
@@ -92,7 +100,6 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_ALTERTABLE_RENAME:
       case HiveParser.TOK_ALTERTABLE_DROPPARTS:
       case HiveParser.TOK_ALTERTABLE_ADDPARTS:
-      case HiveParser.TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE:
       case HiveParser.TOK_ALTERTABLE_PROPERTIES:
       case HiveParser.TOK_ALTERTABLE_SERIALIZER:
       case HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES:
@@ -104,13 +111,21 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_SHOWPARTITIONS:
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:
-      case HiveParser.TOK_ALTERTABLE_FILEFORMAT:
-      case HiveParser.TOK_ALTERTABLE_PROTECTMODE:
       case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT:
       case HiveParser.TOK_ALTERTABLE_TOUCH:
       case HiveParser.TOK_ALTERTABLE_ARCHIVE:
       case HiveParser.TOK_ALTERTABLE_UNARCHIVE:
         return new DDLSemanticAnalyzer(conf);
+      case HiveParser.TOK_ALTERTABLE_PARTITION:
+        String commandType = null;
+        Integer type = ((ASTNode) tree.getChild(1)).getToken().getType();
+        if (tree.getChild(0).getChildCount() > 1) {
+          commandType = tablePartitionCommandType.get(type)[1];
+        } else {
+          commandType = tablePartitionCommandType.get(type)[0];
+        }
+        setSessionCommandType(commandType);
+        return new DDLSemanticAnalyzer(conf);
       case HiveParser.TOK_CREATEFUNCTION:
       case HiveParser.TOK_DROPFUNCTION:
         return new FunctionSemanticAnalyzer(conf);
@@ -120,6 +135,12 @@ public final class SemanticAnalyzerFacto
     }
   }
 
+  private static void setSessionCommandType(String commandType) {
+    if (SessionState.get() != null) {
+      SessionState.get().setCommandType(commandType);
+    }
+  }
+
   private SemanticAnalyzerFactory() {
     // prevent instantiation
   }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java Tue Aug 10 20:41:57 2010
@@ -1,128 +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.plan;
-
-import java.io.Serializable;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-/**
- * Contains the information needed to add a partition.
- */
-public class AlterPartitionProtectModeDesc extends DDLDesc implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  String tableName;
-  String dbName;
-  boolean protectModeEnable;
-  ProtectModeType protectModeType;
-
-  public static enum ProtectModeType {
-    NO_DROP, OFFLINE, READ_ONLY
-  };
-
-  LinkedHashMap<String, String> partSpec;
-
-  /**
-   * For serialization only.
-   */
-  public AlterPartitionProtectModeDesc() {
-  }
-
-  /**
-   * @param dbName
-   *          database to add to.
-   * @param tableName
-   *          table to add to.
-   * @param partSpec
-   *          partition specification.
-   * @param location
-   *          partition location, relative to table location.
-   * @param ifNotExists
-   *          if true, the partition is only added if it doesn't exist
-   */
-  public AlterPartitionProtectModeDesc(String dbName, String tableName,
-      Map<String, String> partSpec) {
-    super();
-    this.dbName = dbName;
-    this.tableName = tableName;
-    this.partSpec = new LinkedHashMap<String,String>(partSpec);
-  }
-
-  /**
-   * @return database name
-   */
-  public String getDbName() {
-    return dbName;
-  }
-
-  /**
-   * @param dbName
-   *          database name
-   */
-  public void setDbName(String dbName) {
-    this.dbName = dbName;
-  }
-
-  /**
-   * @return the table we're going to add the partitions to.
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * @param tableName
-   *          the table we're going to add the partitions to.
-   */
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  /**
-   * @return partition specification.
-   */
-  public LinkedHashMap<String, String> getPartSpec() {
-    return partSpec;
-  }
-
-  /**
-   * @param partSpec
-   *          partition specification
-   */
-  public void setPartSpec(LinkedHashMap<String, String> partSpec) {
-    this.partSpec = partSpec;
-  }
-
-  public boolean isProtectModeEnable() {
-    return protectModeEnable;
-  }
-
-  public void setProtectModeEnable(boolean protectModeEnable) {
-    this.protectModeEnable = protectModeEnable;
-  }
-
-  public ProtectModeType getProtectModeType() {
-    return protectModeType;
-  }
-
-  public void setProtectModeType(ProtectModeType protectModeType) {
-    this.protectModeType = protectModeType;
-  }
-}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java Tue Aug 10 20:41:57 2010
@@ -42,7 +42,7 @@ public class AlterTableDesc extends DDLD
   public static enum AlterTableTypes {
     RENAME, ADDCOLS, REPLACECOLS, ADDPROPS, ADDSERDE, ADDSERDEPROPS,
     ADDFILEFORMAT, ADDCLUSTERSORTCOLUMN, RENAMECOLUMN, ADDPARTITION,
-    TOUCH, ARCHIVE, UNARCHIVE, ALTERPROTECTMODE, ALTERPARTITIONPROTECTMODE,
+    TOUCH, ARCHIVE, UNARCHIVE, ALTERPROTECTMODE, ALTERPARTITIONPROTECTMODE, ALTERLOCATION,
   };
 
   public static enum ProtectModeType {
@@ -70,6 +70,8 @@ public class AlterTableDesc extends DDLD
   boolean first;
   String afterCol;
   boolean expectView;
+  HashMap<String, String> partSpec;
+  private String newLocation;
   boolean protectModeEnable;
   ProtectModeType protectModeType;
 
@@ -149,9 +151,10 @@ public class AlterTableDesc extends DDLD
    *          new table input format
    * @param outputFormat
    *          new table output format
+   * @param partSpec 
    */
   public AlterTableDesc(String name, String inputFormat, String outputFormat,
-      String serdeName, String storageHandler) {
+      String serdeName, String storageHandler, HashMap<String, String> partSpec) {
     super();
     op = AlterTableTypes.ADDFILEFORMAT;
     oldName = name;
@@ -159,6 +162,7 @@ public class AlterTableDesc extends DDLD
     this.outputFormat = outputFormat;
     this.serdeName = serdeName;
     this.storageHandler = storageHandler;
+    this.partSpec = partSpec;
   }
 
   public AlterTableDesc(String tableName, int numBuckets,
@@ -170,6 +174,14 @@ public class AlterTableDesc extends DDLD
     sortColumns = new ArrayList<Order>(sortCols);
   }
 
+  public AlterTableDesc(String tableName, String newLocation,
+      HashMap<String, String> partSpec) {
+    op = AlterTableTypes.ALTERLOCATION;
+    this.oldName = tableName;
+    this.newLocation = newLocation;
+    this.partSpec = partSpec;
+  }
+
   @Explain(displayName = "new columns")
   public List<String> getNewColsString() {
     return Utilities.getFieldSchemaString(getNewCols());
@@ -481,6 +493,34 @@ public class AlterTableDesc extends DDLD
     this.expectView = expectView;
   }
 
+  /**
+   * @return part specification
+   */
+  public HashMap<String, String> getPartSpec() {
+    return partSpec;
+  }
+
+  /**
+   * @param partSpec
+   */
+  public void setPartSpec(HashMap<String, String> partSpec) {
+    this.partSpec = partSpec;
+  }
+
+  /**
+   * @return new location
+   */
+  public String getNewLocation() {
+    return newLocation;
+  }
+
+  /**
+   * @param newLocation new location
+   */
+  public void setNewLocation(String newLocation) {
+    this.newLocation = newLocation;
+  }
+
   public boolean isProtectModeEnable() {
     return protectModeEnable;
   }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java Tue Aug 10 20:41:57 2010
@@ -43,7 +43,6 @@ public class DDLWork implements Serializ
   private ShowPartitionsDesc showPartsDesc;
   private DescTableDesc descTblDesc;
   private AddPartitionDesc addPartitionDesc;
-  private AlterPartitionProtectModeDesc alterPartitionProtectModeDesc;
   private AlterTableSimpleDesc alterTblSimpleDesc;
   private MsckDesc msckDesc;
   private ShowTableStatusDesc showTblStatusDesc;
@@ -185,17 +184,6 @@ public class DDLWork implements Serializ
   }
 
   /**
-   * @param addPartitionDesc
-   *          information about the partitions we want to add.
-   */
-  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
-      AlterPartitionProtectModeDesc alterPartitionProtectModeDesc) {
-    this(inputs, outputs);
-
-    this.alterPartitionProtectModeDesc = alterPartitionProtectModeDesc;
-  }
-
-  /**
    * @param touchDesc
    *          information about the table/partitions that we want to touch
    */
@@ -494,13 +482,4 @@ public class DDLWork implements Serializ
     this.dropIdxDesc = dropIdxDesc;
   }
 
-  public AlterPartitionProtectModeDesc getAlterPartitionProtectModeDesc() {
-    return alterPartitionProtectModeDesc;
-  }
-
-  public void setAlterPartitionProtectModeDesc(
-      AlterPartitionProtectModeDesc alterPartitionProtectModeDesc) {
-    this.alterPartitionProtectModeDesc = alterPartitionProtectModeDesc;
-  }
-
 }

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter_partition_format_loc.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter_partition_format_loc.q?rev=984204&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter_partition_format_loc.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter_partition_format_loc.q Tue Aug 10 20:41:57 2010
@@ -0,0 +1,32 @@
+create table alter_partition_format_test (key int, value string);
+desc extended alter_partition_format_test;
+
+alter table alter_partition_format_test set fileformat rcfile;
+desc extended alter_partition_format_test;
+
+alter table alter_partition_format_test set location "file:/test/test/";
+desc extended alter_partition_format_test;
+
+drop table alter_partition_format_test;
+
+--partitioned table
+create table alter_partition_format_test (key int, value string) partitioned by (ds string);
+
+alter table alter_partition_format_test add partition(ds='2010');
+desc extended alter_partition_format_test partition(ds='2010');
+
+alter table alter_partition_format_test partition(ds='2010') set fileformat rcfile;
+desc extended alter_partition_format_test partition(ds='2010');
+
+alter table alter_partition_format_test partition(ds='2010') set location "file:/test/test/ds=2010";
+desc extended alter_partition_format_test partition(ds='2010');
+
+desc extended alter_partition_format_test;
+
+alter table alter_partition_format_test set fileformat rcfile;
+desc extended alter_partition_format_test;
+
+alter table alter_partition_format_test set location "file:/test/test/";
+desc extended alter_partition_format_test;
+
+drop table alter_partition_format_test;
\ No newline at end of file

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out Tue Aug 10 20:41:57 2010
@@ -24,29 +24,31 @@ POSTHOOK: Output: default@tbl_protectmod
 PREHOOK: query: select * from tbl_protectmode3 where p='p1'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_protectmode3@p=p1
-PREHOOK: Output: file:/tmp/sdong/hive_2010-08-04_12-59-33_213_4026025765027262797/-mr-10000
+PREHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-03_048_1130501594201675939/-mr-10000
 POSTHOOK: query: select * from tbl_protectmode3 where p='p1'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_protectmode3@p=p1
-POSTHOOK: Output: file:/tmp/sdong/hive_2010-08-04_12-59-33_213_4026025765027262797/-mr-10000
+POSTHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-03_048_1130501594201675939/-mr-10000
 PREHOOK: query: select * from tbl_protectmode3 where p='p2'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_protectmode3@p=p2
-PREHOOK: Output: file:/tmp/sdong/hive_2010-08-04_12-59-33_634_5692215991604023907/-mr-10000
+PREHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-03_269_993918999524192390/-mr-10000
 POSTHOOK: query: select * from tbl_protectmode3 where p='p2'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_protectmode3@p=p2
-POSTHOOK: Output: file:/tmp/sdong/hive_2010-08-04_12-59-33_634_5692215991604023907/-mr-10000
+POSTHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-03_269_993918999524192390/-mr-10000
 PREHOOK: query: alter table tbl_protectmode3 partition (p='p1') enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERPARTITION_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode3 partition (p='p1') enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERPARTITION_PROTECTMODE
+POSTHOOK: Input: default@tbl_protectmode3@p=p1
+POSTHOOK: Output: default@tbl_protectmode3@p=p1
 PREHOOK: query: select * from tbl_protectmode3 where p='p2'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_protectmode3@p=p2
-PREHOOK: Output: file:/tmp/sdong/hive_2010-08-04_12-59-33_941_1201439944216339610/-mr-10000
+PREHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-03_610_5521102361534273246/-mr-10000
 POSTHOOK: query: select * from tbl_protectmode3 where p='p2'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_protectmode3@p=p2
-POSTHOOK: Output: file:/tmp/sdong/hive_2010-08-04_12-59-33_941_1201439944216339610/-mr-10000
+POSTHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-03_610_5521102361534273246/-mr-10000
 FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode3 Partition p=p1

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out Tue Aug 10 20:41:57 2010
@@ -50,9 +50,11 @@ POSTHOOK: Output: default@tbl_protectmod
 POSTHOOK: Lineage: tbl_protectmode5_1.col SIMPLE [(tbl_protectmode5)tbl_protectmode5.FieldSchema(name:p, type:string, comment:null), ]
 POSTHOOK: Lineage: tbl_protectmode5_1.col SIMPLE [(tbl_protectmode5)tbl_protectmode5.FieldSchema(name:p, type:string, comment:null), ]
 PREHOOK: query: alter table tbl_protectmode5 partition (p='p1') enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERPARTITION_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode5 partition (p='p1') enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERPARTITION_PROTECTMODE
+POSTHOOK: Input: default@tbl_protectmode5@p=p1
+POSTHOOK: Output: default@tbl_protectmode5@p=p1
 POSTHOOK: Lineage: tbl_protectmode5_1.col SIMPLE [(tbl_protectmode5)tbl_protectmode5.FieldSchema(name:p, type:string, comment:null), ]
 POSTHOOK: Lineage: tbl_protectmode5_1.col SIMPLE [(tbl_protectmode5)tbl_protectmode5.FieldSchema(name:p, type:string, comment:null), ]
 PREHOOK: query: insert overwrite table tbl_protectmode5_1

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part2.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part2.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part2.q.out Tue Aug 10 20:41:57 2010
@@ -22,7 +22,9 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH 
 POSTHOOK: type: LOAD
 POSTHOOK: Output: default@tbl_protectmode6@p=p1
 PREHOOK: query: alter table tbl_protectmode6 partition (p='p1') enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERPARTITION_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode6 partition (p='p1') enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERPARTITION_PROTECTMODE
+POSTHOOK: Input: default@tbl_protectmode6@p=p1
+POSTHOOK: Output: default@tbl_protectmode6@p=p1
 FAILED: Error in semantic analysis: org.apache.hadoop.hive.ql.parse.SemanticException: Query against an offline table or partition tbl_protectmode6:p=p1

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part_no_drop.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part_no_drop.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part_no_drop.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part_no_drop.q.out Tue Aug 10 20:41:57 2010
@@ -17,9 +17,11 @@ POSTHOOK: query: alter table tbl_protect
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@tbl_protectmode_no_drop@p=p1
 PREHOOK: query: alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERPARTITION_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERPARTITION_PROTECTMODE
+POSTHOOK: Input: default@tbl_protectmode_no_drop@p=p1
+POSTHOOK: Output: default@tbl_protectmode_no_drop@p=p1
 PREHOOK: query: desc extended tbl_protectmode_no_drop partition (p='p1')
 PREHOOK: type: DESCTABLE
 POSTHOOK: query: desc extended tbl_protectmode_no_drop partition (p='p1')
@@ -28,7 +30,7 @@ c1	string	
 c2	string	
 p	string	
 	 	 
-Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1280951430, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null)], location:pfile:/data/users/sdong/hive-vendor-trunk-git/build/ql/test/data/warehouse/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{PROTECT_MODE=NO_DROP, transient_lastDdlTime=1280951430})	
+Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1281469940, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{last_modified_by=njain, last_modified_time=1281469940, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1281469940})	
 PREHOOK: query: drop table tbl_protectmode_no_drop
 PREHOOK: type: DROPTABLE
 FAILED: Error in metadata: Table tbl_protectmode_no_drop Partitionp=p1 is protected from being dropped

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl1.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl1.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl1.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl1.q.out Tue Aug 10 20:41:57 2010
@@ -14,15 +14,15 @@ POSTHOOK: Output: default@tbl_protectmod
 PREHOOK: query: select * from tbl_protectmode_1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_protectmode_1
-PREHOOK: Output: file:/tmp/sdong/hive_2010-08-04_15-57-58_359_6963528649235778507/-mr-10000
+PREHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-21_331_4986102848710196881/-mr-10000
 POSTHOOK: query: select * from tbl_protectmode_1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_protectmode_1
-POSTHOOK: Output: file:/tmp/sdong/hive_2010-08-04_15-57-58_359_6963528649235778507/-mr-10000
+POSTHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-21_331_4986102848710196881/-mr-10000
 PREHOOK: query: alter table tbl_protectmode_1 enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_1 enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_1
 POSTHOOK: Output: default@tbl_protectmode_1
 FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode_1

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl2.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl2.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl2.q.out Tue Aug 10 20:41:57 2010
@@ -17,21 +17,21 @@ POSTHOOK: query: alter table tbl_protect
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@tbl_protectmode2@p=p1
 PREHOOK: query: alter table tbl_protectmode2 enable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode2 enable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode2
 POSTHOOK: Output: default@tbl_protectmode2
 PREHOOK: query: alter table tbl_protectmode2 enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode2 enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode2
 POSTHOOK: Output: default@tbl_protectmode2
 PREHOOK: query: alter table tbl_protectmode2 disable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode2 disable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode2
 POSTHOOK: Output: default@tbl_protectmode2
 PREHOOK: query: desc extended tbl_protectmode2
@@ -41,5 +41,5 @@ POSTHOOK: type: DESCTABLE
 col	string	
 p	string	
 	 	 
-Detailed Table Information	Table(tableName:tbl_protectmode2, dbName:default, owner:sdong, createTime:1280964330, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/sdong/hive-vendor-trunk-git/build/ql/test/data/warehouse/tbl_protectmode2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=sdong, last_modified_time=1280964330, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1280964330}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:tbl_protectmode2, dbName:default, owner:njain, createTime:1281469941, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=njain, last_modified_time=1281469942, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1281469942}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
 FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode2

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl3.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl3.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl3.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl3.q.out Tue Aug 10 20:41:57 2010
@@ -14,15 +14,15 @@ POSTHOOK: Output: default@tbl_protectmod
 PREHOOK: query: select col from tbl_protectmode_4
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_protectmode_4
-PREHOOK: Output: file:/tmp/sdong/hive_2010-08-04_15-57-59_668_7963898681822155966/-mr-10000
+PREHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-22_797_804698083585859425/-mr-10000
 POSTHOOK: query: select col from tbl_protectmode_4
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_protectmode_4
-POSTHOOK: Output: file:/tmp/sdong/hive_2010-08-04_15-57-59_668_7963898681822155966/-mr-10000
+POSTHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-22_797_804698083585859425/-mr-10000
 PREHOOK: query: alter table tbl_protectmode_4 enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_4 enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_4
 POSTHOOK: Output: default@tbl_protectmode_4
 PREHOOK: query: desc extended tbl_protectmode_4
@@ -31,5 +31,5 @@ POSTHOOK: query: desc extended tbl_prote
 POSTHOOK: type: DESCTABLE
 col	string	
 	 	 
-Detailed Table Information	Table(tableName:tbl_protectmode_4, dbName:default, owner:sdong, createTime:1280962679, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/sdong/hive-vendor-trunk-git/build/ql/test/data/warehouse/tbl_protectmode_4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{last_modified_by=sdong, last_modified_time=1280962682, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1280962682}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:tbl_protectmode_4, dbName:default, owner:njain, createTime:1281469942, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode_4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{last_modified_by=njain, last_modified_time=1281469945, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1281469945}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
 FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode_4

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl4.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl4.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl4.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl4.q.out Tue Aug 10 20:41:57 2010
@@ -26,21 +26,21 @@ POSTHOOK: query: alter table tbl_protect
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@tbl_protectmode_tbl4@p=p1
 PREHOOK: query: alter table tbl_protectmode_tbl4 enable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_tbl4 enable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_tbl4
 POSTHOOK: Output: default@tbl_protectmode_tbl4
 PREHOOK: query: alter table tbl_protectmode_tbl4 enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_tbl4 enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_tbl4
 POSTHOOK: Output: default@tbl_protectmode_tbl4
 PREHOOK: query: alter table tbl_protectmode_tbl4 disable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_tbl4 disable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_tbl4
 POSTHOOK: Output: default@tbl_protectmode_tbl4
 PREHOOK: query: desc extended tbl_protectmode_tbl4
@@ -50,5 +50,5 @@ POSTHOOK: type: DESCTABLE
 col	string	
 p	string	
 	 	 
-Detailed Table Information	Table(tableName:tbl_protectmode_tbl4, dbName:default, owner:njain, createTime:1280986404, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode_tbl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=njain, last_modified_time=1280986404, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1280986404}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:tbl_protectmode_tbl4, dbName:default, owner:njain, createTime:1281469946, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode_tbl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=njain, last_modified_time=1281469946, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1281469946}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
 FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode_tbl4

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl5.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl5.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl5.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl5.q.out Tue Aug 10 20:41:57 2010
@@ -26,21 +26,21 @@ POSTHOOK: query: alter table tbl_protect
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@tbl_protectmode_tbl5@p=p1
 PREHOOK: query: alter table tbl_protectmode_tbl5 enable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_tbl5 enable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_tbl5
 POSTHOOK: Output: default@tbl_protectmode_tbl5
 PREHOOK: query: alter table tbl_protectmode_tbl5 enable offline
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_tbl5 enable offline
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_tbl5
 POSTHOOK: Output: default@tbl_protectmode_tbl5
 PREHOOK: query: alter table tbl_protectmode_tbl5 disable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode_tbl5 disable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode_tbl5
 POSTHOOK: Output: default@tbl_protectmode_tbl5
 PREHOOK: query: desc extended tbl_protectmode_tbl5
@@ -50,5 +50,5 @@ POSTHOOK: type: DESCTABLE
 col	string	
 p	string	
 	 	 
-Detailed Table Information	Table(tableName:tbl_protectmode_tbl5, dbName:default, owner:njain, createTime:1280986496, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode_tbl5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=njain, last_modified_time=1280986496, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1280986496}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:tbl_protectmode_tbl5, dbName:default, owner:njain, createTime:1281469947, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode_tbl5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=njain, last_modified_time=1281469948, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1281469948}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
 FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode_tbl5

Modified: hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl_no_drop.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl_no_drop.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl_no_drop.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_tbl_no_drop.q.out Tue Aug 10 20:41:57 2010
@@ -14,15 +14,15 @@ POSTHOOK: Output: default@tbl_protectmod
 PREHOOK: query: select * from tbl_protectmode__no_drop
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tbl_protectmode__no_drop
-PREHOOK: Output: file:/tmp/sdong/hive_2010-08-03_19-02-13_062_1358034933662934792/-mr-10000
+PREHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-29_070_3321305319848064834/-mr-10000
 POSTHOOK: query: select * from tbl_protectmode__no_drop
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@tbl_protectmode__no_drop
-POSTHOOK: Output: file:/tmp/sdong/hive_2010-08-03_19-02-13_062_1358034933662934792/-mr-10000
+POSTHOOK: Output: file:/tmp/njain/hive_2010-08-10_12-52-29_070_3321305319848064834/-mr-10000
 PREHOOK: query: alter table tbl_protectmode__no_drop enable no_drop
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: query: alter table tbl_protectmode__no_drop enable no_drop
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_PROTECTMODE
 POSTHOOK: Input: default@tbl_protectmode__no_drop
 POSTHOOK: Output: default@tbl_protectmode__no_drop
 PREHOOK: query: desc extended tbl_protectmode__no_drop
@@ -31,7 +31,7 @@ POSTHOOK: query: desc extended tbl_prote
 POSTHOOK: type: DESCTABLE
 col	string	
 	 	 
-Detailed Table Information	Table(tableName:tbl_protectmode__no_drop, dbName:default, owner:sdong, createTime:1280887333, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/sdong/hive-vendor-trunk-git/build/ql/test/data/warehouse/tbl_protectmode__no_drop, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{last_modified_by=sdong, last_modified_time=1280887333, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1280887333}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:tbl_protectmode__no_drop, dbName:default, owner:njain, createTime:1281469949, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:pfile:/data/users/njain/hive_commit2/hive_commit2/build/ql/test/data/warehouse/tbl_protectmode__no_drop, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{last_modified_by=njain, last_modified_time=1281469949, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1281469949}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
 PREHOOK: query: drop table tbl_protectmode__no_drop
 PREHOOK: type: DROPTABLE
 FAILED: Error in metadata: Table tbl_protectmode__no_drop is protected from being dropped

Added: hadoop/hive/trunk/ql/src/test/results/clientpositive/alter_partition_format_loc.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/alter_partition_format_loc.q.out?rev=984204&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/alter_partition_format_loc.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/alter_partition_format_loc.q.out Tue Aug 10 20:41:57 2010
@@ -0,0 +1,141 @@
+PREHOOK: query: create table alter_partition_format_test (key int, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table alter_partition_format_test (key int, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: desc extended alter_partition_format_test
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test
+POSTHOOK: type: DESCTABLE
+key	int	
+value	string	
+	 	 
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:heyongqiang, createTime:1281461436, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-2/build/ql/test/data/warehouse/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{transient_lastDdlTime=1281461436}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+PREHOOK: query: alter table alter_partition_format_test set fileformat rcfile
+PREHOOK: type: ALTERTABLE_FILEFORMAT
+POSTHOOK: query: alter table alter_partition_format_test set fileformat rcfile
+POSTHOOK: type: ALTERTABLE_FILEFORMAT
+POSTHOOK: Input: default@alter_partition_format_test
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: desc extended alter_partition_format_test
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test
+POSTHOOK: type: DESCTABLE
+key	int	from deserializer
+value	string	from deserializer
+	 	 
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:heyongqiang, createTime:1281461436, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-2/build/ql/test/data/warehouse/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{last_modified_by=heyongqiang, last_modified_time=1281461436, transient_lastDdlTime=1281461436}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+PREHOOK: query: alter table alter_partition_format_test set location "file:/test/test/"
+PREHOOK: type: ALTERTABLE_LOCATION
+POSTHOOK: query: alter table alter_partition_format_test set location "file:/test/test/"
+POSTHOOK: type: ALTERTABLE_LOCATION
+POSTHOOK: Input: default@alter_partition_format_test
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: desc extended alter_partition_format_test
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test
+POSTHOOK: type: DESCTABLE
+key	int	from deserializer
+value	string	from deserializer
+	 	 
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:heyongqiang, createTime:1281461436, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[], parameters:{last_modified_by=heyongqiang, last_modified_time=1281461436, transient_lastDdlTime=1281461436}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+PREHOOK: query: drop table alter_partition_format_test
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table alter_partition_format_test
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: --partitioned table
+create table alter_partition_format_test (key int, value string) partitioned by (ds string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: --partitioned table
+create table alter_partition_format_test (key int, value string) partitioned by (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: alter table alter_partition_format_test add partition(ds='2010')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: alter table alter_partition_format_test add partition(ds='2010')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@alter_partition_format_test@ds=2010
+PREHOOK: query: desc extended alter_partition_format_test partition(ds='2010')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test partition(ds='2010')
+POSTHOOK: type: DESCTABLE
+key	int	
+value	string	
+ds	string	
+	 	 
+Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1281461437, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-2/build/ql/test/data/warehouse/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{transient_lastDdlTime=1281461437})	
+PREHOOK: query: alter table alter_partition_format_test partition(ds='2010') set fileformat rcfile
+PREHOOK: type: ALTERPARTITION_FILEFORMAT
+POSTHOOK: query: alter table alter_partition_format_test partition(ds='2010') set fileformat rcfile
+POSTHOOK: type: ALTERPARTITION_FILEFORMAT
+POSTHOOK: Input: default@alter_partition_format_test@ds=2010
+POSTHOOK: Output: default@alter_partition_format_test@ds=2010
+PREHOOK: query: desc extended alter_partition_format_test partition(ds='2010')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test partition(ds='2010')
+POSTHOOK: type: DESCTABLE
+key	int	
+value	string	
+ds	string	
+	 	 
+Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1281461437, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-2/build/ql/test/data/warehouse/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{last_modified_by=heyongqiang, last_modified_time=1281461437, transient_lastDdlTime=1281461437})	
+PREHOOK: query: alter table alter_partition_format_test partition(ds='2010') set location "file:/test/test/ds=2010"
+PREHOOK: type: ALTERPARTITION_LOCATION
+POSTHOOK: query: alter table alter_partition_format_test partition(ds='2010') set location "file:/test/test/ds=2010"
+POSTHOOK: type: ALTERPARTITION_LOCATION
+POSTHOOK: Input: default@alter_partition_format_test@ds=2010
+POSTHOOK: Output: default@alter_partition_format_test@ds=2010
+PREHOOK: query: desc extended alter_partition_format_test partition(ds='2010')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test partition(ds='2010')
+POSTHOOK: type: DESCTABLE
+key	int	
+value	string	
+ds	string	
+	 	 
+Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1281461437, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), parameters:{last_modified_by=heyongqiang, last_modified_time=1281461438, transient_lastDdlTime=1281461438})	
+PREHOOK: query: desc extended alter_partition_format_test
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test
+POSTHOOK: type: DESCTABLE
+key	int	
+value	string	
+ds	string	
+	 	 
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:heyongqiang, createTime:1281461437, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-2/build/ql/test/data/warehouse/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1281461437}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+PREHOOK: query: alter table alter_partition_format_test set fileformat rcfile
+PREHOOK: type: ALTERTABLE_FILEFORMAT
+POSTHOOK: query: alter table alter_partition_format_test set fileformat rcfile
+POSTHOOK: type: ALTERTABLE_FILEFORMAT
+POSTHOOK: Input: default@alter_partition_format_test
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: desc extended alter_partition_format_test
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test
+POSTHOOK: type: DESCTABLE
+key	int	from deserializer
+value	string	from deserializer
+ds	string	
+	 	 
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:heyongqiang, createTime:1281461437, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:pfile:/Users/heyongqiang/Documents/workspace/Hive-2/build/ql/test/data/warehouse/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=heyongqiang, last_modified_time=1281461438, transient_lastDdlTime=1281461438}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
 	
+PREHOOK: query: alter table alter_partition_format_test set location "file:/test/test/"
+PREHOOK: type: ALTERTABLE_LOCATION
+POSTHOOK: query: alter table alter_partition_format_test set location "file:/test/test/"
+POSTHOOK: type: ALTERTABLE_LOCATION
+POSTHOOK: Input: default@alter_partition_format_test
+POSTHOOK: Output: default@alter_partition_format_test
+PREHOOK: query: desc extended alter_partition_format_test
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc extended alter_partition_format_test
+POSTHOOK: type: DESCTABLE
+key	int	from deserializer
+value	string	from deserializer
+ds	string	
+	 	 
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:heyongqiang, createTime:1281461437, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=heyongqiang, last_modified_time=1281461438, transient_lastDdlTime=1281461438}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+PREHOOK: query: drop table alter_partition_format_test
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table alter_partition_format_test
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: default@alter_partition_format_test

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/diff_part_input_formats.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/diff_part_input_formats.q.out?rev=984204&r1=984203&r2=984204&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/diff_part_input_formats.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/diff_part_input_formats.q.out Tue Aug 10 20:41:57 2010
@@ -15,9 +15,9 @@ POSTHOOK: query: ALTER TABLE part_test A
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@part_test@ds=1
 PREHOOK: query: ALTER TABLE part_test SET FILEFORMAT RCFILE
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_FILEFORMAT
 POSTHOOK: query: ALTER TABLE part_test SET FILEFORMAT RCFILE
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_FILEFORMAT
 POSTHOOK: Input: default@part_test
 POSTHOOK: Output: default@part_test
 PREHOOK: query: ALTER TABLE part_test ADD PARTITION(ds='2')
@@ -27,8 +27,8 @@ POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@part_test@ds=2
 PREHOOK: query: SELECT count(1) FROM part_test WHERE ds='3'
 PREHOOK: type: QUERY
-PREHOOK: Output: file:/tmp/jssarma/hive_2010-07-21_11-21-20_502_7998672376160685808/10000
+PREHOOK: Output: file:/var/folders/6g/6grtCwPMEf4sqHUPpy6xQG9ByHg/-Tmp-/heyongqiang/hive_2010-08-07_00-57-45_916_425825309143884801/-mr-10000
 POSTHOOK: query: SELECT count(1) FROM part_test WHERE ds='3'
 POSTHOOK: type: QUERY
-POSTHOOK: Output: file:/tmp/jssarma/hive_2010-07-21_11-21-20_502_7998672376160685808/10000
+POSTHOOK: Output: file:/var/folders/6g/6grtCwPMEf4sqHUPpy6xQG9ByHg/-Tmp-/heyongqiang/hive_2010-08-07_00-57-45_916_425825309143884801/-mr-10000
 0