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/05 07:36:50 UTC

svn commit: r982476 [1/2] - in /hadoop/hive/trunk: ./ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/metadata/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/j...

Author: namit
Date: Thu Aug  5 05:36:48 2010
New Revision: 982476

URL: http://svn.apache.org/viewvc?rev=982476&view=rev
Log:
HIVE-1413. Ability to take a table offline
(Siying Dong via namit)


Added:
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part1.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part2.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl1.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl2.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl3.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl4.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl5.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/protectmode.q
    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/protectmode.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/metadata/Partition.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.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/ErrorMsg.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/LoadSemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    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/AlterTableDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=982476&r1=982475&r2=982476&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Thu Aug  5 05:36:48 2010
@@ -33,6 +33,9 @@ Trunk -  Unreleased
     HIVE-1414. Automatically invoke .hiverc init script
     (Edward Capriolo via jvs)
 
+    HIVE-1413. Ability to take a table offline
+    (Siying Dong via namit)
+
   IMPROVEMENTS
 
     HIVE-1394. Do not update transient_lastDdlTime if the partition is modified by a housekeeping

Added: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java (added)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java Thu Aug  5 05:36:48 2010
@@ -0,0 +1,83 @@
+/**
+ * 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;
+
+
+public class ProtectMode {
+  public static String PARAMETER_NAME = "PROTECT_MODE";
+
+  public static String FLAG_OFFLINE = "OFFLINE";
+  public static String FLAG_NO_DROP = "NO_DROP";
+  public static String FLAG_READ_ONLY = "READ_ONLY";
+
+  public boolean offline = false;
+  public boolean readOnly = false;
+  public boolean noDrop = false;
+
+  static public ProtectMode getProtectModeFromString(String sourceString) {
+    return new ProtectMode(sourceString);
+  }
+
+  private ProtectMode(String sourceString) {
+    String[] tokens = sourceString.split(",");
+    for (String token: tokens) {
+      if (token.equalsIgnoreCase(FLAG_OFFLINE)) {
+        offline = true;
+      } else if (token.equalsIgnoreCase(FLAG_NO_DROP)) {
+        noDrop = true;
+      } else if (token.equalsIgnoreCase(FLAG_READ_ONLY)) {
+        readOnly = true;
+      }
+    }
+  }
+
+  public ProtectMode() {
+  }
+
+  @Override
+  public String toString() {
+    String retString = null;
+
+    if (offline) {
+        retString = FLAG_OFFLINE;
+    }
+
+    if (noDrop) {
+      if (retString != null) {
+        retString = retString + "," + FLAG_NO_DROP;
+      }
+      else
+      {
+        retString = FLAG_NO_DROP;
+      }
+    }
+
+    if (readOnly) {
+      if (retString != null) {
+        retString = retString + "," + FLAG_READ_ONLY;
+      }
+      else
+      {
+        retString = FLAG_READ_ONLY;
+      }
+    }
+
+    return retString;
+  }
+}

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=982476&r1=982475&r2=982476&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 Thu Aug  5 05:36:48 2010
@@ -50,6 +50,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.ProtectMode;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -70,8 +71,10 @@ 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;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
 import org.apache.hadoop.hive.ql.plan.CreateViewDesc;
@@ -85,7 +88,6 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowPartitionsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
-import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.serde.Constants;
@@ -151,7 +153,7 @@ public class DDLTask extends Task<DDLWor
       if (crtIndex != null) {
         return createIndex(db, crtIndex);
       }
-      
+
       DropIndexDesc dropIdx = work.getDropIdxDesc();
       if(dropIdx != null) {
         return dropIndex(db, dropIdx);
@@ -183,6 +185,12 @@ 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) {
@@ -248,7 +256,7 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int dropIndex(Hive db, DropIndexDesc dropIdx) throws HiveException {
-    db.dropIndex(MetaStoreUtils.DEFAULT_DATABASE_NAME, dropIdx.getTableName(), 
+    db.dropIndex(MetaStoreUtils.DEFAULT_DATABASE_NAME, dropIdx.getTableName(),
         dropIdx.getIndexName(), true);
     return 0;
   }
@@ -261,9 +269,9 @@ public class DDLTask extends Task<DDLWor
 
     db
         .createIndex(
-        crtIndex.getTableName(), crtIndex.getIndexName(), crtIndex.getIndexTypeHandlerClass(), 
+        crtIndex.getTableName(), crtIndex.getIndexName(), crtIndex.getIndexTypeHandlerClass(),
         crtIndex.getIndexedCols(), crtIndex.getIndexTableName(), crtIndex.getDeferredRebuild(),
-        crtIndex.getInputFormat(), crtIndex.getOutputFormat(), crtIndex.getSerde(), 
+        crtIndex.getInputFormat(), crtIndex.getOutputFormat(), crtIndex.getSerde(),
         crtIndex.getStorageHandler(), crtIndex.getLocation(), crtIndex.getIdxProps(), crtIndex.getSerdeProps(),
         crtIndex.getCollItemDelim(), crtIndex.getFieldDelim(), crtIndex.getFieldEscape(),
         crtIndex.getLineDelim(), crtIndex.getMapKeyDelim()
@@ -310,6 +318,53 @@ 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.
@@ -1729,6 +1784,28 @@ public class DDLTask extends Task<DDLWor
       if (alterTbl.getSerdeName() != null) {
         tbl.setSerializationLib(alterTbl.getSerdeName());
       }
+    } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERPROTECTMODE) {
+      ProtectMode mode = tbl.getProtectMode();
+
+      if (alterTbl.isProtectModeEnable() &&
+          alterTbl.getProtectModeType() ==
+            AlterTableDesc.ProtectModeType.OFFLINE) {
+        mode.offline = true;
+      } else if (alterTbl.isProtectModeEnable() &&
+          alterTbl.getProtectModeType() ==
+            AlterTableDesc.ProtectModeType.NO_DROP) {
+        mode.noDrop = true;
+      } else if (!alterTbl.isProtectModeEnable()&&
+          alterTbl.getProtectModeType() ==
+            AlterTableDesc.ProtectModeType.OFFLINE) {
+        mode.offline = false;
+      } else if (!alterTbl.isProtectModeEnable() &&
+          alterTbl.getProtectModeType() ==
+            AlterTableDesc.ProtectModeType.NO_DROP) {
+        mode.noDrop = false;
+      }
+
+      tbl.setProtectMode(mode);
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ADDCLUSTERSORTCOLUMN) {
       // validate sort columns and bucket columns
       List<String> columns = Utilities.getColumnNamesFromFieldSchema(tbl
@@ -1819,6 +1896,10 @@ public class DDLTask extends Task<DDLWor
     try {
       tbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, dropTbl
           .getTableName());
+      if (!tbl.canDrop()) {
+        throw new HiveException("Table " + tbl.getTableName() +
+            " is protected from being dropped");
+      }
     } catch (InvalidTableException e) {
       // drop table is idempotent
     }
@@ -1836,6 +1917,18 @@ public class DDLTask extends Task<DDLWor
     }
 
     if (dropTbl.getPartSpecs() == null) {
+      // We should check that all the partitions of the table can be dropped
+      if (tbl != null && tbl.isPartitioned()) {
+        List<Partition> listPartitions = db.getPartitions(tbl);
+        for (Partition p: listPartitions) {
+            if (!p.canDrop()) {
+              throw new HiveException("Table " + tbl.getTableName() +
+                  " Partition" + p.getName() +
+                  " is protected from being dropped");
+            }
+        }
+      }
+
       // drop the table
       db
           .dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, dropTbl
@@ -1848,6 +1941,7 @@ public class DDLTask extends Task<DDLWor
       List<String> partitionNames = db.getPartitionNames(
           MetaStoreUtils.DEFAULT_DATABASE_NAME, dropTbl.getTableName(),
           (short) -1);
+
       Set<Map<String, String>> partitions = new HashSet<Map<String, String>>();
       for (int i = 0; i < partitionNames.size(); i++) {
         try {
@@ -1872,7 +1966,14 @@ public class DDLTask extends Task<DDLWor
             }
           }
           if (match) {
-            partsToDelete.add(db.getPartition(tbl, part, false));
+            Partition p = db.getPartition(tbl, part, false);
+            if (!p.canDrop()) {
+              throw new HiveException("Table " + tbl.getTableName() +
+                  " Partition " + p.getName() +
+                  " is protected from being dropped");
+            }
+
+            partsToDelete.add(p);
             it.remove();
           }
         }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java?rev=982476&r1=982475&r2=982476&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java Thu Aug  5 05:36:48 2010
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.ProtectMode;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -471,4 +472,52 @@ public class Partition implements Serial
   public void setLocation(String location) {
     tPartition.getSd().setLocation(location);
   }
+
+  /**
+   * @param protectMode
+   */
+  public void setProtectMode(ProtectMode protectMode){
+    Map<String, String> parameters = tPartition.getParameters();
+    parameters.put(ProtectMode.PARAMETER_NAME, protectMode.toString());
+    tPartition.setParameters(parameters);
+  }
+
+  /**
+   * @return protect mode
+   */
+  public ProtectMode getProtectMode(){
+    Map<String, String> parameters = tPartition.getParameters();
+
+    if (!parameters.containsKey(ProtectMode.PARAMETER_NAME)) {
+      return new ProtectMode();
+    } else {
+      return ProtectMode.getProtectModeFromString(
+          parameters.get(ProtectMode.PARAMETER_NAME));
+    }
+  }
+
+  /**
+   * @return True protect mode indicates the partition if offline.
+   */
+  public boolean isOffline(){
+    return getProtectMode().offline;
+  }
+
+  /**
+   * @return True if protect mode attribute of the partition indicate
+   * that it is OK to drop the table
+   */
+  public boolean canDrop() {
+    ProtectMode mode = getProtectMode();
+    return (!mode.noDrop && !mode.offline && !mode.readOnly);
+  }
+
+  /**
+   * @return True if protect mode attribute of the partition indicate
+   * that it is OK to write to the table
+   */
+  public boolean canWrite() {
+    ProtectMode mode = getProtectMode();
+    return (!mode.offline && !mode.readOnly);
+  }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java?rev=982476&r1=982475&r2=982476&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java Thu Aug  5 05:36:48 2010
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.ProtectMode;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -58,7 +59,7 @@ import org.apache.hadoop.mapred.Sequence
 
 /**
  * A Hive Table: is a fundamental unit of data in Hive that shares a common schema/DDL.
- * 
+ *
  * Please note that the ql code should always go through methods of this class to access the
  * metadata, instead of directly accessing org.apache.hadoop.hive.metastore.api.Table.  This
  * helps to isolate the metastore code and the ql code.
@@ -79,7 +80,7 @@ public class Table implements Serializab
   private Class<? extends InputFormat> inputFormatClass;
   private URI uri;
   private HiveStorageHandler storageHandler;
-  
+
   /**
    * Used only for serialization.
    */
