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

svn commit: r1628834 [1/2] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/metadata/ ql/src/java/org/apache/hadoop/hi...

Author: gates
Date: Wed Oct  1 21:02:18 2014
New Revision: 1628834

URL: http://svn.apache.org/r1628834
Log:
HIVE-8290 With DbTxnManager configured, all ORC tables forced to be transactional (Alan Gates, reviewed by Eugene Koifman)

Added:
    hive/trunk/ql/src/test/queries/clientnegative/delete_not_bucketed.q
    hive/trunk/ql/src/test/queries/clientnegative/delete_sorted.q
    hive/trunk/ql/src/test/queries/clientnegative/insert_sorted.q
    hive/trunk/ql/src/test/queries/clientnegative/insert_values_sorted.q
    hive/trunk/ql/src/test/queries/clientnegative/update_not_bucketed.q
    hive/trunk/ql/src/test/queries/clientnegative/update_sorted.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_acid_dynamic_partition.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_acid_not_bucketed.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_values_acid_not_bucketed.q
    hive/trunk/ql/src/test/results/clientnegative/delete_not_bucketed.q.out
    hive/trunk/ql/src/test/results/clientnegative/delete_sorted.q.out
    hive/trunk/ql/src/test/results/clientnegative/insert_sorted.q.out
    hive/trunk/ql/src/test/results/clientnegative/insert_values_sorted.q.out
    hive/trunk/ql/src/test/results/clientnegative/update_not_bucketed.q.out
    hive/trunk/ql/src/test/results/clientnegative/update_sorted.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_acid_dynamic_partition.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_acid_not_bucketed.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_values_acid_not_bucketed.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_acid_dynamic_partition.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_acid_not_bucketed.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_values_acid_not_bucketed.q.out
Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.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/QBMetaData.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
    hive/trunk/ql/src/test/queries/clientnegative/acid_overwrite.q
    hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q
    hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q
    hive/trunk/ql/src/test/queries/clientnegative/update_partition_col.q
    hive/trunk/ql/src/test/queries/clientpositive/acid_vectorization.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_all_non_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_all_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_orig_table.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_tmp_table.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_where_no_match.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_where_non_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_where_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/delete_whole_partition.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_orig_table.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_update_delete.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_values_dynamic_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_values_non_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_values_orig_table.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_values_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/insert_values_tmp_table.q
    hive/trunk/ql/src/test/queries/clientpositive/update_after_multiple_inserts.q
    hive/trunk/ql/src/test/queries/clientpositive/update_all_non_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/update_all_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/update_all_types.q
    hive/trunk/ql/src/test/queries/clientpositive/update_orig_table.q
    hive/trunk/ql/src/test/queries/clientpositive/update_tmp_table.q
    hive/trunk/ql/src/test/queries/clientpositive/update_two_cols.q
    hive/trunk/ql/src/test/queries/clientpositive/update_where_no_match.q
    hive/trunk/ql/src/test/queries/clientpositive/update_where_non_partitioned.q
    hive/trunk/ql/src/test/queries/clientpositive/update_where_partitioned.q
    hive/trunk/ql/src/test/results/clientnegative/acid_overwrite.q.out
    hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out
    hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
    hive/trunk/ql/src/test/results/clientnegative/delete_non_acid_table.q.out
    hive/trunk/ql/src/test/results/clientnegative/update_non_acid_table.q.out
    hive/trunk/ql/src/test/results/clientnegative/update_partition_col.q.out
    hive/trunk/ql/src/test/results/clientpositive/acid_vectorization.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_all_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_all_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_tmp_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_where_no_match.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_where_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_where_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/delete_whole_partition.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_orig_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_update_delete.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_values_dynamic_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_values_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_values_orig_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_values_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_values_tmp_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_all_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_all_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_tmp_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_where_no_match.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_where_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_where_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/delete_whole_partition.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_orig_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_update_delete.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_values_dynamic_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_values_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_values_orig_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_values_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/insert_values_tmp_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_after_multiple_inserts.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_all_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_all_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_all_types.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_tmp_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_two_cols.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_where_no_match.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_where_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/tez/update_where_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_after_multiple_inserts.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_all_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_all_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_all_types.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_tmp_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_two_cols.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_where_no_match.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_where_non_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/update_where_partitioned.q.out

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java Wed Oct  1 21:02:18 2014
@@ -98,7 +98,7 @@ public class TestHiveAuthorizerCheckInvo
     runCmd("create database " + dbName);
     // Need a separate table for ACID testing since it has to be bucketed and it has to be Acid
     runCmd("create table " + acidTableName + " (i int, j int) clustered by (i) into 2 buckets " +
-        "stored as orc");
+        "stored as orc TBLPROPERTIES ('transactional'='true')");
   }
 
   private static void runCmd(String cmd) throws CommandNeedRetryException {

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=1628834&r1=1628833&r2=1628834&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 Wed Oct  1 21:02:18 2014
@@ -418,7 +418,9 @@ public enum ErrorMsg {
   VALUES_TABLE_CONSTRUCTOR_NOT_SUPPORTED(10296,
       "Values clause with table constructor not yet supported"),
   ACID_OP_ON_NONACID_TABLE(10297, "Attempt to do update or delete on table {0} that does not use " +
-      "an AcidOutputFormat", true),
+      "an AcidOutputFormat or is not bucketed", true),
+  ACID_NO_SORTED_BUCKETS(10298, "ACID insert, update, delete not supported on tables that are " +
+      "sorted, table {0}", true),
 
   //========================== 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/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Wed Oct  1 21:02:18 2014
@@ -378,6 +378,27 @@ public class Hive {
       List<String> partCols, Class<? extends InputFormat> fileInputFormat,
       Class<?> fileOutputFormat, int bucketCount, List<String> bucketCols)
       throws HiveException {
+    createTable(tableName, columns, partCols, fileInputFormat, fileOutputFormat, bucketCount,
+        bucketCols, null);
+  }
+
+  /**
+   * Create a table metadata and the directory for the table data
+   * @param tableName table name
+   * @param columns list of fields of the table
+   * @param partCols partition keys of the table
+   * @param fileInputFormat Class of the input format of the table data file
+   * @param fileOutputFormat Class of the output format of the table data file
+   * @param bucketCount number of buckets that each partition (or the table itself) should be
+   *                    divided into
+   * @param bucketCols Bucket columns
+   * @param parameters Parameters for the table
+   * @throws HiveException
+   */
+  public void createTable(String tableName, List<String> columns, List<String> partCols,
+                          Class<? extends InputFormat> fileInputFormat,
+                          Class<?> fileOutputFormat, int bucketCount, List<String> bucketCols,
+                          Map<String, String> parameters) throws HiveException {
     if (columns == null) {
       throw new HiveException("columns not specified for table " + tableName);
     }
@@ -402,6 +423,9 @@ public class Hive {
     tbl.setSerializationLib(LazySimpleSerDe.class.getName());
     tbl.setNumBuckets(bucketCount);
     tbl.setBucketCols(bucketCols);
+    if (parameters != null) {
+      tbl.setParamters(parameters);
+    }
     createTable(tbl);
   }
 

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=1628834&r1=1628833&r2=1628834&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 Wed Oct  1 21:02:18 2014
@@ -385,6 +385,10 @@ public class Table implements Serializab
     tTable.getParameters().put(name, value);
   }
 
+  public void setParamters(Map<String, String> params) {
+    tTable.setParameters(params);
+  }
+
   public String getProperty(String name) {
     return tTable.getParameters().get(name);
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBMetaData.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBMetaData.java?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBMetaData.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBMetaData.java Wed Oct  1 21:02:18 2014
@@ -104,10 +104,18 @@ public class QBMetaData {
     return nameToDestTable.get(alias.toLowerCase());
   }
 
+  public Map<String, Table> getNameToDestTable() {
+    return nameToDestTable;
+  }
+
   public Partition getDestPartitionForAlias(String alias) {
     return nameToDestPartition.get(alias.toLowerCase());
   }
 
+  public Map<String, Partition> getNameToDestPartition() {
+    return nameToDestPartition;
+  }
+
   public String getDestFileForAlias(String alias) {
     return nameToDestFile.get(alias.toLowerCase());
   }

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=1628834&r1=1628833&r2=1628834&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 Wed Oct  1 21:02:18 2014
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.Atomi
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
 
+import com.google.common.annotations.VisibleForTesting;
 import net.hydromatic.optiq.SchemaPlus;
 import net.hydromatic.optiq.tools.Frameworks;
 
@@ -68,6 +69,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryProperties;
 import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
@@ -309,6 +311,9 @@ public class SemanticAnalyzer extends Ba
 
   private static final String VALUES_TMP_TABLE_NAME_PREFIX = "Values__Tmp__Table__";
 
+  @VisibleForTesting
+  static final String ACID_TABLE_PROPERTY = "transactional";
+
   private HashMap<TableScanOperator, ExprNodeDesc> opToPartPruner;
   private HashMap<TableScanOperator, PrunedPartitionList> opToPartList;
   private HashMap<String, Operator<? extends OperatorDesc>> topOps;
@@ -5896,6 +5901,7 @@ public class SemanticAnalyzer extends Ba
     Integer dest_type = qbm.getDestTypeForAlias(dest);
 
     Table dest_tab = null; // destination table if any
+    boolean destTableIsAcid = false; // should the destination table be written to using ACID
     Partition dest_part = null;// destination partition if any
     Path queryTmpdir = null; // the intermediate destination directory
     Path dest_path = null; // the final destination directory
@@ -5912,6 +5918,7 @@ public class SemanticAnalyzer extends Ba
     case QBMetaData.DEST_TABLE: {
 
       dest_tab = qbm.getDestTableForAlias(dest);
+      destTableIsAcid = isAcidTable(dest_tab);
 
       // Is the user trying to insert into a external tables
       if ((!conf.getBoolVar(HiveConf.ConfVars.HIVE_INSERT_INTO_EXTERNAL_TABLES)) &&
@@ -6007,9 +6014,10 @@ public class SemanticAnalyzer extends Ba
       // Create the work for moving the table
       // NOTE: specify Dynamic partitions in dest_tab for WriteEntity
       if (!isNonNativeTable) {
-        AcidUtils.Operation acidOp = getAcidType(table_desc.getOutputFileFormatClass());
-        if (acidOp != AcidUtils.Operation.NOT_ACID) {
-          checkAcidConstraints(qb, table_desc);
+        AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID;
+        if (destTableIsAcid) {
+          acidOp = getAcidType(table_desc.getOutputFileFormatClass());
+          checkAcidConstraints(qb, table_desc, dest_tab);
         }
         ltd = new LoadTableDesc(queryTmpdir,table_desc, dpCtx, acidOp);
         ltd.setReplace(!qb.getParseInfo().isInsertIntoTable(dest_tab.getDbName(),
@@ -6067,6 +6075,7 @@ public class SemanticAnalyzer extends Ba
 
       dest_part = qbm.getDestPartitionForAlias(dest);
       dest_tab = dest_part.getTable();
+      destTableIsAcid = isAcidTable(dest_tab);
       if ((!conf.getBoolVar(HiveConf.ConfVars.HIVE_INSERT_INTO_EXTERNAL_TABLES)) &&
           dest_tab.getTableType().equals(TableType.EXTERNAL_TABLE)) {
         throw new SemanticException(
@@ -6114,9 +6123,10 @@ public class SemanticAnalyzer extends Ba
       lbCtx = constructListBucketingCtx(dest_part.getSkewedColNames(),
           dest_part.getSkewedColValues(), dest_part.getSkewedColValueLocationMaps(),
           dest_part.isStoredAsSubDirectories(), conf);
-      AcidUtils.Operation acidOp = getAcidType(table_desc.getOutputFileFormatClass());
-      if (acidOp != AcidUtils.Operation.NOT_ACID) {
-        checkAcidConstraints(qb, table_desc);
+      AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID;
+      if (destTableIsAcid) {
+        acidOp = getAcidType(table_desc.getOutputFileFormatClass());
+        checkAcidConstraints(qb, table_desc, dest_tab);
       }
       ltd = new LoadTableDesc(queryTmpdir, table_desc, dest_part.getSpec(), acidOp);
       ltd.setReplace(!qb.getParseInfo().isInsertIntoTable(dest_tab.getDbName(),
@@ -6271,9 +6281,7 @@ public class SemanticAnalyzer extends Ba
     ArrayList<ColumnInfo> vecCol = new ArrayList<ColumnInfo>();
 
     if (updating() || deleting()) {
-      vecCol.add(new ColumnInfo(VirtualColumn.ROWID.getName(),
-          //TypeInfoUtils.getTypeInfoFromObjectInspector(VirtualColumn.ROWID.getObjectInspector()),
-          VirtualColumn.ROWID.getTypeInfo(),
+      vecCol.add(new ColumnInfo(VirtualColumn.ROWID.getName(), VirtualColumn.ROWID.getTypeInfo(),
           "", true));
     } else {
       try {
@@ -6302,8 +6310,7 @@ public class SemanticAnalyzer extends Ba
         conf.getBoolVar(HiveConf.ConfVars.HIVEENFORCESORTING))));
 
     // If this table is working with ACID semantics, turn off merging
-    boolean acidTable = isAcidTable(dest_tab);
-    canBeMerged &= !acidTable;
+    canBeMerged &= !destTableIsAcid;
 
     FileSinkDesc fileSinkDesc = new FileSinkDesc(
       queryTmpdir,
@@ -6319,7 +6326,7 @@ public class SemanticAnalyzer extends Ba
 
     // If this is an insert, update, or delete on an ACID table then mark that so the
     // FileSinkOperator knows how to properly write to it.
-    if (acidTable) {
+    if (destTableIsAcid) {
       AcidUtils.Operation wt = updating() ? AcidUtils.Operation.UPDATE :
           (deleting() ? AcidUtils.Operation.DELETE : AcidUtils.Operation.INSERT);
       fileSinkDesc.setWriteType(wt);
@@ -6380,9 +6387,12 @@ public class SemanticAnalyzer extends Ba
   // * no insert overwrites
   // * no use of vectorization
   // * turns off reduce deduplication optimization, as that sometimes breaks acid
+  // * Check that the table is bucketed
+  // * Check that the table is not sorted
   // This method assumes you have already decided that this is an Acid write.  Don't call it if
   // that isn't true.
-  private void checkAcidConstraints(QB qb, TableDesc tableDesc) throws SemanticException {
+  private void checkAcidConstraints(QB qb, TableDesc tableDesc,
+                                    Table table) throws SemanticException {
     String tableName = tableDesc.getTableName();
     if (!qb.getParseInfo().isInsertIntoTable(tableName)) {
       LOG.debug("Couldn't find table " + tableName + " in insertIntoTable");
@@ -6396,6 +6406,16 @@ public class SemanticAnalyzer extends Ba
     conf.setBoolVar(ConfVars.HIVEOPTREDUCEDEDUPLICATION, false);
     conf.setBoolVar(ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES, true);
     conf.set(AcidUtils.CONF_ACID_KEY, "true");
+
+    if (table.getNumBuckets() < 1) {
+      throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TABLE, table.getTableName());
+    }
+    if (table.getSortCols() != null && table.getSortCols().size() > 0) {
+      throw new SemanticException(ErrorMsg.ACID_NO_SORTED_BUCKETS, table.getTableName());
+    }
+
+
+
   }
 
   /**
@@ -12131,9 +12151,9 @@ public class SemanticAnalyzer extends Ba
   // Even if the table is of Acid type, if we aren't working with an Acid compliant TxnManager
   // then return false.
   private boolean isAcidTable(Table tab) {
-    if (tab == null || tab.getOutputFormatClass() == null) return false;
+    if (tab == null) return false;
     if (!SessionState.get().getTxnMgr().supportsAcid()) return false;
-    return isAcidOutputFormat(tab.getOutputFormatClass());
+    return tab.getProperty(ACID_TABLE_PROPERTY) != null;
   }
 
   private boolean isAcidOutputFormat(Class<? extends HiveOutputFormat> of) {

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java Wed Oct  1 21:02:18 2014
@@ -266,9 +266,12 @@ public class TestUpdateDeleteSemanticAna
 
     // I have to create the tables here (rather than in setup()) because I need the Hive
     // connection, which is conviently created by the semantic analyzer.
-    db.createTable("T", Arrays.asList("a", "b"), null, OrcInputFormat.class, OrcOutputFormat.class);
+    Map<String, String> params = new HashMap<String, String>(1);
+    params.put(SemanticAnalyzer.ACID_TABLE_PROPERTY, "true");
+    db.createTable("T", Arrays.asList("a", "b"), null, OrcInputFormat.class,
+        OrcOutputFormat.class, 2, Arrays.asList("a"), params);
     db.createTable("U", Arrays.asList("a", "b"), Arrays.asList("ds"), OrcInputFormat.class,
-        OrcOutputFormat.class);
+        OrcOutputFormat.class, 2, Arrays.asList("a"), params);
     Table u = db.getTable("U");
     Map<String, String> partVals = new HashMap<String, String>(2);
     partVals.put("ds", "yesterday");

Modified: hive/trunk/ql/src/test/queries/clientnegative/acid_overwrite.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/acid_overwrite.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/acid_overwrite.q (original)
+++ hive/trunk/ql/src/test/queries/clientnegative/acid_overwrite.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uanp select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint < 0 order by cint limit 10;
 insert overwrite table acid_uanp select cint, cast(cstring1 as varchar(128)) from alltypesorc;

Modified: hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q (original)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q Wed Oct  1 21:02:18 2014
@@ -9,7 +9,7 @@ set hive.enforce.bucketing=true;
 
 
 -- check update without update priv
-create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc;;
+create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 set user.name=user1;
 delete from auth_nodel where i > 0;

Modified: hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q (original)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q Wed Oct  1 21:02:18 2014
@@ -9,7 +9,7 @@ set hive.enforce.bucketing=true;
 
 
 -- check update without update priv
-create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc;;
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 set user.name=user1;
 update auth_noupd set i = 0 where i > 0;

Added: hive/trunk/ql/src/test/queries/clientnegative/delete_not_bucketed.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/delete_not_bucketed.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/delete_not_bucketed.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/delete_not_bucketed.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,7 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true');
+
+delete from acid_notbucketed where a = 3;

Added: hive/trunk/ql/src/test/queries/clientnegative/delete_sorted.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/delete_sorted.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/delete_sorted.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/delete_sorted.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,7 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_insertsort(a int, b varchar(128)) partitioned by (ds string) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+
+delete from acid_insertsort where a = 3;

Added: hive/trunk/ql/src/test/queries/clientnegative/insert_sorted.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/insert_sorted.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/insert_sorted.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/insert_sorted.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,7 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+
+insert into table acid_insertsort select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;

Added: hive/trunk/ql/src/test/queries/clientnegative/insert_values_sorted.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/insert_values_sorted.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/insert_values_sorted.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/insert_values_sorted.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,7 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+
+insert into table acid_insertsort values (1, 'abc'),(2, 'def');

Added: hive/trunk/ql/src/test/queries/clientnegative/update_not_bucketed.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/update_not_bucketed.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/update_not_bucketed.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/update_not_bucketed.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,7 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_notbucketed(a int, b varchar(128)) partitioned by (ds string) stored as orc TBLPROPERTIES ('transactional'='true');
+
+update acid_notbucketed set b = 'fred' where a = 3;

Modified: hive/trunk/ql/src/test/queries/clientnegative/update_partition_col.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/update_partition_col.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/update_partition_col.q (original)
+++ hive/trunk/ql/src/test/queries/clientnegative/update_partition_col.q Wed Oct  1 21:02:18 2014
@@ -2,6 +2,6 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 update foo set ds = 'fred';

Added: hive/trunk/ql/src/test/queries/clientnegative/update_sorted.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/update_sorted.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/update_sorted.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/update_sorted.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,7 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+
+update acid_insertsort set b = 'fred' where b = 'bob';

Modified: hive/trunk/ql/src/test/queries/clientpositive/acid_vectorization.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/acid_vectorization.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/acid_vectorization.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/acid_vectorization.q Wed Oct  1 21:02:18 2014
@@ -4,7 +4,7 @@ set hive.enforce.bucketing=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.vectorized.execution.enabled=true;
 
-CREATE TABLE acid_vectorized(a INT, b STRING) CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC;
+CREATE TABLE acid_vectorized(a INT, b STRING) CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true');
 insert into table acid_vectorized select cint, cstring1 from alltypesorc where cint is not null order by cint limit 10;
 set hive.vectorized.execution.enabled=true;
 insert into table acid_vectorized values (1, 'bar');

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q Wed Oct  1 21:02:18 2014
@@ -9,7 +9,7 @@ set hive.enforce.bucketing=true;
 set user.name=user1;
 -- current user has been set (comment line before the set cmd is resulting in parse error!!)
 
-CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc;
+CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 -- grant update privilege to another user
 GRANT DELETE ON t_auth_del TO USER userWIns;

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q Wed Oct  1 21:02:18 2014
@@ -9,7 +9,7 @@ set hive.enforce.bucketing=true;
 
 
 set user.name=user1;
-create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc;;
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 delete from auth_noupd where i > 0;
 
 set user.name=hive_admin_user;

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q Wed Oct  1 21:02:18 2014
@@ -9,7 +9,7 @@ set hive.enforce.bucketing=true;
 set user.name=user1;
 -- current user has been set (comment line before the set cmd is resulting in parse error!!)
 
-CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc;
+CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 CREATE TABLE t_select(i int);
 GRANT ALL ON TABLE t_select TO ROLE public;

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q Wed Oct  1 21:02:18 2014
@@ -9,7 +9,7 @@ set hive.enforce.bucketing=true;
 
 
 set user.name=user1;
-create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc;;
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 update auth_noupd set i = 0 where i > 0;
 
 set user.name=hive_admin_user;

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_all_non_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_all_non_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_all_non_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_all_non_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_danp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_danp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_danp select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint < 0 order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_all_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_all_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_all_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_all_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_dap(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table acid_dap(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_dap partition (ds='today') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint < 0 order by cint limit 10;
 insert into table acid_dap partition (ds='tomorrow') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint > 1000 order by cint limit 10;

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_orig_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_orig_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_orig_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_orig_table.q Wed Oct  1 21:02:18 2014
@@ -17,7 +17,7 @@ create table acid_dot(
     ctimestamp1 TIMESTAMP,
     ctimestamp2 TIMESTAMP,
     cboolean1 BOOLEAN,
-    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc location '${system:test.tmp.dir}/delete_orig_table';
+    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc location '${system:test.tmp.dir}/delete_orig_table' TBLPROPERTIES ('transactional'='true');
 
 select count(*) from acid_dot;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_tmp_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_tmp_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_tmp_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_tmp_table.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create temporary table acid_dtt(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_dtt select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_where_no_match.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_where_no_match.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_where_no_match.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_where_no_match.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_dwnm(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_dwnm(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_dwnm select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_where_non_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_where_non_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_where_non_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_where_non_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_dwnp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_dwnp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_dwnp select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_where_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_where_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_where_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_where_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_dwp(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table acid_dwp(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_dwp partition (ds='today') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint < 0 order by cint limit 10;
 insert into table acid_dwp partition (ds='tomorrow') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint > -10000000 order by cint limit 10;

Modified: hive/trunk/ql/src/test/queries/clientpositive/delete_whole_partition.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/delete_whole_partition.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/delete_whole_partition.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/delete_whole_partition.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_dwhp(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table acid_dwhp(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_dwhp partition (ds='today') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint < 0 order by cint limit 10;
 insert into table acid_dwhp partition (ds='tomorrow') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint > -10000000 order by cint limit 10;

Added: hive/trunk/ql/src/test/queries/clientpositive/insert_acid_dynamic_partition.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_acid_dynamic_partition.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_acid_dynamic_partition.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_acid_dynamic_partition.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,10 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+create table acid_dynamic(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+
+insert into table acid_dynamic partition (ds) select cint, cast(cstring1 as varchar(128)), cstring2 from alltypesorc where cint is not null and cint < 0 order by cint limit 5;
+
+select * from acid_dynamic order by a,b;

Added: hive/trunk/ql/src/test/queries/clientpositive/insert_acid_not_bucketed.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_acid_not_bucketed.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_acid_not_bucketed.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_acid_not_bucketed.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_notbucketed(a int, b varchar(128)) stored as orc;
+
+insert into table acid_notbucketed select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
+
+select * from acid_notbucketed;

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_orig_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_orig_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_orig_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_orig_table.q Wed Oct  1 21:02:18 2014
@@ -14,7 +14,7 @@ create table acid_iot(
     ctimestamp1 TIMESTAMP,
     ctimestamp2 TIMESTAMP,
     cboolean1 BOOLEAN,
-    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc;
+    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 LOAD DATA LOCAL INPATH "../../data/files/alltypesorc" into table acid_iot;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_update_delete.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_update_delete.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_update_delete.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_update_delete.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_iud(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_iud(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_iud select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint < 0 order by cint limit 10;
 

Added: hive/trunk/ql/src/test/queries/clientpositive/insert_values_acid_not_bucketed.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_values_acid_not_bucketed.q?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_values_acid_not_bucketed.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_values_acid_not_bucketed.q Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.enforce.bucketing=true;
+
+create table acid_notbucketed(a int, b varchar(128)) stored as orc;
+
+insert into table acid_notbucketed values (1, 'abc'), (2, 'def');
+
+select * from acid_notbucketed;

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_values_dynamic_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_values_dynamic_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_values_dynamic_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_values_dynamic_partitioned.q Wed Oct  1 21:02:18 2014
@@ -5,7 +5,7 @@ set hive.enforce.bucketing=true;
 
 create table ivdp(i int,
                  de decimal(5,2),
-                 vc varchar(128)) partitioned by (ds string) clustered by (i) into 2 buckets stored as orc;
+                 vc varchar(128)) partitioned by (ds string) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table ivdp partition (ds) values 
     (1, 109.23, 'and everywhere that mary went', 'today'),

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_values_non_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_values_non_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_values_non_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_values_non_partitioned.q Wed Oct  1 21:02:18 2014
@@ -14,7 +14,7 @@ create table acid_ivnp(ti tinyint,
                  b boolean,
                  s string,
                  vc varchar(128),
-                 ch char(12)) clustered by (i) into 2 buckets stored as orc;
+                 ch char(12)) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_ivnp values 
     (1, 257, 65537, 4294967297, 3.14, 3.141592654, 109.23, '2014-08-25 17:21:30.0', '2014-08-25', true, 'mary had a little lamb', 'ring around the rosie', 'red'),

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_values_orig_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_values_orig_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_values_orig_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_values_orig_table.q Wed Oct  1 21:02:18 2014
@@ -14,7 +14,7 @@ create table acid_ivot(
     ctimestamp1 TIMESTAMP,
     ctimestamp2 TIMESTAMP,
     cboolean1 BOOLEAN,
-    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc;
+    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 LOAD DATA LOCAL INPATH "../../data/files/alltypesorc" into table acid_ivot;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_values_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_values_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_values_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_values_partitioned.q Wed Oct  1 21:02:18 2014
@@ -13,7 +13,7 @@ create table acid_ivp(ti tinyint,
                  dt date,
                  s string,
                  vc varchar(128),
-                 ch char(12)) partitioned by (ds string) clustered by (i) into 2 buckets stored as orc;
+                 ch char(12)) partitioned by (ds string) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_ivp partition (ds='today') values 
     (1, 257, 65537, 4294967297, 3.14, 3.141592654, 109.23, '2014-08-25 17:21:30.0', '2014-08-25', 'mary had a little lamb', 'ring around the rosie', 'red'),

Modified: hive/trunk/ql/src/test/queries/clientpositive/insert_values_tmp_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/insert_values_tmp_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/insert_values_tmp_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/insert_values_tmp_table.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create temporary table acid_ivtt(i int, de decimal(5,2), vc varchar(128)) clustered by (vc) into 2 buckets stored as orc;
+create temporary table acid_ivtt(i int, de decimal(5,2), vc varchar(128)) clustered by (vc) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_ivtt values 
     (1, 109.23, 'mary had a little lamb'),

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_after_multiple_inserts.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_after_multiple_inserts.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_after_multiple_inserts.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_after_multiple_inserts.q Wed Oct  1 21:02:18 2014
@@ -5,7 +5,7 @@ set hive.enforce.bucketing=true;
 
 create table acid_uami(i int,
                  de decimal(5,2),
-                 vc varchar(128)) clustered by (i) into 2 buckets stored as orc;
+                 vc varchar(128)) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uami values 
     (1, 109.23, 'mary had a little lamb'),

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_all_non_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_all_non_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_all_non_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_all_non_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uanp select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint < 0 order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_all_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_all_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_all_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_all_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_uap(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table acid_uap(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uap partition (ds='today') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint < 0 order by cint limit 10;
 insert into table acid_uap partition (ds='tomorrow') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint > 10 order by cint limit 10;

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_all_types.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_all_types.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_all_types.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_all_types.q Wed Oct  1 21:02:18 2014
@@ -14,7 +14,7 @@ create table acid_uat(ti tinyint,
                  s string,
                  vc varchar(128),
                  ch char(36),
-                 b boolean) clustered by (i) into 2 buckets stored as orc;
+                 b boolean) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uat
     select ctinyint,

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_orig_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_orig_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_orig_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_orig_table.q Wed Oct  1 21:02:18 2014
@@ -17,7 +17,7 @@ create table acid_uot(
     ctimestamp1 TIMESTAMP,
     ctimestamp2 TIMESTAMP,
     cboolean1 BOOLEAN,
-    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc location '${system:test.tmp.dir}/update_orig_table';
+    cboolean2 BOOLEAN) clustered by (cint) into 1 buckets stored as orc location '${system:test.tmp.dir}/update_orig_table' TBLPROPERTIES ('transactional'='true');
 
 update acid_uot set cstring1 = 'fred' where cint < -1070551679;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_tmp_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_tmp_table.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_tmp_table.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_tmp_table.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_utt(a int, b varchar(128)) clustered by (b) into 2 buckets stored as orc;
+create table acid_utt(a int, b varchar(128)) clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_utt select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_two_cols.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_two_cols.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_two_cols.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_two_cols.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_utc(a int, b varchar(128), c float) clustered by (a) into 2 buckets stored as orc;
+create table acid_utc(a int, b varchar(128), c float) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_utc select cint, cast(cstring1 as varchar(128)), cfloat from alltypesorc where cint < 0 order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_where_no_match.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_where_no_match.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_where_no_match.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_where_no_match.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_wnm(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_wnm(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_wnm select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_where_non_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_where_non_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_where_non_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_where_non_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_uwnp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_uwnp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uwnp select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null order by cint limit 10;
 

Modified: hive/trunk/ql/src/test/queries/clientpositive/update_where_partitioned.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/update_where_partitioned.q?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/update_where_partitioned.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/update_where_partitioned.q Wed Oct  1 21:02:18 2014
@@ -2,7 +2,7 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.enforce.bucketing=true;
 
-create table acid_uwp(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table acid_uwp(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uwp partition (ds='today') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint < 0 order by cint limit 10;
 insert into table acid_uwp partition (ds='tomorrow') select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint is not null and cint > 100 order by cint limit 10;

Modified: hive/trunk/ql/src/test/results/clientnegative/acid_overwrite.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/acid_overwrite.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/acid_overwrite.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/acid_overwrite.q.out Wed Oct  1 21:02:18 2014
@@ -1,8 +1,8 @@
-PREHOOK: query: create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc
+PREHOOK: query: create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@acid_uanp
-POSTHOOK: query: create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc
+POSTHOOK: query: create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@acid_uanp

Modified: hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out Wed Oct  1 21:02:18 2014
@@ -1,10 +1,10 @@
 PREHOOK: query: -- check update without update priv
-create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc
+create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@auth_nodel
 POSTHOOK: query: -- check update without update priv
-create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc
+create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@auth_nodel

Modified: hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out Wed Oct  1 21:02:18 2014
@@ -1,10 +1,10 @@
 PREHOOK: query: -- check update without update priv
-create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@auth_noupd
 POSTHOOK: query: -- check update without update priv
-create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@auth_noupd

Modified: hive/trunk/ql/src/test/results/clientnegative/delete_non_acid_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/delete_non_acid_table.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/delete_non_acid_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/delete_non_acid_table.q.out Wed Oct  1 21:02:18 2014
@@ -34,4 +34,4 @@ POSTHOOK: Input: default@not_an_acid_tab
 -1070883071	0ruyd6Y50JpdGRf6HqD
 -1070551679	iUR3Q
 -1069736047	k17Am8uPHWk02cEf1jet
-FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table default.not_an_acid_table2 that does not use an AcidOutputFormat
+FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table default.not_an_acid_table2 that does not use an AcidOutputFormat or is not bucketed

Added: hive/trunk/ql/src/test/results/clientnegative/delete_not_bucketed.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/delete_not_bucketed.q.out?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/delete_not_bucketed.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/delete_not_bucketed.q.out Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_notbucketed
+POSTHOOK: query: create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_notbucketed
+FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table acid_notbucketed that does not use an AcidOutputFormat or is not bucketed

Added: hive/trunk/ql/src/test/results/clientnegative/delete_sorted.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/delete_sorted.q.out?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/delete_sorted.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/delete_sorted.q.out Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: create table acid_insertsort(a int, b varchar(128)) partitioned by (ds string) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_insertsort
+POSTHOOK: query: create table acid_insertsort(a int, b varchar(128)) partitioned by (ds string) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_insertsort
+FAILED: SemanticException [Error 10298]: ACID insert, update, delete not supported on tables that are sorted, table acid_insertsort

Added: hive/trunk/ql/src/test/results/clientnegative/insert_sorted.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/insert_sorted.q.out?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/insert_sorted.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/insert_sorted.q.out Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_insertsort
+POSTHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_insertsort
+FAILED: SemanticException [Error 10298]: ACID insert, update, delete not supported on tables that are sorted, table acid_insertsort

Added: hive/trunk/ql/src/test/results/clientnegative/insert_values_sorted.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/insert_values_sorted.q.out?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/insert_values_sorted.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/insert_values_sorted.q.out Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_insertsort
+POSTHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_insertsort
+FAILED: SemanticException [Error 10298]: ACID insert, update, delete not supported on tables that are sorted, table acid_insertsort

Modified: hive/trunk/ql/src/test/results/clientnegative/update_non_acid_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/update_non_acid_table.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/update_non_acid_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/update_non_acid_table.q.out Wed Oct  1 21:02:18 2014
@@ -34,4 +34,4 @@ POSTHOOK: Input: default@not_an_acid_tab
 -1070883071	0ruyd6Y50JpdGRf6HqD
 -1070551679	iUR3Q
 -1069736047	k17Am8uPHWk02cEf1jet
-FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table default.not_an_acid_table that does not use an AcidOutputFormat
+FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table default.not_an_acid_table that does not use an AcidOutputFormat or is not bucketed

Added: hive/trunk/ql/src/test/results/clientnegative/update_not_bucketed.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/update_not_bucketed.q.out?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/update_not_bucketed.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/update_not_bucketed.q.out Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: create table acid_notbucketed(a int, b varchar(128)) partitioned by (ds string) stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_notbucketed
+POSTHOOK: query: create table acid_notbucketed(a int, b varchar(128)) partitioned by (ds string) stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_notbucketed
+FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table acid_notbucketed that does not use an AcidOutputFormat or is not bucketed

Modified: hive/trunk/ql/src/test/results/clientnegative/update_partition_col.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/update_partition_col.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/update_partition_col.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/update_partition_col.q.out Wed Oct  1 21:02:18 2014
@@ -1,8 +1,8 @@
-PREHOOK: query: create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc
+PREHOOK: query: create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@foo
-POSTHOOK: query: create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc
+POSTHOOK: query: create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@foo

Added: hive/trunk/ql/src/test/results/clientnegative/update_sorted.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/update_sorted.q.out?rev=1628834&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/update_sorted.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/update_sorted.q.out Wed Oct  1 21:02:18 2014
@@ -0,0 +1,9 @@
+PREHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@acid_insertsort
+POSTHOOK: query: create table acid_insertsort(a int, b varchar(128)) clustered by (a) sorted by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@acid_insertsort
+FAILED: SemanticException [Error 10298]: ACID insert, update, delete not supported on tables that are sorted, table acid_insertsort

Modified: hive/trunk/ql/src/test/results/clientpositive/acid_vectorization.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/acid_vectorization.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/acid_vectorization.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/acid_vectorization.q.out Wed Oct  1 21:02:18 2014
@@ -1,8 +1,8 @@
-PREHOOK: query: CREATE TABLE acid_vectorized(a INT, b STRING) CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC
+PREHOOK: query: CREATE TABLE acid_vectorized(a INT, b STRING) CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@acid_vectorized
-POSTHOOK: query: CREATE TABLE acid_vectorized(a INT, b STRING) CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC
+POSTHOOK: query: CREATE TABLE acid_vectorized(a INT, b STRING) CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@acid_vectorized

Modified: hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out Wed Oct  1 21:02:18 2014
@@ -1,12 +1,12 @@
 PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
 
-CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc
+CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t_auth_del
 POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
 
-CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc
+CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t_auth_del

Modified: hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out Wed Oct  1 21:02:18 2014
@@ -1,8 +1,8 @@
-PREHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@auth_noupd
-POSTHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@auth_noupd

Modified: hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out Wed Oct  1 21:02:18 2014
@@ -1,12 +1,12 @@
 PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
 
-CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc
+CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t_auth_up
 POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
 
-CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc
+CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t_auth_up

Modified: hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out Wed Oct  1 21:02:18 2014
@@ -1,8 +1,8 @@
-PREHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@auth_noupd
-POSTHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@auth_noupd

Modified: hive/trunk/ql/src/test/results/clientpositive/delete_all_non_partitioned.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/delete_all_non_partitioned.q.out?rev=1628834&r1=1628833&r2=1628834&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/delete_all_non_partitioned.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/delete_all_non_partitioned.q.out Wed Oct  1 21:02:18 2014
@@ -1,8 +1,8 @@
-PREHOOK: query: create table acid_danp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc
+PREHOOK: query: create table acid_danp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@acid_danp
-POSTHOOK: query: create table acid_danp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc
+POSTHOOK: query: create table acid_danp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@acid_danp