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 2012/12/17 11:49:38 UTC

svn commit: r1422844 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/exec/ java/org/apache/hadoop/hive/ql/parse/ java/org/apache/hadoop/hive/ql/plan/ test/queries/clientpositive/ test/results/clientpositive/

Author: namit
Date: Mon Dec 17 10:49:36 2012
New Revision: 1422844

URL: http://svn.apache.org/viewvc?rev=1422844&view=rev
Log:
HIVE-3646 Add 'IGNORE PROTECTION' predicate for dropping partitions
(Andrew Chalfant via namit)


Added:
    hive/trunk/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q
    hive/trunk/ql/src/test/results/clientpositive/drop_partitions_ignore_protection.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1422844&r1=1422843&r2=1422844&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Mon Dec 17 10:49:36 2012
@@ -3416,18 +3416,20 @@ public class DDLTask extends Task<DDLWor
             break;
           }
         }
-        for (Partition p : partitions) {
-          if (!p.canDrop()) {
-            throw new HiveException("Table " + tbl.getTableName()
-                + " Partition " + p.getName()
-                + " is protected from being dropped");
-          } else if (ArchiveUtils.isArchived(p)) {
-            int partAchiveLevel = ArchiveUtils.getArchivingLevel(p);
-            // trying to drop partitions inside a har, disallow it.
-            if (partAchiveLevel < partPrefixToDrop) {
-              throw new HiveException(
-                  "Cannot drop a subset of partitions in an archive, partition "
-                      + p.getName());
+        if (!dropTbl.getIgnoreProtection()) {
+          for (Partition p : partitions) {
+            if (!p.canDrop()) {
+              throw new HiveException("Table " + tbl.getTableName()
+                  + " Partition " + p.getName()
+                  + " is protected from being dropped");
+            } else if (ArchiveUtils.isArchived(p)) {
+              int partAchiveLevel = ArchiveUtils.getArchivingLevel(p);
+              // trying to drop partitions inside a har, disallow it.
+              if (partAchiveLevel < partPrefixToDrop) {
+                throw new HiveException(
+                    "Cannot drop a subset of partitions in an archive, partition "
+                        + p.getName());
+              }
             }
           }
         }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1422844&r1=1422843&r2=1422844&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Mon Dec 17 10:49:36 2012
@@ -2211,9 +2211,9 @@ public class DDLSemanticAnalyzer extends
           !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT);
       addTableDropPartsOutputs(tblName, partSpecs, throwException, stringPartitionColumns);
     }
-
+    boolean ignoreProtection = (ast.getFirstChildWithType(HiveParser.TOK_IGNOREPROTECTION) != null);
     DropTableDesc dropTblDesc =
-        new DropTableDesc(tblName, partSpecs, expectView, stringPartitionColumns);
+        new DropTableDesc(tblName, partSpecs, expectView, stringPartitionColumns, ignoreProtection);
 
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         dropTblDesc), conf));

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=1422844&r1=1422843&r2=1422844&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Mon Dec 17 10:49:36 2012
@@ -1,9 +1,9 @@
 /**
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
+   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 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
@@ -277,6 +277,7 @@ TOK_SKEWED_LOCATIONS;
 TOK_SKEWED_LOCATION_LIST;
 TOK_SKEWED_LOCATION_MAP;
 TOK_STOREDASDIRS;
+TOK_IGNOREPROTECTION;
 }
 
 
@@ -410,6 +411,12 @@ orReplace
     -> ^(TOK_ORREPLACE)
     ;
 
+ignoreProtection
+@init { msgs.push("ignore protection clause"); }
+@after { msgs.pop(); }
+        : KW_IGNORE KW_PROTECTION
+        -> ^(TOK_IGNOREPROTECTION)
+        ;
 
 createDatabaseStatement
 @init { msgs.push("create database statement"); }
@@ -717,8 +724,8 @@ partitionLocation
 alterStatementSuffixDropPartitions
 @init { msgs.push("drop partition statement"); }
 @after { msgs.pop(); }
-    : Identifier KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)*
-    -> ^(TOK_ALTERTABLE_DROPPARTS Identifier dropPartitionSpec+ ifExists?)
+    : Identifier KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection?
+    -> ^(TOK_ALTERTABLE_DROPPARTS Identifier dropPartitionSpec+ ifExists? ignoreProtection?)
     ;
 
 alterStatementSuffixProperties
@@ -2417,6 +2424,8 @@ KW_AFTER: 'AFTER';
 KW_DESCRIBE: 'DESCRIBE';
 KW_DROP: 'DROP';
 KW_RENAME: 'RENAME';
+KW_IGNORE: 'IGNORE';
+KW_PROTECTION: 'PROTECTION';
 KW_TO: 'TO';
 KW_COMMENT: 'COMMENT';
 KW_BOOLEAN: 'BOOLEAN';

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java?rev=1422844&r1=1422843&r2=1422844&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java Mon Dec 17 10:49:36 2012
@@ -34,6 +34,7 @@ public class DropTableDesc extends DDLDe
   ArrayList<PartitionSpec> partSpecs;
   boolean expectView;
   boolean ifExists;
+  boolean ignoreProtection;
   boolean stringPartitionColumns; // This is due to JDO not working very well with
                                   // non-string partition columns.
                                   // We need a different codepath for them
@@ -50,17 +51,20 @@ public class DropTableDesc extends DDLDe
     partSpecs = null;
     this.expectView = expectView;
     this.ifExists = ifExists;
+    this.ignoreProtection = false;
     this.stringPartitionColumns = stringPartitionColumns;
   }
 
   public DropTableDesc(String tableName, List<PartitionSpec> partSpecs,
-                       boolean expectView, boolean stringPartitionColumns) {
+                       boolean expectView, boolean stringPartitionColumns,
+                       boolean ignoreProtection) {
 
     this.tableName = tableName;
     this.partSpecs = new ArrayList<PartitionSpec>(partSpecs.size());
     for (int i = 0; i < partSpecs.size(); i++) {
       this.partSpecs.add(partSpecs.get(i));
     }
+    this.ignoreProtection = ignoreProtection;
     this.expectView = expectView;
     this.stringPartitionColumns = stringPartitionColumns;
   }
@@ -97,6 +101,21 @@ public class DropTableDesc extends DDLDe
   }
 
   /**
+   * @return whether or not protection will be ignored for the partition
+   */
+  public boolean getIgnoreProtection() {
+    return ignoreProtection;
+  }
+
+  /**
+   * @param ignoreProtection
+   *          set whether or not protection will be ignored for the partition
+   */
+   public void setIgnoreProtection(boolean ignoreProtection) {
+     this.ignoreProtection = ignoreProtection;
+   }
+
+  /**
    * @return whether to expect a view being dropped
    */
   public boolean getExpectView() {

Added: hive/trunk/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q?rev=1422844&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q Mon Dec 17 10:49:36 2012
@@ -0,0 +1,10 @@
+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');
+
+-- The partition will be dropped, even though we have enabled no_drop
+-- as 'ignore protection' has been specified in the command predicate
+alter table tbl_protectmode_no_drop drop partition (p='p1') ignore protection;
+drop table tbl_protectmode_no_drop;
+

Added: hive/trunk/ql/src/test/results/clientpositive/drop_partitions_ignore_protection.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/drop_partitions_ignore_protection.q.out?rev=1422844&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/drop_partitions_ignore_protection.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/drop_partitions_ignore_protection.q.out Mon Dec 17 10:49:36 2012
@@ -0,0 +1,57 @@
+PREHOOK: query: create table tbl_protectmode_no_drop  (c1 string,c2 string) partitioned by (p string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table tbl_protectmode_no_drop  (c1 string,c2 string) partitioned by (p string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tbl_protectmode_no_drop
+PREHOOK: query: alter table tbl_protectmode_no_drop add partition (p='p1')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@tbl_protectmode_no_drop
+POSTHOOK: query: alter table tbl_protectmode_no_drop add partition (p='p1')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@tbl_protectmode_no_drop
+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: ALTERPARTITION_PROTECTMODE
+PREHOOK: Input: default@tbl_protectmode_no_drop
+PREHOOK: Output: default@tbl_protectmode_no_drop@p=p1
+POSTHOOK: query: alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop
+POSTHOOK: type: ALTERPARTITION_PROTECTMODE
+POSTHOOK: Input: default@tbl_protectmode_no_drop
+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')
+POSTHOOK: type: DESCTABLE
+# col_name            	data_type           	comment             
+	 	 
+c1                  	string              	None                
+c2                  	string              	None                
+p                   	string              	None                
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+p                   	string              	None                
+	 	 
+#### A masked pattern was here ####
+PREHOOK: query: -- The partition will be dropped, even though we have enabled no_drop
+-- as 'ignore protection' has been specified in the command predicate
+alter table tbl_protectmode_no_drop drop partition (p='p1') ignore protection
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@tbl_protectmode_no_drop
+PREHOOK: Output: default@tbl_protectmode_no_drop@p=p1
+POSTHOOK: query: -- The partition will be dropped, even though we have enabled no_drop
+-- as 'ignore protection' has been specified in the command predicate
+alter table tbl_protectmode_no_drop drop partition (p='p1') ignore protection
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@tbl_protectmode_no_drop
+POSTHOOK: Output: default@tbl_protectmode_no_drop@p=p1
+PREHOOK: query: drop table tbl_protectmode_no_drop
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@tbl_protectmode_no_drop
+PREHOOK: Output: default@tbl_protectmode_no_drop
+POSTHOOK: query: drop table tbl_protectmode_no_drop
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@tbl_protectmode_no_drop
+POSTHOOK: Output: default@tbl_protectmode_no_drop