@@ -108,14 +109,14 @@ public class Table implements Serializab
   public org.apache.hadoop.hive.metastore.api.Table getTTable() {
     return tTable;
   }
-  
+
   /**
    * This function should only be called by Java serialization.
    */
   public void setTTable(org.apache.hadoop.hive.metastore.api.Table tTable) {
     this.tTable = tTable;
   }
-  
+
   /**
    * Initialize an emtpy table.
    */
@@ -136,7 +137,7 @@ public class Table implements Serializab
       sd.setInputFormat(SequenceFileInputFormat.class.getName());
       sd.setOutputFormat(HiveSequenceFileOutputFormat.class.getName());
     }
-    
+
     org.apache.hadoop.hive.metastore.api.Table t = new org.apache.hadoop.hive.metastore.api.Table();
     {
       t.setSd(sd);
@@ -179,7 +180,7 @@ public class Table implements Serializab
       assert(getViewOriginalText() == null);
       assert(getViewExpandedText() == null);
     }
-    
+
     Iterator<FieldSchema> iterCols = getCols().iterator();
     List<String> colNames = new ArrayList<String>();
     while (iterCols.hasNext()) {
@@ -246,7 +247,7 @@ public class Table implements Serializab
   }
 
   final public Deserializer getDeserializer() {
-    if (deserializer == null) { 
+    if (deserializer == null) {
       try {
         deserializer = MetaStoreUtils.getDeserializer(Hive.get().getConf(), tTable);
       } catch (MetaException e) {
@@ -290,12 +291,12 @@ public class Table implements Serializab
         throw new RuntimeException(e);
       }
     }
-    return inputFormatClass; 
+    return inputFormatClass;
   }
 
   final public Class<? extends HiveOutputFormat> getOutputFormatClass() {
     // Replace FileOutputFormat for backward compatibility
-    
+
     if (outputFormatClass == null) {
       try {
         String className = tTable.getSd().getOutputFormat();
@@ -490,7 +491,7 @@ public class Table implements Serializab
   /**
    * Returns a list of all the columns of the table (data columns + partition
    * columns in that order.
-   * 
+   *
    * @return List<FieldSchema>
    */
   public List<FieldSchema> getAllCols() {
@@ -515,7 +516,7 @@ public class Table implements Serializab
   /**
    * Replaces files in the partition with new data set specified by srcf. Works
    * by moving files
-   * 
+   *
    * @param srcf
    *          Files to be replaced. Leaf directories or globbed file paths
    * @param tmpd
@@ -533,7 +534,7 @@ public class Table implements Serializab
 
   /**
    * Inserts files specified into the partition. Works by moving files
-   * 
+   *
    * @param srcf
    *          Files to be moved. Leaf directories or globbed file paths
    */
@@ -662,15 +663,15 @@ public class Table implements Serializab
   public void setTableName(String tableName) {
     tTable.setTableName(tableName);
   }
-  
+
   public void setDbName(String databaseName) {
     tTable.setDbName(databaseName);
   }
-  
+
   public List<FieldSchema> getPartitionKeys() {
     return tTable.getPartitionKeys();
   }
-  
+
   /**
    * @return the original view text, or null if this table is not a view
    */
@@ -713,7 +714,7 @@ public class Table implements Serializab
 
   /**
    * Creates a partition name -> value spec map object
-   * 
+   *
    * @param tp
    *          Use the information from this partition.
    * @return Partition name to value mapping.
@@ -735,7 +736,7 @@ public class Table implements Serializab
   public Table copy() throws HiveException {
     return new Table(tTable.clone());
   }
-  
+
   public void setCreateTime(int createTime) {
     tTable.setCreateTime(createTime);
   }
@@ -745,4 +746,52 @@ public class Table implements Serializab
       org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_STORAGE)
       != null;
   }
+
+  /**
+   * @param protectMode
+   */
+  public void setProtectMode(ProtectMode protectMode){
+    Map<String, String> parameters = tTable.getParameters();
+    parameters.put(ProtectMode.PARAMETER_NAME, protectMode.toString());
+    tTable.setParameters(parameters);
+  }
+
+  /**
+   * @return protect mode
+   */
+  public ProtectMode getProtectMode(){
+    Map<String, String> parameters = tTable.getParameters();
+
+    if (!parameters.containsKey(ProtectMode.PARAMETER_NAME)) {
+      return new ProtectMode();
+    } else {
+      return ProtectMode.getProtectModeFromString(
+          parameters.get(ProtectMode.PARAMETER_NAME));
+    }
+  }
+
+  /**
+   * @return True protect mode indicates the table if offline.
+   */
+  public boolean isOffline(){
+    return getProtectMode().offline;
+  }
+
+  /**
+   * @return True if protect mode attribute of the partition indicate
+   * that it is OK to drop the partition
+   */
+  public boolean canDrop() {
+    ProtectMode mode = getProtectMode();
+    return (!mode.noDrop && !mode.offline && !mode.readOnly);
+  }
+
+  /**
+   * @return True if protect mode attribute of the table indicate
+   * that it is OK to write the table
+   */
+  public boolean canWrite() {
+    ProtectMode mode = getProtectMode();
+    return (!mode.offline && !mode.readOnly);
+  }
 };

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=982476&r1=982475&r2=982476&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 Thu Aug  5 05:36:48 2010
@@ -55,6 +55,7 @@ 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;
@@ -165,6 +166,8 @@ 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) {
@@ -173,6 +176,8 @@ public class DDLSemanticAnalyzer extends
       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) {
@@ -198,7 +203,7 @@ public class DDLSemanticAnalyzer extends
     String typeName = unescapeSQLString(ast.getChild(1).getText());
     String tableName = unescapeIdentifier(ast.getChild(2).getText());
     List<String> indexedCols = getColumnNames((ASTNode) ast.getChild(3));
-    
+
     IndexType indexType = HiveIndex.getIndexType(typeName);
     if (indexType != null) {
       typeName = indexType.getHandlerClsName();
@@ -209,16 +214,16 @@ public class DDLSemanticAnalyzer extends
         throw new SemanticException("class name provided for index handler not found.", e);
       }
     }
-    
+
     String indexTableName = null;
     boolean deferredRebuild = false;
     String location = null;
     Map<String, String> idxProps = null;
-    
+
     RowFormatParams rowFormatParams = new RowFormatParams();
     StorageFormat storageFormat = new StorageFormat();
     AnalyzeCreateCommonVars shared = new AnalyzeCreateCommonVars();
-    
+
     for (int idx = 4; idx < ast.getChildCount(); idx++) {
       ASTNode child = (ASTNode) ast.getChild(idx);
       if (storageFormat.fillStorageFormat(child, shared)) {
@@ -251,9 +256,9 @@ public class DDLSemanticAnalyzer extends
         break;
       }
     }
-    
+
     storageFormat.fillDefaultStorageFormat(shared);
-    
+
     CreateIndexDesc crtIndexDesc = new CreateIndexDesc(tableName, indexName,
         indexedCols, indexTableName, deferredRebuild, storageFormat.inputFormat, storageFormat.outputFormat,
         storageFormat.storageHandler, typeName, location, idxProps,
@@ -263,7 +268,7 @@ public class DDLSemanticAnalyzer extends
     Task<?> createIndex = TaskFactory.get(new DDLWork(crtIndexDesc), conf);
     rootTasks.add(createIndex);
   }
-  
+
   private void analyzeDropIndex(ASTNode ast) {
     String indexName = unescapeIdentifier(ast.getChild(0).getText());
     String tableName = unescapeIdentifier(ast.getChild(1).getText());
@@ -271,18 +276,19 @@ public class DDLSemanticAnalyzer extends
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         dropIdxDesc), conf));
   }
