HIVE-15844 Make ReduceSinkOperator independent of Acid (Eugene Koifman, 
reviewed by Prasanth Jayachandran, Matt McCline)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/917221e8
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/917221e8
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/917221e8

Branch: refs/heads/master
Commit: 917221e8378ec48ea05ef6b6c7d9515609b8ec01
Parents: 03216f5
Author: Eugene Koifman <ekoif...@hortonworks.com>
Authored: Wed Mar 1 11:26:16 2017 -0800
Committer: Eugene Koifman <ekoif...@hortonworks.com>
Committed: Wed Mar 1 11:26:16 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    4 +-
 .../insert_into_dynamic_partitions.q.out        |    1 +
 .../insert_overwrite_dynamic_partitions.q.out   |    1 +
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |   89 +-
 .../BucketingSortingReduceSinkOptimizer.java    |    3 -
 .../optimizer/SortedDynPartitionOptimizer.java  |   45 +-
 ...tedDynPartitionTimeGranularityOptimizer.java |    8 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    8 +-
 .../hadoop/hive/ql/plan/FileSinkDesc.java       |   10 +-
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |    2 +-
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     |   16 +-
 .../hive/ql/plan/VectorReduceSinkDesc.java      |    7 -
 .../hive/ql/txn/compactor/CompactorMR.java      |    2 +-
 .../dynpart_sort_optimization_acid.q            |  162 +--
 .../clientpositive/autoColumnStats_4.q.out      |    1 +
 .../dynpart_sort_optimization_acid2.q.out       |    1 +
 .../llap/dynamic_semijoin_reduction_3.q.out     |    7 +
 .../llap/dynpart_sort_opt_vectorization.q.out   |   12 +
 .../llap/dynpart_sort_optimization.q.out        |   18 +
 .../llap/dynpart_sort_optimization2.q.out       |    2 +
 .../llap/dynpart_sort_optimization_acid.q.out   | 1211 +++++++++---------
 .../results/clientpositive/llap/sqlmerge.q.out  |    4 +
 .../llap/vector_adaptor_usage_mode.q.out        |    2 +-
 .../llap/vector_auto_smb_mapjoin_14.q.out       |   10 +-
 .../llap/vector_between_columns.q.out           |    4 +-
 .../clientpositive/llap/vector_between_in.q.out |   32 +-
 .../llap/vector_binary_join_groupby.q.out       |    8 +-
 .../clientpositive/llap/vector_char_2.q.out     |    8 +-
 .../llap/vector_char_mapjoin1.q.out             |   12 +-
 .../llap/vector_char_simple.q.out               |    2 +-
 .../clientpositive/llap/vector_coalesce.q.out   |   10 +-
 .../clientpositive/llap/vector_coalesce_2.q.out |    2 +-
 .../clientpositive/llap/vector_count.q.out      |    8 +-
 .../llap/vector_count_distinct.q.out            |    4 +-
 .../clientpositive/llap/vector_data_types.q.out |    2 +-
 .../llap/vector_decimal_aggregate.q.out         |    2 +-
 .../llap/vector_decimal_expressions.q.out       |    2 +-
 .../llap/vector_decimal_mapjoin.q.out           |    2 +-
 .../llap/vector_decimal_round.q.out             |    4 +-
 .../llap/vector_decimal_round_2.q.out           |    8 +-
 .../clientpositive/llap/vector_distinct_2.q.out |    2 +-
 .../clientpositive/llap/vector_groupby4.q.out   |    4 +-
 .../clientpositive/llap/vector_groupby6.q.out   |    4 +-
 .../clientpositive/llap/vector_groupby_3.q.out  |    2 +-
 .../llap/vector_groupby_mapjoin.q.out           |    4 +-
 .../llap/vector_groupby_reduce.q.out            |   14 +-
 .../clientpositive/llap/vector_if_expr.q.out    |    2 +-
 .../llap/vector_include_no_sel.q.out            |    4 +-
 .../clientpositive/llap/vector_inner_join.q.out |   18 +-
 .../clientpositive/llap/vector_interval_1.q.out |   16 +-
 .../clientpositive/llap/vector_interval_2.q.out |   20 +-
 .../llap/vector_interval_arithmetic.q.out       |   14 +-
 .../llap/vector_interval_mapjoin.q.out          |    2 +-
 .../clientpositive/llap/vector_join30.q.out     |   84 +-
 .../llap/vector_left_outer_join2.q.out          |    8 +-
 .../llap/vector_leftsemi_mapjoin.q.out          |  376 +++---
 .../llap/vector_nullsafe_join.q.out             |   36 +-
 .../llap/vector_number_compare_projection.q.out |    4 +-
 .../clientpositive/llap/vector_orderby_5.q.out  |    4 +-
 .../llap/vector_outer_join0.q.out               |    4 +-
 .../llap/vector_outer_join1.q.out               |   10 +-
 .../llap/vector_outer_join2.q.out               |    6 +-
 .../llap/vector_partition_diff_num_cols.q.out   |   10 +-
 .../llap/vector_partitioned_date_time.q.out     |   18 +-
 .../clientpositive/llap/vector_reduce1.q.out    |    2 +-
 .../clientpositive/llap/vector_reduce2.q.out    |    2 +-
 .../clientpositive/llap/vector_reduce3.q.out    |    2 +-
 .../llap/vector_reduce_groupby_decimal.q.out    |    4 +-
 .../llap/vector_string_concat.q.out             |    4 +-
 .../llap/vector_varchar_simple.q.out            |    2 +-
 .../llap/vector_when_case_null.q.out            |    2 +-
 .../clientpositive/llap/vectorization_0.q.out   |   24 +-
 .../clientpositive/llap/vectorization_7.q.out   |    4 +-
 .../clientpositive/llap/vectorization_8.q.out   |    4 +-
 .../llap/vectorization_short_regress.q.out      |   24 +-
 .../llap/vectorized_bucketmapjoin1.q.out        |    8 +-
 .../llap/vectorized_date_funcs.q.out            |    4 +-
 .../vectorized_dynamic_semijoin_reduction.q.out |   58 +-
 .../llap/vectorized_mapjoin.q.out               |    2 +-
 .../llap/vectorized_shufflejoin.q.out           |    4 +-
 .../llap/vectorized_timestamp_funcs.q.out       |   10 +-
 .../spark/vector_between_in.q.out               |   32 +-
 .../spark/vector_count_distinct.q.out           |    4 +-
 .../spark/vector_data_types.q.out               |    2 +-
 .../spark/vector_decimal_aggregate.q.out        |    2 +-
 .../spark/vector_distinct_2.q.out               |    2 +-
 .../clientpositive/spark/vector_groupby_3.q.out |    2 +-
 .../clientpositive/spark/vector_orderby_5.q.out |    4 +-
 .../spark/vector_outer_join1.q.out              |    2 +-
 .../spark/vector_outer_join2.q.out              |    2 +-
 .../spark/vector_string_concat.q.out            |    4 +-
 .../clientpositive/spark/vectorization_0.q.out  |   24 +-
 .../spark/vectorization_div0.q.out              |    4 +-
 .../spark/vectorization_short_regress.q.out     |   24 +-
 .../spark/vectorized_shufflejoin.q.out          |    4 +-
 .../spark/vectorized_timestamp_funcs.q.out      |   10 +-
 .../tez/vector_non_string_partition.q.out       |    4 +-
 .../clientpositive/tez/vectorization_div0.q.out |    4 +-
 .../tez/vectorization_limit.q.out               |   10 +-
 .../vector_binary_join_groupby.q.out            |    4 +-
 .../results/clientpositive/vector_char_2.q.out  |    4 +-
 .../clientpositive/vector_char_mapjoin1.q.out   |    6 +-
 .../clientpositive/vector_char_simple.q.out     |    2 +-
 .../clientpositive/vector_coalesce.q.out        |   10 +-
 .../clientpositive/vector_coalesce_2.q.out      |    2 +-
 .../results/clientpositive/vector_count.q.out   |    8 +-
 .../clientpositive/vector_data_types.q.out      |    2 +-
 .../vector_decimal_aggregate.q.out              |    2 +-
 .../vector_decimal_expressions.q.out            |    2 +-
 .../clientpositive/vector_decimal_round.q.out   |    4 +-
 .../clientpositive/vector_decimal_round_2.q.out |    8 +-
 .../clientpositive/vector_distinct_2.q.out      |    2 +-
 .../clientpositive/vector_empty_where.q.out     |    8 +-
 .../clientpositive/vector_groupby4.q.out        |    2 +-
 .../clientpositive/vector_groupby6.q.out        |    2 +-
 .../clientpositive/vector_groupby_3.q.out       |    2 +-
 .../clientpositive/vector_groupby_reduce.q.out  |    8 +-
 .../results/clientpositive/vector_if_expr.q.out |    2 +-
 .../clientpositive/vector_include_no_sel.q.out  |    2 +-
 .../clientpositive/vector_interval_1.q.out      |   16 +-
 .../vector_interval_arithmetic.q.out            |   14 +-
 .../vector_non_string_partition.q.out           |    4 +-
 .../clientpositive/vector_orderby_5.q.out       |    2 +-
 .../clientpositive/vector_outer_join1.q.out     |    2 +-
 .../clientpositive/vector_outer_join2.q.out     |    2 +-
 .../clientpositive/vector_outer_join3.q.out     |    6 +-
 .../clientpositive/vector_outer_join4.q.out     |    2 +-
 .../results/clientpositive/vector_reduce1.q.out |    2 +-
 .../results/clientpositive/vector_reduce2.q.out |    2 +-
 .../results/clientpositive/vector_reduce3.q.out |    2 +-
 .../vector_reduce_groupby_decimal.q.out         |    2 +-
 .../clientpositive/vector_string_concat.q.out   |    2 +-
 .../clientpositive/vector_varchar_simple.q.out  |    2 +-
 .../clientpositive/vector_when_case_null.q.out  |    2 +-
 .../clientpositive/vectorization_7.q.out        |    4 +-
 .../clientpositive/vectorization_8.q.out        |    4 +-
 .../clientpositive/vectorization_div0.q.out     |    4 +-
 .../clientpositive/vectorization_limit.q.out    |    8 +-
 .../vectorization_offset_limit.q.out            |    2 +-
 .../clientpositive/vectorized_date_funcs.q.out  |    2 +-
 .../clientpositive/vectorized_mapjoin2.q.out    |    2 +-
 .../clientpositive/vectorized_timestamp.q.out   |    2 +-
 .../vectorized_timestamp_funcs.q.out            |   10 +-
 .../objectinspector/ObjectInspectorUtils.java   |    5 +
 144 files changed, 1445 insertions(+), 1444 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 71bd19a..d13e03d 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1794,8 +1794,8 @@ public class HiveConf extends Configuration {
         "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager",
         "Set to org.apache.hadoop.hive.ql.lockmgr.DbTxnManager as part of 
turning on Hive\n" +
         "transactions, which also requires appropriate settings for 
hive.compactor.initiator.on,\n" +
-        "hive.compactor.worker.threads, hive.support.concurrency (true), 
hive.enforce.bucketing\n" +
-        "(true), and hive.exec.dynamic.partition.mode (nonstrict).\n" +
+        "hive.compactor.worker.threads, hive.support.concurrency (true),\n" +
+        "and hive.exec.dynamic.partition.mode (nonstrict).\n" +
         "The default DummyTxnManager replicates pre-Hive-0.13 behavior and 
provides\n" +
         "no transactions."),
     HIVE_TXN_STRICT_LOCKING_MODE("hive.txn.strict.locking.mode", true, "In 
strict mode non-ACID\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
----------------------------------------------------------------------
diff --git 
a/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
 
b/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
index ce1ac16..e55b1c2 100644
--- 
a/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
+++ 
b/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
@@ -149,6 +149,7 @@ STAGE PLANS:
             compressed: false
             GlobalTableId: 1
             directory: ### BLOBSTORE_STAGING_PATH ###
+            Dp Sort State: PARTITION_BUCKET_SORTED
             NumFilesPerFileSink: 1
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column 
stats: NONE
             Stats Publishing Key Prefix: ### BLOBSTORE_STAGING_PATH ###

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
----------------------------------------------------------------------
diff --git 
a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
 
b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
index 1ca24bc..660cebb 100644
--- 
a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
+++ 
b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
@@ -167,6 +167,7 @@ STAGE PLANS:
             compressed: false
             GlobalTableId: 1
             directory: ### BLOBSTORE_STAGING_PATH ###
+            Dp Sort State: PARTITION_BUCKET_SORTED
             NumFilesPerFileSink: 1
             Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column 
stats: NONE
             Stats Publishing Key Prefix: ### BLOBSTORE_STAGING_PATH ###

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
index 789d2a3..e03f4b7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
@@ -18,16 +18,15 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import static 
org.apache.hadoop.hive.ql.optimizer.SortedDynPartitionOptimizer.BUCKET_NUMBER_COL_NAME;
 import static 
org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.UNIFORM;
 
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.Random;
-import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
@@ -44,15 +42,12 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.Serializer;
-import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import 
org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector.StandardUnion;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
 import org.apache.hadoop.io.BinaryComparable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -80,11 +75,13 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
   private transient ObjectInspector[] partitionObjectInspectors;
   private transient ObjectInspector[] bucketObjectInspectors;
   private transient int buckColIdxInKey;
-  private transient int buckColIdxInKeyForAcid = -1;
+  /**
+   * {@link org.apache.hadoop.hive.ql.optimizer.SortedDynPartitionOptimizer}
+   */
+  private transient int buckColIdxInKeyForSdpo = -1;
   private boolean firstRow;
   private transient int tag;
   private boolean skipTag = false;
-  private transient InspectableObject tempInspectableObject = new 
InspectableObject();
   private transient int[] valueIndex; // index for value(+ from keys, - from 
values)
 
   protected transient OutputCollector out;
