You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/02/20 01:03:20 UTC

svn commit: r1569998 - in /hive/trunk: hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/ metastore/if/ metastore/src/gen/thrift/gen-cpp/ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/gen/thrift/...

Author: hashutosh
Date: Thu Feb 20 00:03:19 2014
New Revision: 1569998

URL: http://svn.apache.org/r1569998
Log:
HIVE-6406 : Introduce immutable-table table property and if set, disallow insert-into (Sushanth Sowmyan via Ashutosh Chauhan)

Added:
    hive/trunk/ql/src/test/queries/clientnegative/insert_into5.q
    hive/trunk/ql/src/test/queries/clientnegative/insert_into6.q
    hive/trunk/ql/src/test/results/clientnegative/insert_into5.q.out
    hive/trunk/ql/src/test/results/clientnegative/insert_into6.q.out
Modified:
    hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputFormatContainer.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseOutputFormat.java
    hive/trunk/metastore/if/hive_metastore.thrift
    hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp
    hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h
    hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java
    hive/trunk/metastore/src/gen/thrift/gen-php/metastore/Types.php
    hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py
    hive/trunk/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputFormatContainer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputFormatContainer.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputFormatContainer.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputFormatContainer.java Thu Feb 20 00:03:19 2014
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -60,13 +61,6 @@ import java.util.Map;
  */
 class FileOutputFormatContainer extends OutputFormatContainer {
 
-  private static final PathFilter hiddenFileFilter = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
   /**
    * @param of base OutputFormat to contain
    */
@@ -198,13 +192,9 @@ class FileOutputFormatContainer extends 
       Path tablePath = new Path(table.getTTable().getSd().getLocation());
       FileSystem fs = tablePath.getFileSystem(context.getConfiguration());
 
-      if (fs.exists(tablePath)) {
-        FileStatus[] status = fs.globStatus(new Path(tablePath, "*"), hiddenFileFilter);
-
-        if (status.length > 0) {
-          throw new HCatException(ErrorType.ERROR_NON_EMPTY_TABLE,
+      if (!MetaStoreUtils.isDirEmpty(fs,tablePath)){
+        throw new HCatException(ErrorType.ERROR_NON_EMPTY_TABLE,
             table.getDbName() + "." + table.getTableName());
-        }
       }
     }
   }

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseOutputFormat.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseOutputFormat.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseOutputFormat.java Thu Feb 20 00:03:19 2014
@@ -84,7 +84,7 @@ public abstract class HCatBaseOutputForm
     OutputJobInfo jobInfo = getJobInfo(context);
     HiveStorageHandler storageHandler = HCatUtil.getStorageHandler(context.getConfiguration(), 
         jobInfo.getTableInfo().getStorerInfo());
-    //why do we need this?
+    // Always configure storage handler with jobproperties/jobconf before calling any methods on it
     configureOutputStorageHandler(context);
     if (storageHandler instanceof FosterStorageHandler) {
       return new FileOutputFormatContainer(ReflectionUtils.newInstance(

Modified: hive/trunk/metastore/if/hive_metastore.thrift
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/if/hive_metastore.thrift?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/if/hive_metastore.thrift (original)
+++ hive/trunk/metastore/if/hive_metastore.thrift Thu Feb 20 00:03:19 2014
@@ -778,6 +778,12 @@ const string IS_ARCHIVED = "is_archived"
 // is dropped, this directory will be deleted
 const string ORIGINAL_LOCATION = "original_location",
 
+// Whether or not the table is considered immutable - immutable tables can only be
+// overwritten or created if unpartitioned, or if partitioned, partitions inside them
+// can only be overwritten or created. Immutability supports write-once and replace
+// semantics, but not append.
+const string IS_IMMUTABLE = "immutable",
+
 // these should be needed only for backward compatibility with filestore
 const string META_TABLE_COLUMNS   = "columns",
 const string META_TABLE_COLUMN_TYPES   = "columns.types",

Modified: hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp Thu Feb 20 00:03:19 2014
@@ -23,6 +23,8 @@ hive_metastoreConstants::hive_metastoreC
 
   ORIGINAL_LOCATION = "original_location";
 
+  IS_IMMUTABLE = "immutable";
+
   META_TABLE_COLUMNS = "columns";
 
   META_TABLE_COLUMN_TYPES = "columns.types";

Modified: hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h Thu Feb 20 00:03:19 2014
@@ -21,6 +21,7 @@ class hive_metastoreConstants {
   std::string HIVE_FILTER_FIELD_LAST_ACCESS;
   std::string IS_ARCHIVED;
   std::string ORIGINAL_LOCATION;
+  std::string IS_IMMUTABLE;
   std::string META_TABLE_COLUMNS;
   std::string META_TABLE_COLUMN_TYPES;
   std::string BUCKET_FIELD_NAME;

Modified: hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java Thu Feb 20 00:03:19 2014
@@ -45,6 +45,8 @@ public class hive_metastoreConstants {
 
   public static final String ORIGINAL_LOCATION = "original_location";
 
+  public static final String IS_IMMUTABLE = "immutable";
+
   public static final String META_TABLE_COLUMNS = "columns";
 
   public static final String META_TABLE_COLUMN_TYPES = "columns.types";

Modified: hive/trunk/metastore/src/gen/thrift/gen-php/metastore/Types.php
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-php/metastore/Types.php?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-php/metastore/Types.php (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-php/metastore/Types.php Thu Feb 20 00:03:19 2014
@@ -7765,6 +7765,8 @@ $GLOBALS['hive_metastore_CONSTANTS']['IS
 
 $GLOBALS['hive_metastore_CONSTANTS']['ORIGINAL_LOCATION'] = "original_location";
 
+$GLOBALS['hive_metastore_CONSTANTS']['IS_IMMUTABLE'] = "immutable";
+
 $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_COLUMNS'] = "columns";
 
 $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_COLUMN_TYPES'] = "columns.types";

Modified: hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py Thu Feb 20 00:03:19 2014
@@ -15,6 +15,7 @@ HIVE_FILTER_FIELD_PARAMS = "hive_filter_
 HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"
 IS_ARCHIVED = "is_archived"
 ORIGINAL_LOCATION = "original_location"
+IS_IMMUTABLE = "immutable"
 META_TABLE_COLUMNS = "columns"
 META_TABLE_COLUMN_TYPES = "columns.types"
 BUCKET_FIELD_NAME = "bucket_field_name"

Modified: hive/trunk/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb (original)
+++ hive/trunk/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb Thu Feb 20 00:03:19 2014
@@ -19,6 +19,8 @@ IS_ARCHIVED = %q"is_archived"
 
 ORIGINAL_LOCATION = %q"original_location"
 
+IS_IMMUTABLE = %q"immutable"
+
 META_TABLE_COLUMNS = %q"columns"
 
 META_TABLE_COLUMN_TYPES = %q"columns.types"

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Thu Feb 20 00:03:19 2014
@@ -44,6 +44,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -1180,6 +1181,29 @@ public class MetaStoreUtils {
     return "TRUE".equalsIgnoreCase(params.get("EXTERNAL"));
   }
 
+  /**
+   * Determines whether a table is an immutable table.
+   * Immutable tables are write-once/replace, and do not support append. Partitioned
+   * immutable tables do support additions by way of creation of new partitions, but
+   * do not allow the partitions themselves to be appended to. "INSERT INTO" will not
+   * work for Immutable tables.
+   *
+   * @param table table of interest
+   *
+   * @return true if immutable
+   */
+  public static boolean isImmutableTable(Table table) {
+    if (table == null){
+      return false;
+    }
+    Map<String, String> params = table.getParameters();
+    if (params == null) {
+      return false;
+    }
+
+    return "TRUE".equalsIgnoreCase(params.get(hive_metastoreConstants.IS_IMMUTABLE));
+  }
+
   public static boolean isArchived(
       org.apache.hadoop.hive.metastore.api.Partition part) {
     Map<String, String> params = part.getParameters();
@@ -1208,6 +1232,35 @@ public class MetaStoreUtils {
   }
 
   /**
+   * Filter that filters out hidden files
+   */
+  private static final PathFilter hiddenFileFilter = new PathFilter() {
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+  };
+
+  /**
+   * Utility method that determines if a specified directory already has
+   * contents (non-hidden files) or not - useful to determine if an
+   * immutable table already has contents, for example.
+   *
+   * @param path
+   * @throws IOException
+   */
+  public static boolean isDirEmpty(FileSystem fs, Path path) throws IOException {
+
+    if (fs.exists(path)) {
+      FileStatus[] status = fs.globStatus(new Path(path, "*"), hiddenFileFilter);
+      if (status.length > 0) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
    * Returns true if partial has the same values as full for all values that
    * aren't empty in partial.
    */

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java Thu Feb 20 00:03:19 2014
@@ -374,6 +374,7 @@ public enum ErrorMsg {
   UNNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP(10254,
       "Principal type GROUP is not supported in this authorization setting", "28000"),
   INVALID_TABLE_NAME(10255, "Invalid table name {0}", true),
+  INSERT_INTO_IMMUTABLE_TABLE(10256, "Inserting into a non-empty immutable table is not allowed"),
 
   //========================== 20000 range starts here ========================//
   SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."),

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java Thu Feb 20 00:03:19 2014
@@ -45,6 +45,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.HivePassThroughOutputFormat;
@@ -401,6 +402,11 @@ public class Table implements Serializab
     return tTable.getParameters().get(name);
   }
 
+  public boolean isImmutable(){
+    return (tTable.getParameters().containsKey(hive_metastoreConstants.IS_IMMUTABLE)
+        && tTable.getParameters().get(hive_metastoreConstants.IS_IMMUTABLE).equalsIgnoreCase("true"));
+  }
+
   public void setTableType(TableType tableType) {
      tTable.setTableType(tableType.toString());
    }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1569998&r1=1569997&r2=1569998&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Thu Feb 20 00:03:19 2014
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.common.Jav
 import org.apache.hadoop.hive.common.ObjectPair;
 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.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -5435,6 +5436,25 @@ public class SemanticAnalyzer extends Ba
       Map<String, String> partSpec = qbm.getPartSpecForAlias(dest);
       dest_path = dest_tab.getPath();
 
+      // If the query here is an INSERT_INTO and the target is an immutable table,
+      // verify that our destination is empty before proceeding
+      if (dest_tab.isImmutable() &&
+          qb.getParseInfo().isInsertIntoTable(dest_tab.getDbName(),dest_tab.getTableName())){
+        try {
+          FileSystem fs = dest_path.getFileSystem(conf);
+          if (! MetaStoreUtils.isDirEmpty(fs,dest_path)){
+            LOG.warn("Attempted write into an immutable table : "
+                + dest_tab.getTableName() + " : " + dest_path);
+            throw new SemanticException(
+                ErrorMsg.INSERT_INTO_IMMUTABLE_TABLE.getMsg(dest_tab.getTableName()));
+          }
+        } catch (IOException ioe) {
+            LOG.warn("Error while trying to determine if immutable table has any data : "
+                + dest_tab.getTableName() + " : " + dest_path);
+          throw new SemanticException(ErrorMsg.INSERT_INTO_IMMUTABLE_TABLE.getMsg(ioe.getMessage()));
+        }
+      }
+
       // check for partition
       List<FieldSchema> parts = dest_tab.getPartitionKeys();
       if (parts != null && parts.size() > 0) { // table is partitioned
@@ -5570,6 +5590,26 @@ public class SemanticAnalyzer extends Ba
       Path tabPath = dest_tab.getPath();
       Path partPath = dest_part.getDataLocation();
 
+      // If the query here is an INSERT_INTO and the target is an immutable table,
+      // verify that our destination is empty before proceeding
+      if (dest_tab.isImmutable() &&
+          qb.getParseInfo().isInsertIntoTable(dest_tab.getDbName(),dest_tab.getTableName())){
+        qb.getParseInfo().isInsertToTable();
+        try {
+          FileSystem fs = partPath.getFileSystem(conf);
+          if (! MetaStoreUtils.isDirEmpty(fs,partPath)){
+            LOG.warn("Attempted write into an immutable table partition : "
+                + dest_tab.getTableName() + " : " + partPath);
+            throw new SemanticException(
+                ErrorMsg.INSERT_INTO_IMMUTABLE_TABLE.getMsg(dest_tab.getTableName()));
+          }
+        } catch (IOException ioe) {
+            LOG.warn("Error while trying to determine if immutable table partition has any data : "
+                + dest_tab.getTableName() + " : " + partPath);
+          throw new SemanticException(ErrorMsg.INSERT_INTO_IMMUTABLE_TABLE.getMsg(ioe.getMessage()));
+        }
+      }
+
       // if the table is in a different dfs than the partition,
       // replace the partition's dfs with the table's dfs.
       dest_path = new Path(tabPath.toUri().getScheme(), tabPath.toUri()

Added: hive/trunk/ql/src/test/queries/clientnegative/insert_into5.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/insert_into5.q?rev=1569998&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/insert_into5.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/insert_into5.q Thu Feb 20 00:03:19 2014
@@ -0,0 +1,9 @@
+DROP TABLE if exists insert_into5_neg;
+
+CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true");
+
+INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100;
+
+INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100;
+
+DROP TABLE insert_into5_neg;

Added: hive/trunk/ql/src/test/queries/clientnegative/insert_into6.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/insert_into6.q?rev=1569998&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/insert_into6.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/insert_into6.q Thu Feb 20 00:03:19 2014
@@ -0,0 +1,17 @@
+DROP TABLE IF EXISTS insert_into6_neg;
+
+CREATE TABLE insert_into6_neg (key int, value string)
+  PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true") ;
+
+INSERT INTO TABLE insert_into6_neg PARTITION (ds='1')
+  SELECT * FROM src LIMIT 100;
+
+INSERT INTO TABLE insert_into6_neg PARTITION (ds='2')
+  SELECT * FROM src LIMIT 100;
+
+SELECT COUNT(*) from insert_into6_neg;
+
+INSERT INTO TABLE insert_into6_neg PARTITION (ds='1')
+  SELECT * FROM src LIMIT 100;
+
+DROP TABLE insert_into6_neg;

Added: hive/trunk/ql/src/test/results/clientnegative/insert_into5.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/insert_into5.q.out?rev=1569998&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/insert_into5.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/insert_into5.q.out Thu Feb 20 00:03:19 2014
@@ -0,0 +1,20 @@
+PREHOOK: query: DROP TABLE if exists insert_into5_neg
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE if exists insert_into5_neg
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true")
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@insert_into5_neg
+PREHOOK: query: INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@insert_into5_neg
+POSTHOOK: query: INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@insert_into5_neg
+POSTHOOK: Lineage: insert_into5_neg.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into5_neg.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+FAILED: SemanticException [Error 10256]: Inserting into a non-empty immutable table is not allowed insert_into5_neg

Added: hive/trunk/ql/src/test/results/clientnegative/insert_into6.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/insert_into6.q.out?rev=1569998&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/insert_into6.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/insert_into6.q.out Thu Feb 20 00:03:19 2014
@@ -0,0 +1,55 @@
+PREHOOK: query: DROP TABLE IF EXISTS insert_into6_neg
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS insert_into6_neg
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE insert_into6_neg (key int, value string)
+  PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true")
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE insert_into6_neg (key int, value string)
+  PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@insert_into6_neg
+PREHOOK: query: INSERT INTO TABLE insert_into6_neg PARTITION (ds='1')
+  SELECT * FROM src LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@insert_into6_neg@ds=1
+POSTHOOK: query: INSERT INTO TABLE insert_into6_neg PARTITION (ds='1')
+  SELECT * FROM src LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@insert_into6_neg@ds=1
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=1).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: INSERT INTO TABLE insert_into6_neg PARTITION (ds='2')
+  SELECT * FROM src LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@insert_into6_neg@ds=2
+POSTHOOK: query: INSERT INTO TABLE insert_into6_neg PARTITION (ds='2')
+  SELECT * FROM src LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@insert_into6_neg@ds=2
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=1).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=2).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: SELECT COUNT(*) from insert_into6_neg
+PREHOOK: type: QUERY
+PREHOOK: Input: default@insert_into6_neg
+PREHOOK: Input: default@insert_into6_neg@ds=1
+PREHOOK: Input: default@insert_into6_neg@ds=2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT COUNT(*) from insert_into6_neg
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@insert_into6_neg
+POSTHOOK: Input: default@insert_into6_neg@ds=1
+POSTHOOK: Input: default@insert_into6_neg@ds=2
+#### A masked pattern was here ####
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=1).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=2).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: insert_into6_neg PARTITION(ds=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+200
+FAILED: SemanticException [Error 10256]: Inserting into a non-empty immutable table is not allowed insert_into6_neg