-      
+
   private void analyzeUpdateIndex(ASTNode ast) throws SemanticException {
     String baseTableName = unescapeIdentifier(ast.getChild(0).getText());
     String indexName = unescapeIdentifier(ast.getChild(1).getText());
     HashMap<String, String> partSpec = null;
     Tree part = ast.getChild(2);
-    if (part != null)
+    if (part != null) {
       partSpec = extractPartitionSpecs(part);
+    }
     List<Task<?>> indexBuilder = getIndexBuilderMapRed(baseTableName, indexName, partSpec);
     rootTasks.addAll(indexBuilder);
   }
-    
+
   private List<Task<?>> getIndexBuilderMapRed(String baseTableName, String indexName,
       HashMap<String, String> partSpec) throws SemanticException {
     try {
@@ -291,7 +297,7 @@ public class DDLSemanticAnalyzer extends
       String baseTblName = index.getOrigTableName();
       Table baseTbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
           baseTblName);
-     
+
       String handlerCls = index.getIndexHandlerClass();
       HiveIndexHandler handler = HiveUtils.getIndexHandler(conf, handlerCls);
 
@@ -300,9 +306,9 @@ public class DDLSemanticAnalyzer extends
       if(indexTbl != null) {
         indexTblPartitions = new ArrayList<Partition>();
         baseTblPartitions = preparePartitions(baseTbl, partSpec,
-            indexTbl, db, indexTblPartitions);        
+            indexTbl, db, indexTblPartitions);
       }
-      
+
       List<Task<?>> ret = handler.generateIndexBuildTaskList(baseTbl, index,
           indexTblPartitions, baseTblPartitions, indexTbl, db);
       return ret;
@@ -310,7 +316,7 @@ public class DDLSemanticAnalyzer extends
       throw new SemanticException(e);
     }
   }
-  
+
   private List<Partition> preparePartitions(
       org.apache.hadoop.hive.ql.metadata.Table baseTbl,
       HashMap<String, String> partSpec,
@@ -348,7 +354,7 @@ public class DDLSemanticAnalyzer extends
     }
     return baseTblPartitions;
   }
-  
+
   private void analyzeAlterTableProps(ASTNode ast, boolean expectView)
     throws SemanticException {
 
@@ -442,6 +448,49 @@ public class DDLSemanticAnalyzer extends
         alterTblDesc), conf));
   }
 