@@ -143,12 +140,6 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
   // TODO: we only ever use one row of these at a time. Why do we need to 
cache multiple?
   protected transient Object[][] cachedKeys;
 
-  private StructField recIdField; // field to look for record identifier in
-  private StructField bucketField; // field to look for bucket in record 
identifier
-  private StructObjectInspector acidRowInspector; // row inspector used by 
acid options
-  private StructObjectInspector recIdInspector; // OI for the record identifier
-  private IntObjectInspector bucketInspector; // OI for the bucket field in 
the record id
-
   protected transient long numRows = 0;
   protected transient long cntr = 1;
   protected transient long logEveryNRows = 0;
@@ -186,8 +177,8 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
       keyEval = new ExprNodeEvaluator[keys.size()];
       int i = 0;
       for (ExprNodeDesc e : keys) {
-        if (e instanceof ExprNodeConstantDesc && 
("_bucket_number").equals(((ExprNodeConstantDesc)e).getValue())) {
-          buckColIdxInKeyForAcid = i;
+        if (e instanceof ExprNodeConstantDesc && 
(BUCKET_NUMBER_COL_NAME).equals(((ExprNodeConstantDesc)e).getValue())) {
+          buckColIdxInKeyForSdpo = i;
         }
         keyEval[i++] = ExprNodeEvaluatorFactory.get(e);
       }
@@ -319,20 +310,6 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
         // TODO: this is fishy - we init object inspectors based on first tag. 
We
         //       should either init for each tag, or if rowInspector doesn't 
really
         //       matter, then we can create this in ctor and get rid of 
firstRow.
-        if (conf.getWriteType() == AcidUtils.Operation.UPDATE ||
-            conf.getWriteType() == AcidUtils.Operation.DELETE) {
-          assert rowInspector instanceof StructObjectInspector :
-              "Expected rowInspector to be instance of StructObjectInspector 
but it is a " +
-                  rowInspector.getClass().getName();
-          acidRowInspector = (StructObjectInspector)rowInspector;
-          // The record identifier is always in the first column
-          recIdField = acidRowInspector.getAllStructFieldRefs().get(0);
-          recIdInspector = 
(StructObjectInspector)recIdField.getFieldObjectInspector();
-          // The bucket field is in the second position
-          bucketField = recIdInspector.getAllStructFieldRefs().get(1);
-          bucketInspector = 
(IntObjectInspector)bucketField.getFieldObjectInspector();
-        }
-
         if (isLogInfoEnabled) {
           LOG.info("keys are " + conf.getOutputKeyColumnNames() + " num 
distributions: " +
               conf.getNumDistributionKeys());
@@ -360,14 +337,9 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
       if (bucketEval != null) {
         bucketNumber = computeBucketNumber(row, conf.getNumBuckets());
         cachedKeys[0][buckColIdxInKey] = new 
Text(String.valueOf(bucketNumber));
-      } else if (conf.getWriteType() == AcidUtils.Operation.UPDATE ||
-          conf.getWriteType() == AcidUtils.Operation.DELETE) {
-        // In the non-partitioned case we still want to compute the bucket 
number for updates and
-        // deletes.
-        bucketNumber = computeBucketNumber(row, conf.getNumBuckets());
-        if (buckColIdxInKeyForAcid != -1) {
-          cachedKeys[0][buckColIdxInKeyForAcid] = new 
Text(String.valueOf(bucketNumber));
-        }
+      }
+      if (buckColIdxInKeyForSdpo != -1) {
+        cachedKeys[0][buckColIdxInKeyForSdpo] = new 
Text(String.valueOf(bucketNumber));
       }
 
       HiveKey firstKey = toHiveKey(cachedKeys[0], tag, null);
@@ -427,24 +399,11 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
   }
 
   private int computeBucketNumber(Object row, int numBuckets) throws 
HiveException {
-    if (conf.getWriteType() == AcidUtils.Operation.UPDATE ||
-        conf.getWriteType() == AcidUtils.Operation.DELETE) {
-      // We don't need to evaluate the hash code.  Instead read the bucket 
number directly from
-      // the row.  I don't need to evaluate any expressions as I know I am 
reading the ROW__ID
-      // column directly.
-      Object recIdValue = acidRowInspector.getStructFieldData(row, recIdField);
-      int buckNum = 
bucketInspector.get(recIdInspector.getStructFieldData(recIdValue, bucketField));
-      if (isLogTraceEnabled) {
-        LOG.trace("Acid choosing bucket number " + buckNum);
-      }
-      return buckNum;
-    } else {
-      Object[] bucketFieldValues = new Object[bucketEval.length];
-      for (int i = 0; i < bucketEval.length; i++) {
-        bucketFieldValues[i] = bucketEval[i].evaluate(row);
-      }
-      return ObjectInspectorUtils.getBucketNumber(bucketFieldValues, 
bucketObjectInspectors, numBuckets);
+    Object[] bucketFieldValues = new Object[bucketEval.length];
+    for (int i = 0; i < bucketEval.length; i++) {
+      bucketFieldValues[i] = bucketEval[i].evaluate(row);
     }
+    return ObjectInspectorUtils.getBucketNumber(bucketFieldValues, 
bucketObjectInspectors, numBuckets);
   }
 
   private void populateCachedDistributionKeys(Object row, int index) throws 
HiveException {
@@ -477,23 +436,23 @@ public class ReduceSinkOperator extends 
TerminalOperator<ReduceSinkDesc>
     return hash.hash(firstKey.getBytes(), firstKey.getDistKeyLength(), 0);
   }
 
+  /**
+   * For Acid Update/Delete case, we expect a single partitionEval of the form
+   * UDFToInteger(ROW__ID) and buckNum == -1 so that the result of this method
+   * is to return the bucketId extracted from ROW__ID unless it optimized by
+   * {@link org.apache.hadoop.hive.ql.optimizer.SortedDynPartitionOptimizer} 
+   */
   private int computeHashCode(Object row, int buckNum) throws HiveException {
     // Evaluate the HashCode
     int keyHashCode = 0;
     if (partitionEval.length == 0) {
-      // If no partition cols and not doing an update or delete, just 
distribute the data uniformly
+      // If no partition cols, just distribute the data uniformly
       // to provide better load balance. If the requirement is to have a 
single reducer, we should
       // set the number of reducers to 1. Use a constant seed to make the code 
deterministic.
-      // For acid operations make sure to send all records with the same key 
to the same
-      // FileSinkOperator, as the RecordUpdater interface can't manage 
multiple writers for a file.
-      if (conf.getWriteType() == AcidUtils.Operation.NOT_ACID) {
-        if (random == null) {
-          random = new Random(12345);
-        }
-        keyHashCode = random.nextInt();
-      } else {
-        keyHashCode = 1;
+      if (random == null) {
+        random = new Random(12345);
       }
+      keyHashCode = random.nextInt();
     } else {
       Object[] bucketFieldValues = new Object[partitionEval.length];
       for(int i = 0; i < partitionEval.length; i++) {

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
index 8f40998..ac1c803 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
@@ -401,9 +401,6 @@ public class BucketingSortingReduceSinkOptimizer extends 
Transform {
         return null;
       }
 
-      assert fsOp.getConf().getWriteType() == rsOp.getConf().getWriteType() :
-        "WriteType mismatch. fsOp is " + fsOp.getConf().getWriteType() +
-          "; rsOp is " + rsOp.getConf().getWriteType();
       // Don't do this optimization with updates or deletes
       if (fsOp.getConf().getWriteType() == AcidUtils.Operation.UPDATE ||
         fsOp.getConf().getWriteType() == AcidUtils.Operation.DELETE) {

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
index 926386b..76aa39f 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.ql.optimizer;
 
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -40,7 +40,7 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.Utilities.ReduceField;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.AcidUtils.Operation;
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
@@ -81,7 +82,7 @@ import com.google.common.collect.Sets;
  */
 public class SortedDynPartitionOptimizer extends Transform {
 
-  private static final String BUCKET_NUMBER_COL_NAME = "_bucket_number";
+  public static final String BUCKET_NUMBER_COL_NAME = "_bucket_number";
   @Override
   public ParseContext transform(ParseContext pCtx) throws SemanticException {
 
@@ -191,9 +192,21 @@ public class SortedDynPartitionOptimizer extends Transform 
{
         // When doing updates and deletes we always want to sort on the rowid 
because the ACID
         // reader will expect this sort order when doing reads.  So
         // ignore whatever comes from the table and enforce this sort order 
instead.
-        sortPositions = Arrays.asList(0);
-        sortOrder = Arrays.asList(1); // 1 means asc, could really use enum 
here in the thrift if
-        bucketColumns = new ArrayList<>(); // Bucketing column is already 
present in ROW__ID, which is specially handled in ReduceSink
+        sortPositions = Collections.singletonList(0);
+        sortOrder = Collections.singletonList(1); // 1 means asc, could really 
use enum here in the thrift if
+        bucketColumns = new ArrayList<>();
+        /**
+         * ROW__ID is always the 1st column of Insert representing 
Update/Delete operation
+         * (set up in {@link 
org.apache.hadoop.hive.ql.parse.UpdateDeleteSemanticAnalyzer})
+         * and we wrap it in UDFToInteger 
+         * (in {@link 
org.apache.hadoop.hive.ql.parse.SemanticAnalyzer#getPartitionColsFromBucketColsForUpdateDelete(Operator,
 boolean)})
+         * which extracts bucketId from it
+         * see {@link 
org.apache.hadoop.hive.ql.udf.UDFToInteger#evaluate(RecordIdentifier)}*/
+        ColumnInfo ci = fsParent.getSchema().getSignature().get(0);
+        if(!VirtualColumn.ROWID.getTypeInfo().equals(ci.getType())) {
+          throw new IllegalStateException("expected 1st column to be ROW__ID 
but got wrong type: " + ci.toString());
+        }
+        bucketColumns.add(new ExprNodeColumnDesc(ci));
       } else {
         if (!destTable.getSortCols().isEmpty()) {
           // Sort columns specified by table
@@ -231,7 +244,7 @@ public class SortedDynPartitionOptimizer extends Transform {
 
       // Create ReduceSink operator
       ReduceSinkOperator rsOp = getReduceSinkOp(partitionPositions, 
sortPositions, sortOrder, sortNullOrder,
-          allRSCols, bucketColumns, numBuckets, fsParent, 
fsOp.getConf().getWriteType());
+          allRSCols, bucketColumns, numBuckets, fsParent);
 
       List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>(allRSCols.size());
       List<String> colNames = new ArrayList<String>();
@@ -247,7 +260,7 @@ public class SortedDynPartitionOptimizer extends Transform {
         }
       }
       RowSchema selRS = new RowSchema(fsParent.getSchema());
-      if (!bucketColumns.isEmpty() || fsOp.getConf().getWriteType() == 
Operation.DELETE || fsOp.getConf().getWriteType() == Operation.UPDATE) {
+      if (!bucketColumns.isEmpty()) {
         descs.add(new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, 
ReduceField.KEY.toString()+".'"+BUCKET_NUMBER_COL_NAME+"'", null, false));
         colNames.add("'"+BUCKET_NUMBER_COL_NAME+"'");
         ColumnInfo ci = new ColumnInfo(BUCKET_NUMBER_COL_NAME, 
TypeInfoFactory.stringTypeInfo, selRS.getSignature().get(0).getTabAlias(), 
true, true);
@@ -268,7 +281,7 @@ public class SortedDynPartitionOptimizer extends Transform {
 
       // Set if partition sorted or partition bucket sorted
       fsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_SORTED);
-      if (bucketColumns.size() > 0 || fsOp.getConf().getWriteType() == 
Operation.DELETE || fsOp.getConf().getWriteType() == Operation.UPDATE) {
+      if (!bucketColumns.isEmpty()) {
         
fsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_BUCKET_SORTED);
       }
 
@@ -428,7 +441,7 @@ public class SortedDynPartitionOptimizer extends Transform {
     public ReduceSinkOperator getReduceSinkOp(List<Integer> partitionPositions,
         List<Integer> sortPositions, List<Integer> sortOrder, List<Integer> 
sortNullOrder,
         ArrayList<ExprNodeDesc> allCols, ArrayList<ExprNodeDesc> 
bucketColumns, int numBuckets,
-        Operator<? extends OperatorDesc> parent, AcidUtils.Operation 
writeType) throws SemanticException {
+        Operator<? extends OperatorDesc> parent) throws SemanticException {
 
       // Order of KEY columns
       // 1) Partition columns
@@ -441,7 +454,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       int numPartAndBuck = partitionPositions.size();
 
       keyColsPosInVal.addAll(partitionPositions);
-      if (!bucketColumns.isEmpty() || writeType == Operation.DELETE || 
writeType == Operation.UPDATE) {
+      if (!bucketColumns.isEmpty()) {
         keyColsPosInVal.add(-1);
         numPartAndBuck += 1;
       }
@@ -450,7 +463,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       // by default partition and bucket columns are sorted in ascending order
       Integer order = 1;
       if (sortOrder != null && !sortOrder.isEmpty()) {
-        if (sortOrder.get(0).intValue() == 0) {
+        if (sortOrder.get(0) == 0) {
           order = 0;
         }
       }
@@ -461,7 +474,7 @@ public class SortedDynPartitionOptimizer extends Transform {
 
       String orderStr = "";
       for (Integer i : newSortOrder) {
-        if(i.intValue() == 1) {
+        if(i == 1) {
           orderStr += "+";
         } else {
           orderStr += "-";
@@ -472,7 +485,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       // nulls come first; otherwise nulls come last
       Integer nullOrder = order == 1 ? 0 : 1;
       if (sortNullOrder != null && !sortNullOrder.isEmpty()) {
-        if (sortNullOrder.get(0).intValue() == 0) {
+        if (sortNullOrder.get(0) == 0) {
           nullOrder = 0;
         } else {
           nullOrder = 1;
@@ -485,7 +498,7 @@ public class SortedDynPartitionOptimizer extends Transform {
 
       String nullOrderStr = "";
       for (Integer i : newSortNullOrder) {
-        if(i.intValue() == 0) {
+        if(i == 0) {
           nullOrderStr += "a";
         } else {
           nullOrderStr += "z";
@@ -563,7 +576,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       // Number of reducers is set to default (-1)
       ReduceSinkDesc rsConf = new ReduceSinkDesc(keyCols, keyCols.size(), 
valCols,
           keyColNames, distinctColumnIndices, valColNames, -1, partCols, -1, 
keyTable,
-          valueTable, writeType);
+          valueTable);
       rsConf.setBucketCols(bucketColumns);
       rsConf.setNumBuckets(numBuckets);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
index 7a4f22a..7670dc1 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
@@ -170,7 +170,7 @@ public class SortedDynPartitionTimeGranularityOptimizer 
extends Transform {
       List<Integer> sortNullOrder = new ArrayList<Integer>(1);
       sortNullOrder.add(0); // nulls first
       ReduceSinkOperator rsOp = getReduceSinkOp(keyPositions, sortOrder,
-          sortNullOrder, allRSCols, granularitySelOp, 
fsOp.getConf().getWriteType());
+          sortNullOrder, allRSCols, granularitySelOp);
 
       // Create backtrack SelectOp
       List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>(allRSCols.size());
@@ -295,8 +295,8 @@ public class SortedDynPartitionTimeGranularityOptimizer 
extends Transform {
     }
 
     private ReduceSinkOperator getReduceSinkOp(List<Integer> keyPositions, 
List<Integer> sortOrder,
-        List<Integer> sortNullOrder, ArrayList<ExprNodeDesc> allCols, 
Operator<? extends OperatorDesc> parent,
-        AcidUtils.Operation writeType) throws SemanticException {
+        List<Integer> sortNullOrder, ArrayList<ExprNodeDesc> allCols, 
Operator<? extends OperatorDesc> parent
+    ) throws SemanticException {
 
       ArrayList<ExprNodeDesc> keyCols = Lists.newArrayList();
       // we will clone here as RS will update bucket column key with its
@@ -353,7 +353,7 @@ public class SortedDynPartitionTimeGranularityOptimizer 
extends Transform {
       // Number of reducers is set to default (-1)
       ReduceSinkDesc rsConf = new ReduceSinkDesc(keyCols, keyCols.size(), 
valCols,
           keyColNames, distinctColumnIndices, valColNames, -1, partCols, -1, 
keyTable,
-          valueTable, writeType);
+          valueTable);
 
       ArrayList<ColumnInfo> signature = new ArrayList<>();
       for (int index = 0; index < parent.getSchema().getSignature().size(); 
index++) {

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index fadbc20..f09bfa4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -2980,11 +2980,7 @@ public class Vectorizer implements PhysicalPlanResolver {
         HiveConf.ConfVars.HIVE_VECTORIZATION_REDUCESINK_NEW_ENABLED);
 
     String engine = HiveConf.getVar(hiveConf, 
HiveConf.ConfVars.HIVE_EXECUTION_ENGINE);
-
-    boolean acidChange =
-        desc.getWriteType() == AcidUtils.Operation.UPDATE ||
-            desc.getWriteType() == AcidUtils.Operation.DELETE;
-
+    
     boolean hasBuckets = desc.getBucketCols() != null && 
!desc.getBucketCols().isEmpty();
 
     boolean hasTopN = desc.getTopN() >= 0;
@@ -3004,7 +3000,6 @@ public class Vectorizer implements PhysicalPlanResolver {
     // Remember the condition variables for EXPLAIN regardless.
     
vectorDesc.setIsVectorizationReduceSinkNativeEnabled(isVectorizationReduceSinkNativeEnabled);
     vectorDesc.setEngine(engine);
-    vectorDesc.setAcidChange(acidChange);
     vectorDesc.setHasBuckets(hasBuckets);
     vectorDesc.setHasTopN(hasTopN);
     vectorDesc.setUseUniformHash(useUniformHash);
@@ -3015,7 +3010,6 @@ public class Vectorizer implements PhysicalPlanResolver {
     // Many restrictions.
     if (!isVectorizationReduceSinkNativeEnabled ||
         !isTezOrSpark ||
-        acidChange ||
         hasBuckets ||
         hasTopN ||
         !useUniformHash ||

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index 5cc1c45..4716adc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -438,7 +438,10 @@ public class FileSinkDesc extends AbstractOperatorDesc {
   public DPSortState getDpSortState() {
     return dpSortState;
   }
-
+  @Explain(displayName = "Dp Sort State")
+  public String getDpSortStateString() {
+    return getDpSortState() == DPSortState.NONE ? null : 
getDpSortState().toString();
+  }
   public void setDpSortState(DPSortState dpSortState) {
     this.dpSortState = dpSortState;
   }
@@ -450,7 +453,10 @@ public class FileSinkDesc extends AbstractOperatorDesc {
   public AcidUtils.Operation getWriteType() {
     return writeType;
   }
-
+  @Explain(displayName = "Write Type")
+  public String getWriteTypeString() {
+    return getWriteType() == AcidUtils.Operation.NOT_ACID ? null : 
getWriteType().toString();
+  }
   public void setTransactionId(long id) {
     txnId = id;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 05d2c81..14f2a12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -765,7 +765,7 @@ public final class PlanUtils {
     return new ReduceSinkDesc(keyCols, numKeys, valueCols, outputKeyCols,
         distinctColIndices, outputValCols,
         tag, partitionCols, numReducers, keyTable,
-        valueTable, writeType);
+        valueTable);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
index d77a223..9ae30ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
@@ -21,13 +21,11 @@ package org.apache.hadoop.hive.ql.plan;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 import org.apache.hadoop.hive.ql.plan.Explain.Vectorization;
 import org.apache.hadoop.hive.ql.plan.VectorReduceSinkDesc.ReduceSinkKeyType;
@@ -123,9 +121,6 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
   // Is reducer auto-parallelism unset (FIXED, UNIFORM, PARALLEL)
   private EnumSet<ReducerTraits> reduceTraits = 
EnumSet.of(ReducerTraits.UNSET);
 
-  // Write type, since this needs to calculate buckets differently for updates 
and deletes
-  private AcidUtils.Operation writeType;
-
   // whether this RS is deduplicated
   private transient boolean isDeduplicated = false;
 
@@ -144,8 +139,7 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
       List<List<Integer>> distinctColumnIndices,
       ArrayList<String> outputValueColumnNames, int tag,
       ArrayList<ExprNodeDesc> partitionCols, int numReducers,
-      final TableDesc keySerializeInfo, final TableDesc valueSerializeInfo,
-      AcidUtils.Operation writeType) {
+      final TableDesc keySerializeInfo, final TableDesc valueSerializeInfo) {
     this.keyCols = keyCols;
     this.numDistributionKeys = numDistributionKeys;
     this.valueCols = valueCols;
@@ -159,7 +153,6 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     this.distinctColumnIndices = distinctColumnIndices;
     this.setNumBuckets(-1);
     this.setBucketCols(null);
-    this.writeType = writeType;
     this.vectorDesc = null;
   }
 
@@ -472,10 +465,6 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     }
   }
 
-  public AcidUtils.Operation getWriteType() {
-    return writeType;
-  }
-
   public boolean isDeduplicated() {
     return isDeduplicated;
   }
@@ -545,9 +534,6 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
               engineInSupported,
               engineInSupportedCondName),
           new VectorizationCondition(
-              !vectorReduceSinkDesc.getAcidChange(),
-              "Not ACID UPDATE or DELETE"),
-          new VectorizationCondition(
               !vectorReduceSinkDesc.getHasBuckets(),
               "No buckets"),
           new VectorizationCondition(

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
index 288a440..2eb44b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
@@ -64,7 +64,6 @@ public class VectorReduceSinkDesc extends AbstractVectorDesc  
{
 
   private boolean isVectorizationReduceSinkNativeEnabled;
   private String engine;
-  private boolean acidChange;
   private boolean hasBuckets;
   private boolean hasTopN;
   private boolean useUniformHash;
@@ -87,12 +86,6 @@ public class VectorReduceSinkDesc extends AbstractVectorDesc 
 {
   public String getEngine() {
     return engine;
   }
-  public void setAcidChange(boolean acidChange) {
-    this.acidChange = acidChange;
-  }
-  public boolean getAcidChange() {
-    return acidChange;
-  }
   public void setHasBuckets(boolean hasBuckets) {
     this.hasBuckets = hasBuckets;
   }  

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java 
b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
index 1aef7ac..05b6fc4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
@@ -266,7 +266,7 @@ public class CompactorMR {
 
     if (parsedDeltas.size() == 0 && dir.getOriginalFiles() == null) {
       // Skip compaction if there's no delta files AND there's no original 
files
-      LOG.error("No delta files or original files found to compact in " + 
sd.getLocation());
+      LOG.error("No delta files or original files found to compact in " + 
sd.getLocation() + " for compactionId=" + ci.id);
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/queries/clientpositive/dynpart_sort_optimization_acid.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/dynpart_sort_optimization_acid.q 
b/ql/src/test/queries/clientpositive/dynpart_sort_optimization_acid.q
index 414d070..bf8cbc8 100644
--- a/ql/src/test/queries/clientpositive/dynpart_sort_optimization_acid.q
+++ b/ql/src/test/queries/clientpositive/dynpart_sort_optimization_acid.q
@@ -7,131 +7,131 @@ set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.optimize.sort.dynamic.partition=false;
 
 -- single level partition, sorted dynamic partition disabled
-drop table acid;
-CREATE TABLE acid(key string, value string) PARTITIONED BY(ds string) 
CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
-insert into table acid partition(ds)  select key,value,ds from srcpart;
+drop table if exists acid_part;
+CREATE TABLE acid_part(key string, value string) PARTITIONED BY(ds string) 
CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
+insert into table acid_part partition(ds)  select key,value,ds from srcpart;
 -- explicitly set statistics to avoid flakiness
-alter table acid partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
-select count(*) from acid where ds='2008-04-08';
+alter table acid_part partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
+select count(*) from acid_part where ds='2008-04-08';
 
-insert into table acid partition(ds='2008-04-08') values("foo", "bar");
-select count(*) from acid where ds='2008-04-08';
+insert into table acid_part partition(ds='2008-04-08') values("foo", "bar");
+select count(*) from acid_part where ds='2008-04-08';
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08';
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08';
-select count(*) from acid where ds='2008-04-08';
+explain update acid_part set value = 'bar' where key = 'foo' and 
ds='2008-04-08';
+update acid_part set value = 'bar' where key = 'foo' and ds='2008-04-08';
+select count(*) from acid_part where ds='2008-04-08';
 
-explain update acid set value = 'bar' where key = 'foo' and ds in 
('2008-04-08');
-update acid set value = 'bar' where key = 'foo' and ds in ('2008-04-08');
-select count(*) from acid where ds in ('2008-04-08');
+explain update acid_part set value = 'bar' where key = 'foo' and ds in 
('2008-04-08');
+update acid_part set value = 'bar' where key = 'foo' and ds in ('2008-04-08');
+select count(*) from acid_part where ds in ('2008-04-08');
 
-delete from acid where key = 'foo' and ds='2008-04-08';
-select count(*) from acid where ds='2008-04-08';
+delete from acid_part where key = 'foo' and ds='2008-04-08';
+select count(*) from acid_part where ds='2008-04-08';
 
 set hive.optimize.sort.dynamic.partition=true;
 
 -- single level partition, sorted dynamic partition enabled
-drop table acid;
-CREATE TABLE acid(key string, value string) PARTITIONED BY(ds string) 
CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
-insert into table acid partition(ds)  select key,value,ds from srcpart;
-alter table acid partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
-select count(*) from acid where ds='2008-04-08';
+drop table if exists acid_part_sdpo;
+CREATE TABLE acid_part_sdpo(key string, value string) PARTITIONED BY(ds 
string) CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
+insert into table acid_part_sdpo partition(ds)  select key,value,ds from 
srcpart;
+alter table acid_part_sdpo partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
+select count(*) from acid_part_sdpo where ds='2008-04-08';
 
-insert into table acid partition(ds='2008-04-08') values("foo", "bar");
-select count(*) from acid where ds='2008-04-08';
+insert into table acid_part_sdpo partition(ds='2008-04-08') values("foo", 
"bar");
+select count(*) from acid_part_sdpo where ds='2008-04-08';
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08';
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08';
-select count(*) from acid where ds='2008-04-08';
+explain update acid_part_sdpo set value = 'bar' where key = 'foo' and 
ds='2008-04-08';
+update acid_part_sdpo set value = 'bar' where key = 'foo' and ds='2008-04-08';
+select count(*) from acid_part_sdpo where ds='2008-04-08';
 
-explain update acid set value = 'bar' where key = 'foo' and ds in 
('2008-04-08');
-update acid set value = 'bar' where key = 'foo' and ds in ('2008-04-08');
-select count(*) from acid where ds in ('2008-04-08');
+explain update acid_part_sdpo set value = 'bar' where key = 'foo' and ds in 
('2008-04-08');
+update acid_part_sdpo set value = 'bar' where key = 'foo' and ds in 
('2008-04-08');
+select count(*) from acid_part_sdpo where ds in ('2008-04-08');
 
-delete from acid where key = 'foo' and ds='2008-04-08';
-select count(*) from acid where ds='2008-04-08';
+delete from acid_part_sdpo where key = 'foo' and ds='2008-04-08';
+select count(*) from acid_part_sdpo where ds='2008-04-08';
 
 set hive.optimize.sort.dynamic.partition=false;
 
 -- 2 level partition, sorted dynamic partition disabled
-drop table acid;
-CREATE TABLE acid(key string, value string) PARTITIONED BY(ds string, hr int) 
CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
-insert into table acid partition(ds,hr)  select * from srcpart;
-alter table acid partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
-select count(*) from acid where ds='2008-04-08' and hr=11;
+drop table if exists acid_2L_part;
+CREATE TABLE acid_2L_part(key string, value string) PARTITIONED BY(ds string, 
hr int) CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
+insert into table acid_2L_part partition(ds,hr)  select * from srcpart;
+alter table acid_2L_part partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
+select count(*) from acid_2L_part where ds='2008-04-08' and hr=11;
 
-insert into table acid partition(ds='2008-04-08',hr=11) values("foo", "bar");
-select count(*) from acid where ds='2008-04-08' and hr=11;
+insert into table acid_2L_part partition(ds='2008-04-08',hr=11) values("foo", 
"bar");
+select count(*) from acid_2L_part where ds='2008-04-08' and hr=11;
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr=11;
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11;
-select count(*) from acid where ds='2008-04-08' and hr=11;
+explain update acid_2L_part set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr=11;
+update acid_2L_part set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr=11;
+select count(*) from acid_2L_part where ds='2008-04-08' and hr=11;
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr>=11;
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11;
-select count(*) from acid where ds='2008-04-08' and hr>=11;
+explain update acid_2L_part set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr>=11;
+update acid_2L_part set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr>=11;
+select count(*) from acid_2L_part where ds='2008-04-08' and hr>=11;
 
-delete from acid where key = 'foo' and ds='2008-04-08' and hr=11;
-select count(*) from acid where ds='2008-04-08' and hr=11;
+delete from acid_2L_part where key = 'foo' and ds='2008-04-08' and hr=11;
+select count(*) from acid_2L_part where ds='2008-04-08' and hr=11;
 
 -- test with bucketing column not in select list
 explain
-delete from acid where value = 'bar';
-delete from acid where value = 'bar';
-select count(*) from acid;
+delete from acid_2L_part where value = 'bar';
+delete from acid_2L_part where value = 'bar';
+select count(*) from acid_2L_part;
 
 set hive.optimize.sort.dynamic.partition=true;
 
 -- 2 level partition, sorted dynamic partition enabled
-drop table acid;
-CREATE TABLE acid(key string, value string) PARTITIONED BY(ds string, hr int) 
CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
-insert into table acid partition(ds,hr)  select * from srcpart;
-alter table acid partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
-select count(*) from acid where ds='2008-04-08' and hr=11;
+drop table if exists acid_2L_part_sdpo;
+CREATE TABLE acid_2L_part_sdpo(key string, value string) PARTITIONED BY(ds 
string, hr int) CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
+insert into table acid_2L_part_sdpo partition(ds,hr)  select * from srcpart;
+alter table acid_2L_part_sdpo partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
+select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr=11;
 
-insert into table acid partition(ds='2008-04-08',hr=11) values("foo", "bar");
-select count(*) from acid where ds='2008-04-08' and hr=11;
+insert into table acid_2L_part_sdpo partition(ds='2008-04-08',hr=11) 
values("foo", "bar");
+select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr=11;
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr=11;
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11;
-select count(*) from acid where ds='2008-04-08' and hr=11;
+explain update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr=11;
+update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr=11;
+select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr=11;
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr>=11;
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11;
-select count(*) from acid where ds='2008-04-08' and hr>=11;
+explain update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr>=11;
+update acid_2L_part_sdpo set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr>=11;
+select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr>=11;
 
-delete from acid where key = 'foo' and ds='2008-04-08' and hr=11;
-select count(*) from acid where ds='2008-04-08' and hr=11;
+delete from acid_2L_part_sdpo where key = 'foo' and ds='2008-04-08' and hr=11;
+select count(*) from acid_2L_part_sdpo where ds='2008-04-08' and hr=11;
 
 -- test with bucketing column not in select list
 explain
-delete from acid where value = 'bar';
-delete from acid where value = 'bar';
-select count(*) from acid;
+delete from acid_2L_part_sdpo where value = 'bar';
+delete from acid_2L_part_sdpo where value = 'bar';
+select count(*) from acid_2L_part_sdpo;
 
 
 set hive.optimize.sort.dynamic.partition=true;
 set hive.optimize.constant.propagation=false;
 
 -- 2 level partition, sorted dynamic partition enabled, constant propagation 
disabled
-drop table acid;
-CREATE TABLE acid(key string, value string) PARTITIONED BY(ds string, hr int) 
CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES 
('transactional'='true');
-insert into table acid partition(ds,hr)  select * from srcpart;
-alter table acid partition(ds='2008-04-08') update statistics 
set('numRows'='1600', 'rawDataSize'='18000');
-select count(*) from acid where ds='2008-04-08' and hr=11;
+drop table if exists acid_2L_part_sdpo_no_cp;
+CREATE TABLE acid_2L_part_sdpo_no_cp(key string, value string) PARTITIONED 
BY(ds string, hr int) CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC 
TBLPROPERTIES ('transactional'='true');
+insert into table acid_2L_part_sdpo_no_cp partition(ds,hr)  select * from 
srcpart;
+alter table acid_2L_part_sdpo_no_cp partition(ds='2008-04-08') update 
statistics set('numRows'='1600', 'rawDataSize'='18000');
+select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr=11;
 
-insert into table acid partition(ds='2008-04-08',hr=11) values("foo", "bar");
-select count(*) from acid where ds='2008-04-08' and hr=11;
+insert into table acid_2L_part_sdpo_no_cp partition(ds='2008-04-08',hr=11) 
values("foo", "bar");
+select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr=11;
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr=11;
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11;
-select count(*) from acid where ds='2008-04-08' and hr=11;
+explain update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr=11;
+update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr=11;
+select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr=11;
 
-explain update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' 
and hr>=11;
-update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11;
-select count(*) from acid where ds='2008-04-08' and hr>=11;
+explain update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr>=11;
+update acid_2L_part_sdpo_no_cp set value = 'bar' where key = 'foo' and 
ds='2008-04-08' and hr>=11;
+select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr>=11;
 
-delete from acid where key = 'foo' and ds='2008-04-08' and hr=11;
-select count(*) from acid where ds='2008-04-08' and hr=11;
+delete from acid_2L_part_sdpo_no_cp where key = 'foo' and ds='2008-04-08' and 
hr=11;
+select count(*) from acid_2L_part_sdpo_no_cp where ds='2008-04-08' and hr=11;
 
 set hive.optimize.sort.dynamic.partition=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/results/clientpositive/autoColumnStats_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/autoColumnStats_4.q.out 
b/ql/src/test/results/clientpositive/autoColumnStats_4.q.out
index c7b9b4f..3ae2f20 100644
--- a/ql/src/test/results/clientpositive/autoColumnStats_4.q.out
+++ b/ql/src/test/results/clientpositive/autoColumnStats_4.q.out
@@ -110,6 +110,7 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                 serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                 name: default.acid_dtt
+            Write Type: INSERT
           Select Operator
             expressions: _col0 (type: int), _col1 (type: varchar(128))
             outputColumnNames: a, b

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out 
b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out
index 0b6e992..76d0b7b 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid2.q.out
@@ -47,6 +47,7 @@ STAGE PLANS:
           Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE 
Column stats: NONE
           File Output Operator
             compressed: false
+            Dp Sort State: PARTITION_BUCKET_SORTED
             Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE 
Column stats: NONE
             table:
                 input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out 
b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
index ef5a4b2..829f28f 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
@@ -174,6 +174,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: DELETE
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -189,6 +190,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: UPDATE
         Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
@@ -228,6 +230,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: INSERT
         Reducer 8 
             Execution mode: llap
             Reduce Operator Tree:
@@ -398,6 +401,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: INSERT
         Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
@@ -639,6 +643,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: DELETE
         Reducer 5 
             Execution mode: llap
             Reduce Operator Tree:
@@ -654,6 +659,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: UPDATE
         Reducer 6 
             Execution mode: llap
             Reduce Operator Tree:
@@ -693,6 +699,7 @@ STAGE PLANS:
                       output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.acidtbl
+                  Write Type: INSERT
         Reducer 8 
             Execution mode: llap
             Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out 
b/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
index eaa394d..0e16ff1 100644
--- 
a/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
@@ -198,6 +198,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -290,6 +291,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 2960 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 10 Data size: 2960 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -363,6 +365,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -435,6 +438,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -575,6 +579,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -667,6 +672,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 2960 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 10 Data size: 2960 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -740,6 +746,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -812,6 +819,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
@@ -1426,6 +1434,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1520,6 +1529,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 2960 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 10 Data size: 2960 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1676,6 +1686,7 @@ STAGE PLANS:
                   Statistics: Num rows: 524 Data size: 155436 Basic stats: 
COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Dp Sort State: PARTITION_SORTED
                     Statistics: Num rows: 524 Data size: 155436 Basic stats: 
COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2133,6 +2144,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 1048 Data size: 310873 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out 
b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
index 95a4e0f..1ef0740 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
@@ -155,6 +155,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -247,6 +248,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 240 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 10 Data size: 240 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -320,6 +322,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -392,6 +395,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -532,6 +536,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -624,6 +629,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 240 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 10 Data size: 240 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -697,6 +703,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -769,6 +776,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1383,6 +1391,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1477,6 +1486,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 240 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 10 Data size: 240 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1633,6 +1643,7 @@ STAGE PLANS:
                   Statistics: Num rows: 2221 Data size: 53305 Basic stats: 
COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Dp Sort State: PARTITION_SORTED
                     Statistics: Num rows: 2221 Data size: 53305 Basic stats: 
COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2090,6 +2101,7 @@ STAGE PLANS:
                 Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
                   Statistics: Num rows: 4442 Data size: 106611 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2464,6 +2476,7 @@ STAGE PLANS:
                 Statistics: Num rows: 429 Data size: 53255 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 429 Data size: 53255 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2538,6 +2551,7 @@ STAGE PLANS:
                 Statistics: Num rows: 429 Data size: 53255 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 429 Data size: 53255 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2612,6 +2626,7 @@ STAGE PLANS:
                 Statistics: Num rows: 429 Data size: 53255 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 429 Data size: 53255 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2686,6 +2701,7 @@ STAGE PLANS:
                 Statistics: Num rows: 214 Data size: 26565 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 214 Data size: 26565 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2760,6 +2776,7 @@ STAGE PLANS:
                 Statistics: Num rows: 214 Data size: 26565 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 214 Data size: 26565 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -2834,6 +2851,7 @@ STAGE PLANS:
                 Statistics: Num rows: 214 Data size: 26565 Basic stats: 
COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 214 Data size: 26565 Basic stats: 
COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/917221e8/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out 
b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
index 41a7709..667d980 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization2.q.out
@@ -120,6 +120,7 @@ STAGE PLANS:
                   Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE 
Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Dp Sort State: PARTITION_SORTED
                     Statistics: Num rows: 2 Data size: 24 Basic stats: 
COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -357,6 +358,7 @@ STAGE PLANS:
                 Statistics: Num rows: 5 Data size: 60 Basic stats: COMPLETE 
Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 5 Data size: 60 Basic stats: COMPLETE 
Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat

Reply via email to