+  private void analyzeAlterTableProtectMode(ASTNode ast)
+      throws SemanticException {
+    String tableName = unescapeIdentifier(ast.getChild(0).getText());
+
+    AlterTableDesc alterTblDesc =
+      new AlterTableDesc(AlterTableTypes.ALTERPROTECTMODE);
+
+    alterTblDesc.setOldName(tableName);
+
+    ASTNode child = (ASTNode) ast.getChild(1);
+
+    switch (child.getToken().getType()) {
+    case HiveParser.TOK_ENABLE:
+      alterTblDesc.setProtectModeEnable(true);
+      break;
+    case HiveParser.TOK_DISABLE:
+      alterTblDesc.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:
+      alterTblDesc.setProtectModeType(AlterTableDesc.ProtectModeType.OFFLINE);
+      break;
+    case HiveParser.TOK_NO_DROP:
+      alterTblDesc.setProtectModeType(AlterTableDesc.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(),
+        alterTblDesc), conf));
+  }
+
   private void analyzeAlterTableClusterSort(ASTNode ast)
       throws SemanticException {
     String tableName = unescapeIdentifier(ast.getChild(0).getText());
@@ -782,6 +831,59 @@ 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

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java?rev=982476&r1=982475&r2=982476&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java Thu Aug  5 05:36:48 2010
@@ -160,6 +160,7 @@ public enum ErrorMsg {
   RESERVED_PART_VAL("Partition value contains a reserved substring"),
   HOLD_DDLTIME_ON_NONEXIST_PARTITIONS("HOLD_DDLTIME hint cannot be applied to dynamic " +
   		"partitions or non-existent partitions"),
+  OFFLINE_TABLE_OR_PARTITION("Query against an offline table or partition")
       ;
 
   private String mesg;

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=982476&r1=982475&r2=982476&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 Thu Aug  5 05:36:48 2010
@@ -99,6 +99,7 @@ TOK_ALTERTABLE_RENAMECOL;
 TOK_ALTERTABLE_REPLACECOLS;
 TOK_ALTERTABLE_ADDPARTS;
 TOK_ALTERTABLE_DROPPARTS;
+TOK_ALTERTABLE_ALTERPARTS_PROTECTMODE;
 TOK_ALTERTABLE_TOUCH;
 TOK_ALTERTABLE_ARCHIVE;
 TOK_ALTERTABLE_UNARCHIVE;
@@ -106,6 +107,7 @@ TOK_ALTERTABLE_SERDEPROPERTIES;
 TOK_ALTERTABLE_SERIALIZER;
 TOK_ALTERTABLE_FILEFORMAT;
 TOK_ALTERTABLE_PROPERTIES;
+TOK_ALTERTABLE_PROTECTMODE;
 TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
 TOK_ALTERINDEX_REBUILD;
 TOK_MSCK;
@@ -128,6 +130,11 @@ TOK_TBLSEQUENCEFILE;
 TOK_TBLTEXTFILE;
 TOK_TBLRCFILE;
 TOK_TABLEFILEFORMAT;
+TOK_OFFLINE;
+TOK_ENABLE;
+TOK_DISABLE;
+TOK_READONLY;
+TOK_NO_DROP;
 TOK_STORAGEHANDLER;
 TOK_ALTERTABLE_CLUSTER_SORT;
 TOK_TABCOLNAME;
@@ -346,12 +353,14 @@ alterTableStatementSuffix
     | alterStatementSuffixRenameCol
     | alterStatementSuffixDropPartitions
     | alterStatementSuffixAddPartitions
+    | alterStatementSuffixAlterPartitionsProtectMode
     | alterStatementSuffixTouch
     | alterStatementSuffixArchive
     | alterStatementSuffixUnArchive
     | alterStatementSuffixProperties
     | alterStatementSuffixSerdeProperties
     | alterStatementSuffixFileFormat
+    | alterStatementSuffixProtectMode
     | alterStatementSuffixClusterbySortby
     ;
 
@@ -395,6 +404,13 @@ alterStatementSuffixAddPartitions
     : Identifier KW_ADD ifNotExists? partitionSpec partitionLocation? (partitionSpec partitionLocation?)*
     -> ^(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"); }
@@ -461,6 +477,29 @@ alterStatementSuffixFileFormat
 	-> ^(TOK_ALTERTABLE_FILEFORMAT $name fileFormat)
 	;
 
+alterStatementSuffixProtectMode
+@init {msgs.push("alter protectmode statement"); }
+@after {msgs.pop(); }
+	:name=Identifier alterProtectMode
+	-> ^(TOK_ALTERTABLE_PROTECTMODE $name alterProtectMode)
+	;
+	
+alterProtectMode
+@init { msgs.push("protect mode specification enable"); }
+@after { msgs.pop(); }
+    : KW_ENABLE alterProtectModeMode  -> ^(TOK_ENABLE alterProtectModeMode)
+    | KW_DISABLE alterProtectModeMode  -> ^(TOK_DISABLE alterProtectModeMode)
+    ;
+
+alterProtectModeMode
+@init { msgs.push("protect mode specification enable"); }
+@after { msgs.pop(); }
+    : KW_OFFLINE  -> ^(TOK_OFFLINE)
+    | KW_NO_DROP  -> ^(TOK_NO_DROP)
+    | KW_READONLY  -> ^(TOK_READONLY)
+    ;
+
+
 alterStatementSuffixClusterbySortby
 @init {msgs.push("alter cluster by sort by statement");}
 @after{msgs.pop();}
@@ -1651,6 +1690,11 @@ KW_TEXTFILE: 'TEXTFILE';
 KW_RCFILE: 'RCFILE';
 KW_INPUTFORMAT: 'INPUTFORMAT';
 KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
+KW_OFFLINE: 'OFFLINE';
+KW_ENABLE: 'ENABLE';
+KW_DISABLE: 'DISABLE';
+KW_READONLY: 'READONLY';
+KW_NO_DROP: 'NO_DROP';
 KW_LOCATION: 'LOCATION';
 KW_TABLESAMPLE: 'TABLESAMPLE';
 KW_BUCKET: 'BUCKET';

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java?rev=982476&r1=982475&r2=982476&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java Thu Aug  5 05:36:48 2010
@@ -36,6 +36,9 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.plan.CopyWork;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
 import org.apache.hadoop.hive.ql.plan.MoveWork;
@@ -192,6 +195,11 @@ public class LoadSemanticAnalyzer extend
     // initialize destination table/partition
     tableSpec ts = new tableSpec(db, conf, (ASTNode) tableTree);
 
+    if (ts.tableHandle.isOffline()){
+      throw new SemanticException(
+          ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(":Table " + ts.tableName));
+    }
+
     if (ts.tableHandle.isView()) {
       throw new SemanticException(ErrorMsg.DML_AGAINST_VIEW.getMsg());
     }
@@ -231,7 +239,21 @@ public class LoadSemanticAnalyzer extend
     Map<String, String> partSpec = ts.getPartSpec();
     if (partSpec == null) {
       partSpec = new LinkedHashMap<String, String>();
+    } else {
+      try{
+        Partition part = Hive.get().getPartition(ts.tableHandle, partSpec, false);
+        if (part != null) {
+          if (part.isOffline()) {
+            throw new SemanticException(ErrorMsg.OFFLINE_TABLE_OR_PARTITION.
+                getMsg(ts.tableName + ":" + part.getName()));
+          }
+        }
+      } catch(HiveException e) {
+        throw new SemanticException(e);
+      }
     }
+
+
     LoadTableDesc loadTableWork = new LoadTableDesc(fromURI.toString(),
         loadTmpPath, Utilities.getTableDesc(ts.tableHandle), partSpec, isOverWrite);
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=982476&r1=982475&r2=982476&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Thu Aug  5 05:36:48 2010
@@ -23,8 +23,8 @@ import static org.apache.hadoop.hive.ser
 import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT;
 import static org.apache.hadoop.util.StringUtils.stringifyException;
 
-import java.io.Serializable;
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -39,8 +39,8 @@ import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
@@ -49,17 +49,16 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.Context;
-import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.ExecDriver;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
-import org.apache.hadoop.hive.ql.exec.ExecDriver;
 import org.apache.hadoop.hive.ql.exec.MapRedTask;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
@@ -85,6 +84,7 @@ import org.apache.hadoop.hive.ql.lib.Nod
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
@@ -749,6 +749,19 @@ public class SemanticAnalyzer extends Ba
           throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(qb
               .getParseInfo().getSrcForAlias(alias)));
         }
+
+        // We check offline of the table, as if people only select from an
+        // non-existing partition of an offline table, the partition won't
+        // be added to inputs and validate() won't have the information to
+        // check the table's offline status.
+        // TODO: Modify the code to remove the checking here and consolidate
+        // it in validate()
+        //
+        if (tab.isOffline()) {
+          throw new SemanticException(ErrorMsg.OFFLINE_TABLE_OR_PARTITION.
+              getMsg("Table " + qb.getParseInfo().getSrcForAlias(alias)));
+        }
+
         if (tab.isView()) {
           replaceViewReferenceWithDefinition(qb, tab, tab_name, alias);
           continue;
@@ -1193,9 +1206,10 @@ public class SemanticAnalyzer extends Ba
         new FilterDesc(genExprNodeDesc(condn, inputRR), false), new RowSchema(
         inputRR.getColumnInfos()), input), inputRR);
 
-    if (LOG.isDebugEnabled()) 
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created Filter Plan for " + qb.getId() + " row schema: "
                 + inputRR.toString());
+    }
     return output;
   }
 
@@ -1232,7 +1246,7 @@ public class SemanticAnalyzer extends Ba
       if (tabAlias != null && !tmp[0].equalsIgnoreCase(tabAlias)) {
         continue;
       }
-      
+
       if(colInfo.getIsVirtualCol() && colInfo.isHiddenVirtualCol()) {
         continue;
       }
@@ -1697,8 +1711,9 @@ public class SemanticAnalyzer extends Ba
 
     Operator<?> op = genSelectPlan(selExprList, qb, input);
 
-    if (LOG.isDebugEnabled()) 
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created Select Plan for clause: " + dest);
+    }
 
     return op;
   }
@@ -1707,8 +1722,9 @@ public class SemanticAnalyzer extends Ba
   private Operator<?> genSelectPlan(ASTNode selExprList, QB qb,
       Operator<?> input) throws SemanticException {
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("tree: " + selExprList.toStringTree());
+    }
 
     ArrayList<ExprNodeDesc> col_list = new ArrayList<ExprNodeDesc>();
     RowResolver out_rwsch = new RowResolver();
@@ -1802,8 +1818,9 @@ public class SemanticAnalyzer extends Ba
       exprList = selExprList;
     }
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("genSelectPlan: input = " + inputRR.toString());
+    }
 
     // For UDTF's, skip the function name to get the expressions
     int startPosn = isUDTF ? posn + 1 : posn;
@@ -1913,8 +1930,9 @@ public class SemanticAnalyzer extends Ba
       output = genUDTFPlan(genericUDTF, udtfTableAlias, udtfColAliases, qb,
           output);
     }
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created Select Plan row schema: " + out_rwsch.toString());
+    }
     return output;
   }
 
@@ -3230,6 +3248,7 @@ public class SemanticAnalyzer extends Ba
     case QBMetaData.DEST_TABLE: {
 
       dest_tab = qbm.getDestTableForAlias(dest);
+
       Map<String, String> partSpec = qbm.getPartSpecForAlias(dest);
       dest_path = dest_tab.getPath();
 
@@ -3321,6 +3340,7 @@ public class SemanticAnalyzer extends Ba
     case QBMetaData.DEST_PARTITION: {
 
       Partition dest_part = qbm.getDestPartitionForAlias(dest);
+
       dest_tab = dest_part.getTable();
 
       dest_path = dest_part.getPath()[0];
@@ -3514,9 +3534,10 @@ public class SemanticAnalyzer extends Ba
         .mapDirToFop(ltd.getSourceDir(), (FileSinkOperator)output);
     }
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created FileSink Plan for clause: " + dest + "dest_path: "
                 + dest_path + " row schema: " + inputRR.toString());
+    }
 
     return output;
   }
@@ -3669,9 +3690,10 @@ public class SemanticAnalyzer extends Ba
         new LimitDesc(limit), new RowSchema(inputRR.getColumnInfos()), input),
         inputRR);
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created LimitOperator Plan for clause: " + dest
                 + " row schema: " + inputRR.toString());
+    }
 
     return limitMap;
   }
@@ -3698,9 +3720,10 @@ public class SemanticAnalyzer extends Ba
       throw new SemanticException(ErrorMsg.UDTF_LATERAL_VIEW.getMsg());
     }
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Table alias: " + outputTableAlias + " Col aliases: "
                 + colAliases);
+    }
 
     // Use the RowResolver from the input operator to generate a input
     // ObjectInspector that can be used to initialize the UDTF. Then, the
@@ -3922,7 +3945,7 @@ public class SemanticAnalyzer extends Ba
     for (ColumnInfo colInfo : interim_rwsch.getColumnInfos()) {
       String[] info = interim_rwsch.reverseLookup(colInfo.getInternalName());
       out_rwsch.put(info[0], info[1], new ColumnInfo(
-          getColumnInternalName(pos), colInfo.getType(), info[0], 
+          getColumnInternalName(pos), colInfo.getType(), info[0],
           colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()));
       pos = Integer.valueOf(pos.intValue() + 1);
     }
@@ -3932,9 +3955,10 @@ public class SemanticAnalyzer extends Ba
         Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema(
         out_rwsch.getColumnInfos()), interim), out_rwsch);
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created ReduceSink Plan for table: " + tab.getTableName() +
                 " row schema: " + out_rwsch.toString());
+    }
     return output;
 
   }
@@ -4034,7 +4058,7 @@ public class SemanticAnalyzer extends Ba
     for (ColumnInfo colInfo : interim_rwsch.getColumnInfos()) {
       String[] info = interim_rwsch.reverseLookup(colInfo.getInternalName());
       out_rwsch.put(info[0], info[1], new ColumnInfo(
-          getColumnInternalName(pos), colInfo.getType(), info[0], 
+          getColumnInternalName(pos), colInfo.getType(), info[0],
           colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()));
       pos = Integer.valueOf(pos.intValue() + 1);
     }
@@ -4044,9 +4068,10 @@ public class SemanticAnalyzer extends Ba
         Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema(
         out_rwsch.getColumnInfos()), interim), out_rwsch);
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created ReduceSink Plan for clause: " + dest + " row schema: "
                 + out_rwsch.toString());
+    }
     return output;
   }
 
@@ -4165,7 +4190,7 @@ public class SemanticAnalyzer extends Ba
               .toString()
               + "." + col, valueInfo.getType(), src, valueInfo
               .getIsVirtualCol(), valueInfo.isHiddenVirtualCol());
-          
+
           colExprMap.put(newColInfo.getInternalName(), inputExpr);
           outputRS.put(src, field, newColInfo);
         }
@@ -5162,8 +5187,9 @@ public class SemanticAnalyzer extends Ba
       }
     }
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created Body Plan for Query Block " + qb.getId());
+    }
 
     return curr;
   }
@@ -5403,7 +5429,7 @@ public class SemanticAnalyzer extends Ba
         rwsch.put(alias, part_col.getName(), new ColumnInfo(part_col.getName(),
             TypeInfoFactory.stringTypeInfo, alias, true));
       }
-      
+
       //put all virutal columns in RowResolver.
       Iterator<VirtualColumn> vcs = VirtualColumn.registry.values().iterator();
       //use a list for easy cumtomize
@@ -5561,12 +5587,13 @@ public class SemanticAnalyzer extends Ba
 
     Operator output = putOpInsertMap(tableOp, rwsch);
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created Table Plan for " + alias + " " + tableOp.toString());
+    }
 
     return output;
   }
-  
+
   private Operator genPlan(QBExpr qbexpr) throws SemanticException {
     if (qbexpr.getOpcode() == QBExpr.Opcode.NULLOP) {
       return genPlan(qbexpr.getQB());
@@ -5631,8 +5658,9 @@ public class SemanticAnalyzer extends Ba
 
     Operator bodyOpInfo = genBodyPlan(qb, srcOpInfo);
 
-    if (LOG.isDebugEnabled())
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Created Plan for Query Block " + qb.getId());
+    }
 
     this.qb = qb;
     return bodyOpInfo;
@@ -5678,7 +5706,7 @@ public class SemanticAnalyzer extends Ba
             String[] tabCol = source.reverseLookup(col.getInternalName());
             lvForwardRR.put(tabCol[0], tabCol[1], col);
           }
-          
+
           Operator lvForward = putOpInsertMap(OperatorFactory.getAndMakeChild(
               new LateralViewForwardDesc(), new RowSchema(lvForwardRR.getColumnInfos()),
               op), lvForwardRR);
@@ -5822,6 +5850,7 @@ public class SemanticAnalyzer extends Ba
               Iterator<Partition> iterParts = parts.iterator();
               while (iterParts.hasNext()) {
                 Partition part = iterParts.next();
+
                 listP.add(part.getPartitionPath().toString());
                 try {
                   partP.add(Utilities.getPartitionDesc(part));
@@ -6459,6 +6488,83 @@ public class SemanticAnalyzer extends Ba
 
   @Override
   public void validate() throws SemanticException {
+    // Validate inputs and outputs have right protectmode to execute the query
+    for (ReadEntity readEntity: getInputs()) {
+      ReadEntity.Type type = readEntity.getType();
+
+      if (type != ReadEntity.Type.TABLE &&
+          type != ReadEntity.Type.PARTITION) {
+        // In current implementation it will never happen, but we leave it
+        // here to make the logic complete.
+        continue;
+      }
+
+      Table tbl = readEntity.getTable();
+      Partition p = readEntity.getPartition();
+
+
+      if (tbl.isOffline()) {
+        throw new SemanticException(
+            ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
+            "Table " + tbl.getTableName()));
+      }
+
+      if (type == ReadEntity.Type.PARTITION && p != null && p.isOffline()) {
+        throw new SemanticException(
+            ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
+            "Table " + tbl.getTableName() +
+            " Partition " + p.getName()));
+      }
+    }
+
+    for (WriteEntity writeEntity: getOutputs()) {
+      WriteEntity.Type type = writeEntity.getType();
+
+      if (type != WriteEntity.Type.TABLE &&
+          type != WriteEntity.Type.PARTITION) {
+        continue;
+      }
+
+      Table tbl;
+      Partition p;
+
+
+      if (type == WriteEntity.Type.PARTITION) {
+        Partition inputPartition = writeEntity.getPartition();
+
+        // If it is a partition, Partition's metastore is not fetched. We
+        // need to fetch it.
+        try {
+          p = Hive.get().getPartition(
+              inputPartition.getTable(), inputPartition.getSpec(), false);
+          if (p != null) {
+            tbl = p.getTable();
+          } else {
+            // if p is null, we assume that we insert to a new partition
+            tbl = inputPartition.getTable();
+          }
+        } catch (HiveException e) {
+          throw new SemanticException(e);
+        }
+
+        if (type == WriteEntity.Type.PARTITION && p!=null && p.isOffline()) {
+          throw new SemanticException(
+              ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
+              "Table " + tbl.getTableName() +
+              " Partition " + p.getName()));
+        }
+      }
+      else {
+        tbl = writeEntity.getTable();
+      }
+
+      if (tbl.isOffline()) {
+        throw new SemanticException(
+            ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
+            "Table " + tbl.getTableName()));
+      }
+    }
+
     // validate all tasks
     for (Task<? extends Serializable> rootTask : rootTasks) {
       validate(rootTask);
@@ -6514,7 +6620,7 @@ public class SemanticAnalyzer extends Ba
     RowFormatParams rowFormatParams = new RowFormatParams();
     StorageFormat storageFormat = new StorageFormat();
     AnalyzeCreateCommonVars shared = new AnalyzeCreateCommonVars();
-    
+
     LOG.info("Creating table " + tableName + " position="
         + ast.getCharPositionInLine());
     int numCh = ast.getChildCount();
@@ -6642,8 +6748,8 @@ public class SemanticAnalyzer extends Ba
     case CREATE_TABLE: // REGULAR CREATE TABLE DDL
       crtTblDesc = new CreateTableDesc(tableName, isExt, cols, partCols,
           bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim, rowFormatParams.fieldEscape,
-          rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, comment, 
-          storageFormat.inputFormat, storageFormat.outputFormat, location, shared.serde, 
+          rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim, rowFormatParams.lineDelim, comment,
+          storageFormat.inputFormat, storageFormat.outputFormat, location, shared.serde,
           storageFormat.storageHandler, shared.serdeProps, tblProps, ifNotExists);
 
       validateCreateTable(crtTblDesc);
@@ -6845,13 +6951,15 @@ public class SemanticAnalyzer extends Ba
     throws SemanticException {
 
     // bypass for explain queries for now
-    if (ctx.getExplain())
+    if (ctx.getExplain()) {
       return;
+    }
 
     // user has told us to run in local mode or doesn't want auto-local mode
     if (ctx.isLocalOnlyExecutionMode() ||
-        !conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO))
+        !conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) {
       return;
+    }
 
     final Context lCtx = ctx;
     PathFilter p = new PathFilter () {
@@ -6871,7 +6979,7 @@ public class SemanticAnalyzer extends Ba
         int numReducers = getNumberOfReducers(mrtask.getWork(), conf);
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Task: " + mrtask.getId() + ", Summary: " + 
+          LOG.debug("Task: " + mrtask.getId() + ", Summary: " +
                    inputSummary.getLength() + "," + inputSummary.getFileCount() + ","
                    + numReducers);
         }
@@ -6884,7 +6992,7 @@ public class SemanticAnalyzer extends Ba
         throw new SemanticException (e);
       }
     }
-    
+
     if(!hasNonLocalJob) {
       // none of the mapred tasks needs to be run locally. That means that the
       // query can be executed entirely in local mode. Save the current tracker
@@ -6894,7 +7002,7 @@ public class SemanticAnalyzer extends Ba
       console.printInfo("Automatically selecting local only mode for query");
 
       // If all the tasks can be run locally, we can use local disk for
-      // storing intermediate data. 
+      // storing intermediate data.
 
       /**
        * This code is commented out pending further testing/development
@@ -6908,11 +7016,13 @@ public class SemanticAnalyzer extends Ba
    * Make a best guess at trying to find the number of reducers
    */
   private static int getNumberOfReducers(MapredWork mrwork, HiveConf conf) {
-    if (mrwork.getReducer() == null)
+    if (mrwork.getReducer() == null) {
       return 0;
+    }
 
-    if (mrwork.getNumReduceTasks() >= 0)
+    if (mrwork.getNumReduceTasks() >= 0) {
       return mrwork.getNumReduceTasks();
+    }
 
     return conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS);
   }

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=982476&r1=982475&r2=982476&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 Thu Aug  5 05:36:48 2010
@@ -92,6 +92,7 @@ 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,6 +105,7 @@ public final class SemanticAnalyzerFacto
       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:

Added: 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=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterPartitionProtectModeDesc.java Thu Aug  5 05:36:48 2010
@@ -0,0 +1,128 @@
+/**
+ * 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=982476&r1=982475&r2=982476&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 Thu Aug  5 05:36:48 2010
@@ -42,9 +42,14 @@ public class AlterTableDesc extends DDLD
   public static enum AlterTableTypes {
     RENAME, ADDCOLS, REPLACECOLS, ADDPROPS, ADDSERDE, ADDSERDEPROPS,
     ADDFILEFORMAT, ADDCLUSTERSORTCOLUMN, RENAMECOLUMN, ADDPARTITION,
-    TOUCH, ARCHIVE, UNARCHIVE,
+    TOUCH, ARCHIVE, UNARCHIVE, ALTERPROTECTMODE, ALTERPARTITIONPROTECTMODE,
   };
 
+  public static enum ProtectModeType {
+    NO_DROP, OFFLINE, READ_ONLY
+  };
+
+
   AlterTableTypes op;
   String oldName;
   String newName;
@@ -65,6 +70,8 @@ public class AlterTableDesc extends DDLD
   boolean first;
   String afterCol;
   boolean expectView;
+  boolean protectModeEnable;
+  ProtectModeType protectModeType;
 
   public AlterTableDesc() {
   }
@@ -473,4 +480,20 @@ public class AlterTableDesc extends DDLD
   public void setExpectView(boolean expectView) {
     this.expectView = expectView;
   }
+
+  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/DDLWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java?rev=982476&r1=982475&r2=982476&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 Thu Aug  5 05:36:48 2010
@@ -43,6 +43,7 @@ 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;
@@ -184,6 +185,17 @@ 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
    */
@@ -482,4 +494,13 @@ 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/clientnegative/protectmode_part.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,15 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode3;
+
+create table tbl_protectmode3  (col string) partitioned by (p string);
+alter table tbl_protectmode3 add partition (p='p1');
+alter table tbl_protectmode3 add partition (p='p2');
+
+select * from tbl_protectmode3 where p='p1';
+select * from tbl_protectmode3 where p='p2';
+
+alter table tbl_protectmode3 partition (p='p1') enable offline;
+
+select * from tbl_protectmode3 where p='p2';
+select * from tbl_protectmode3 where p='p1';

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part1.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part1.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part1.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part1.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,21 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode5;
+
+create table tbl_protectmode5_1 (col string);
+
+create table tbl_protectmode5  (col string) partitioned by (p string);
+alter table tbl_protectmode5 add partition (p='p1');
+alter table tbl_protectmode5 add partition (p='p2');
+
+insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p1';
+insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p2';
+
+alter table tbl_protectmode5 partition (p='p1') enable offline;
+
+insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p2';
+insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p1';

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part2.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part2.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part2.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part2.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,9 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode6;
+
+create table tbl_protectmode6  (c1 string,c2 string) partitioned by (p string);
+alter table tbl_protectmode6 add partition (p='p1');
+LOAD DATA LOCAL INPATH '../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');
+alter table tbl_protectmode6 partition (p='p1') enable offline; 
+LOAD DATA LOCAL INPATH '../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,10 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode_no_drop;
+
+create table tbl_protectmode_no_drop  (c1 string,c2 string) partitioned by (p string);
+alter table tbl_protectmode_no_drop add partition (p='p1');
+alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop;
+desc extended tbl_protectmode_no_drop partition (p='p1');
+
+drop table tbl_protectmode_no_drop;
\ No newline at end of file

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl1.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl1.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl1.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl1.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,8 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode_1;
+
+create table tbl_protectmode_1  (col string);
+select * from tbl_protectmode_1;
+alter table tbl_protectmode_1 enable offline;
+select * from tbl_protectmode_1;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl2.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl2.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl2.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl2.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,12 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode2;
+
+create table tbl_protectmode2  (col string) partitioned by (p string);
+alter table tbl_protectmode2 add partition (p='p1');
+alter table tbl_protectmode2 enable no_drop;
+alter table tbl_protectmode2 enable offline;
+alter table tbl_protectmode2 disable no_drop;
+desc extended tbl_protectmode2;
+
+select * from tbl_protectmode2 where p='p1';

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl3.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl3.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl3.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl3.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,10 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode_4;
+
+create table tbl_protectmode_4  (col string);
+select col from tbl_protectmode_4;
+alter table tbl_protectmode_4 enable offline;
+desc extended tbl_protectmode_4;
+
+select col from tbl_protectmode_4;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl4.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl4.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl4.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl4.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,15 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode_tbl4;
+drop table tbl_protectmode_tbl4_src;
+
+create table tbl_protectmode_tbl4_src (col string);
+
+create table tbl_protectmode_tbl4  (col string) partitioned by (p string);
+alter table tbl_protectmode_tbl4 add partition (p='p1');
+alter table tbl_protectmode_tbl4 enable no_drop;
+alter table tbl_protectmode_tbl4 enable offline;
+alter table tbl_protectmode_tbl4 disable no_drop;
+desc extended tbl_protectmode_tbl4;
+
+select col from tbl_protectmode_tbl4 where p='not_exist';

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl5.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl5.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl5.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl5.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,15 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode_tbl5;
+drop table tbl_protectmode_tbl5_src;
+
+create table tbl_protectmode_tbl5_src (col string);
+
+create table tbl_protectmode_tbl5  (col string) partitioned by (p string);
+alter table tbl_protectmode_tbl5 add partition (p='p1');
+alter table tbl_protectmode_tbl5 enable no_drop;
+alter table tbl_protectmode_tbl5 enable offline;
+alter table tbl_protectmode_tbl5 disable no_drop;
+desc extended tbl_protectmode_tbl5;
+
+insert overwrite table tbl_protectmode_tbl5 partition (p='not_exist') select col from tbl_protectmode_tbl5_src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,9 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode__no_drop;
+
+create table tbl_protectmode__no_drop  (col string);
+select * from tbl_protectmode__no_drop;
+alter table tbl_protectmode__no_drop enable no_drop;
+desc extended tbl_protectmode__no_drop;
+drop table tbl_protectmode__no_drop;

Added: hadoop/hive/trunk/ql/src/test/queries/clientpositive/protectmode.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/protectmode.q?rev=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/protectmode.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/protectmode.q Thu Aug  5 05:36:48 2010
@@ -0,0 +1,61 @@
+-- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl1;
+drop table tbl2;
+
+create table tbl1  (col string);
+select * from tbl1;
+select col from tbl1;
+alter table tbl1 enable offline;
+desc extended tbl1;
+alter table tbl1 disable offline;
+desc extended tbl1;
+select * from tbl1;
+select col from tbl1;
+ 
+create table tbl2  (col string) partitioned by (p string);
+alter table tbl2 add partition (p='p1');
+alter table tbl2 add partition (p='p2');
+alter table tbl2 drop partition (p='not_exist');
+
+select * from tbl2 where p='p1';
+select * from tbl2 where p='p2';
+
+alter table tbl2 partition (p='p1') enable offline;
+desc extended tbl2 partition (p='p1');
+
+alter table tbl2 enable offline;
+desc extended tbl2;
+
+alter table tbl2 enable no_drop;
+desc extended tbl2;
+
+alter table tbl2 disable offline;
+desc extended tbl2;
+
+alter table tbl2 disable no_drop;
+desc extended tbl2;
+
+select * from tbl2 where p='p2';
+select col from tbl2 where p='p2';
+
+alter table tbl2 partition (p='p1') disable offline;
+desc extended tbl2 partition (p='p1');
+
+select * from tbl2 where p='p1';
+select col from tbl2 where p='p1';
+
+insert overwrite table tbl1 select col from tbl2 where p='p1';
+insert overwrite table tbl1 select col from tbl1;
+
+alter table tbl2 partition (p='p1') enable no_drop;
+alter table tbl2 partition (p='p1') disable no_drop;
+
+alter table tbl2 partition (p='p2') enable no_drop;
+
+alter table tbl2 drop partition (p='p1');
+
+alter table tbl2 partition (p='p2') disable no_drop;
+
+drop table tbl1;
+drop table tbl2;
\ No newline at end of file

Added: 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=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part.q.out Thu Aug  5 05:36:48 2010
@@ -0,0 +1,52 @@
+PREHOOK: query: -- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode3
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: -- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode3
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table tbl_protectmode3  (col string) partitioned by (p string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table tbl_protectmode3  (col string) partitioned by (p string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tbl_protectmode3
+PREHOOK: query: alter table tbl_protectmode3 add partition (p='p1')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: alter table tbl_protectmode3 add partition (p='p1')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@tbl_protectmode3@p=p1
+PREHOOK: query: alter table tbl_protectmode3 add partition (p='p2')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: alter table tbl_protectmode3 add partition (p='p2')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@tbl_protectmode3@p=p2
+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
+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
+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
+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
+PREHOOK: query: alter table tbl_protectmode3 partition (p='p1') enable offline
+PREHOOK: type: null
+POSTHOOK: query: alter table tbl_protectmode3 partition (p='p1') enable offline
+POSTHOOK: type: null
+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
+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
+FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode3 Partition p=p1

Added: 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=982476&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/protectmode_part1.q.out Thu Aug  5 05:36:48 2010
@@ -0,0 +1,71 @@
+PREHOOK: query: -- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode5
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: -- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
+
+drop table tbl_protectmode5
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table tbl_protectmode5_1 (col string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table tbl_protectmode5_1 (col string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tbl_protectmode5_1
+PREHOOK: query: create table tbl_protectmode5  (col string) partitioned by (p string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table tbl_protectmode5  (col string) partitioned by (p string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tbl_protectmode5
+PREHOOK: query: alter table tbl_protectmode5 add partition (p='p1')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: alter table tbl_protectmode5 add partition (p='p1')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@tbl_protectmode5@p=p1
+PREHOOK: query: alter table tbl_protectmode5 add partition (p='p2')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: alter table tbl_protectmode5 add partition (p='p2')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@tbl_protectmode5@p=p2
+PREHOOK: query: insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p1'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_protectmode5@p=p1
+PREHOOK: Output: default@tbl_protectmode5_1
+POSTHOOK: query: insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p1'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_protectmode5@p=p1
+POSTHOOK: Output: default@tbl_protectmode5_1
+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
+select col from tbl_protectmode5 where p='p2'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_protectmode5@p=p2
+PREHOOK: Output: default@tbl_protectmode5_1
+POSTHOOK: query: insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p2'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_protectmode5@p=p2
+POSTHOOK: Output: default@tbl_protectmode5_1
+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
+POSTHOOK: query: alter table tbl_protectmode5 partition (p='p1') enable offline
+POSTHOOK: type: null
+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
+select col from tbl_protectmode5 where p='p2'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_protectmode5@p=p2
+PREHOOK: Output: default@tbl_protectmode5_1
+POSTHOOK: query: insert overwrite table tbl_protectmode5_1
+select col from tbl_protectmode5 where p='p2'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_protectmode5@p=p2
+POSTHOOK: Output: default@tbl_protectmode5_1
+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), ]
+POSTHOOK: Lineage: tbl_protectmode5_1.col SIMPLE [(tbl_protectmode5)tbl_protectmode5.FieldSchema(name:p, type:string, comment:null), ]
+FAILED: Error in semantic analysis: Query against an offline table or partition Table tbl_protectmode5 Partition p=p1